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/08/25 03:24:08 UTC

[GitHub] [pulsar-client-go] jonyhy96 opened a new pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

jonyhy96 opened a new pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360


   Fixes #257
   
   ### Motivation
   
   Once the connection is closed the reconnectToBroker logic of both producer and consumer will try to reconnect to the broker infinatly.
   
   ### Modifications
   
   Add a field in the Options represent the max number of retries and not break current behavior if this field is not fullfilled.
   
   ### Verifying this change
   
   - [ ] Make sure that the change passes the CI checks.
   
   
   This change is a trivial rework / code cleanup without any test coverage.
   
   
   ### Does this pull request potentially affect one of the following parts:
   
   *If `yes` was chosen, please highlight the changes*
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API: (no)
     - The schema: (yes)
     - The default values of configurations: (no)
     - The wire protocol: (no)
   
   ### Documentation
   
     - Does this pull request introduce a new feature? (yes)
     - If yes, how is the feature documented? (GoDocs)
     - If a feature is not applicable for documentation, explain why?
     - If a feature is not documented yet in this PR, please create a followup issue for adding the documentation
   


----------------------------------------------------------------
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-client-go] shohi commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r477011490



##########
File path: pulsar/consumer.go
##########
@@ -140,6 +140,9 @@ type ConsumerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ConsumerInterceptor interface.
 	Interceptors ConsumerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: ultimate)
+	MaxReconnectToBroker *uint

Review comment:
       invite @wolfstudy to 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.

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



[GitHub] [pulsar-client-go] wolfstudy commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r502859913



##########
File path: pulsar/producer.go
##########
@@ -138,6 +138,9 @@ type ProducerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ProducerInterceptor interface
 	Interceptors ProducerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: no retry)
+	MaxReconnectToBroker int

Review comment:
       @jonyhy96 Can we not modify the original default behavior, and we will always retry by default. When we set the maximum number of retries, we will turn off the retry logic after reaching the specified threshold.
   
   Because the current behavior will break the backward compatibility of the version




----------------------------------------------------------------
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-client-go] shohi commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r476253761



##########
File path: pulsar/consumer.go
##########
@@ -140,6 +140,9 @@ type ConsumerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ConsumerInterceptor interface.
 	Interceptors ConsumerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: ultimate)
+	MaxReconnectToBroker *uint

Review comment:
       I see your point, but it's common to use int for this kind of configuration, such as [go-redis](https://github.com/go-redis/redis/blob/master/options.go#L59) / [nats.go](https://github.com/nats-io/nats.go/blob/master/nats.go#L266), we can make the following convention for the config to cope with different scenarios,
   * 0 for no-try
   * negative number for retry forever
   * positive number for retry attempts
   
   As the current library is still in pre-1.0, i think it's ok to make a break 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.

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



[GitHub] [pulsar-client-go] shohi commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r476253761



##########
File path: pulsar/consumer.go
##########
@@ -140,6 +140,9 @@ type ConsumerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ConsumerInterceptor interface.
 	Interceptors ConsumerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: ultimate)
+	MaxReconnectToBroker *uint

Review comment:
       I see your point, but it's common to use int for this kind of configuration, such as [go-redis](https://github.com/go-redis/redis/blob/master/options.go#L59) / [nats.go](https://github.com/nats-io/nats.go/blob/master/nats.go#L266), we can make the following convention for the config to cope with different scenarios,
   * 0 for no retry
   * negative number for retry forever
   * positive number to set retry attempts
   
   As the current library is still in pre-1.0, i think it's ok to make a break 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.

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



[GitHub] [pulsar-client-go] wolfstudy merged pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
wolfstudy merged pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360


   


----------------------------------------------------------------
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-client-go] jonyhy96 commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
jonyhy96 commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r503058339



##########
File path: pulsar/producer.go
##########
@@ -138,6 +138,9 @@ type ProducerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ProducerInterceptor interface
 	Interceptors ProducerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: no retry)
+	MaxReconnectToBroker int

Review comment:
       > We can consider setting -1 as the default value, and then check whether MaxReconnectToBroker is set, and if so, use the user-defined value.
   
   I understand,but how can we determine whether MaxReconnectToBroker is set or not when it's default value is meaningful. e.g. user set this option to 0.
   
   > At present, I see that you have added the option of MaxReconnectToBroker to the relevant test cases. This may not be very user-friendly. We need to ensure that the interface is as simple as possible.
   
   I totally agree, how about we use the origin commit of this pr to implement this feature?




----------------------------------------------------------------
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-client-go] jonyhy96 commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
jonyhy96 commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r477016152



##########
File path: pulsar/consumer.go
##########
@@ -140,6 +140,9 @@ type ConsumerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ConsumerInterceptor interface.
 	Interceptors ConsumerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: ultimate)
+	MaxReconnectToBroker *uint

Review comment:
       > invite @wolfstudy to review?
   
   Sounds great!




----------------------------------------------------------------
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-client-go] jonyhy96 commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
jonyhy96 commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r476237458



##########
File path: pulsar/producer_partition.go
##########
@@ -236,8 +236,18 @@ func (p *partitionProducer) ConnectionClosed() {
 }
 
 func (p *partitionProducer) reconnectToBroker() {
-	backoff := internal.Backoff{}
-	for {
+	var (
+		maxRetry int
+		backoff  = internal.Backoff{}
+	)
+
+	if p.options.MaxReconnectToBroker == nil {
+		maxRetry = -1
+	} else {
+		maxRetry = int(*p.options.MaxReconnectToBroker)
+	}

Review comment:
       > The handling for uninitialized maxRetry seems not correct, it may be better to extract the retry test into a separate function, e.g `shouldRetry`.
   
   ```go
   func (p *partitionProducer) limitRetry() bool {
    return p.options.MaxReconnectToBroker != nil
   }
   
   func (p *partitionProducer) reconnectToBroker() {
   ...
    var maxRetry = -1
    if p.limitRetry() {
      maxRetry = int(*p.options.MaxReconnectToBroker)  
    }
   ...
   }
   ```
   
   Do you mean something like 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-client-go] jonyhy96 commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
jonyhy96 commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r508956246



##########
File path: pulsar/producer.go
##########
@@ -138,6 +138,9 @@ type ProducerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ProducerInterceptor interface
 	Interceptors ProducerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: no retry)
+	MaxReconnectToBroker int

Review comment:
       > @jonyhy96 Sorry for the reply later, i think we should reset this pull request to the original commit for backward compatibility.
   
   Already reset this pr to original commit.
   By the way, the go.mod file seems a little bit dirty,you can reproduce this by run comamnd `go mod tidy` locally.
   
   ```shell
   ➜  pulsar-client-go git:(feat-max-reconnect-to-broker) go mod tidy -v        
   unused github.com/modern-go/concurrent
   unused github.com/modern-go/reflect2
   ```
   
   perhaps add script below to ci for check if go.mod has been commit correctly
   
   ```shell
   go mod tidy
   git diff --exit-code -- go.mod go.sum
   ```




----------------------------------------------------------------
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-client-go] jonyhy96 commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
jonyhy96 commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r476231877



##########
File path: pulsar/consumer.go
##########
@@ -140,6 +140,9 @@ type ConsumerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ConsumerInterceptor interface.
 	Interceptors ConsumerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: ultimate)
+	MaxReconnectToBroker *uint

Review comment:
       > IMO using `int` type is more friendly, the default value `0` represents unlimited.
   
   How aboud `0` represents no retry but nil means retry ultimate times? Then this changes won't break current behavior of reconnectToBroker and users can set this to `0` to disable the retry logic.




----------------------------------------------------------------
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-client-go] jonyhy96 commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
jonyhy96 commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r476255226



##########
File path: pulsar/producer_partition.go
##########
@@ -236,8 +236,18 @@ func (p *partitionProducer) ConnectionClosed() {
 }
 
 func (p *partitionProducer) reconnectToBroker() {
-	backoff := internal.Backoff{}
-	for {
+	var (
+		maxRetry int
+		backoff  = internal.Backoff{}
+	)
+
+	if p.options.MaxReconnectToBroker == nil {
+		maxRetry = -1
+	} else {
+		maxRetry = int(*p.options.MaxReconnectToBroker)
+	}

Review comment:
       Great, This looks more clear. 
   I think it would be better if we decide the type of p.opts.MaxReconnectToBroker first.
   Besides, i think `shouldRetry()` logic may be a little bit slower than the origin one, we can reduce the `shouldRetry()` call times by move some common logic outside of the for loop. 




----------------------------------------------------------------
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-client-go] shohi commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r476184448



##########
File path: pulsar/consumer.go
##########
@@ -140,6 +140,9 @@ type ConsumerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ConsumerInterceptor interface.
 	Interceptors ConsumerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: ultimate)
+	MaxReconnectToBroker *uint

Review comment:
       IMO using `int` type is more friendly,  the default value `0` represents unlimited.




----------------------------------------------------------------
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-client-go] wolfstudy commented on pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#issuecomment-681214778


   Thanks @jonyhy96 Here, I have only one confusion. When users use it, they need to set it on the producer and consumer at the same time, right? Can we consider exposing this parameter to the user in ClientOptions so that the user only needs to set it once


----------------------------------------------------------------
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-client-go] wolfstudy commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r502131613



##########
File path: pulsar/consumer.go
##########
@@ -140,6 +140,9 @@ type ConsumerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ConsumerInterceptor interface.
 	Interceptors ConsumerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: ultimate)
+	MaxReconnectToBroker *uint

Review comment:
       Thanks @jonyhy96,  @shohi proposal LGTM +, can you help fix 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-client-go] jonyhy96 commented on pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
jonyhy96 commented on pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#issuecomment-681302071


   > Thanks @jonyhy96 Here, I have only one confusion. When users use it, they need to set it on the producer and consumer at the same time, right? Can we consider exposing this parameter to the user in ClientOptions so that the user only needs to set it once
   
   That would be more convenience for users to use, But the user loses more detailed control over the program.
   By the way,do you mean add an option at [here](https://github.com/apache/pulsar-client-go/blob/master/pulsar/client.go#L77) and [here](https://github.com/apache/pulsar-client-go/blob/master/pulsar/client_impl.go#L40),in reconnectToBroker use p.clinet.maxReconnectToBroker to decide the retry logic?


----------------------------------------------------------------
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-client-go] wolfstudy commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r508929877



##########
File path: pulsar/producer.go
##########
@@ -138,6 +138,9 @@ type ProducerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ProducerInterceptor interface
 	Interceptors ProducerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: no retry)
+	MaxReconnectToBroker int

Review comment:
       @jonyhy96 Sorry for the reply later, i think we should reset this pull request to the original commit for backward compatibility.




----------------------------------------------------------------
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-client-go] wolfstudy commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r502131613



##########
File path: pulsar/consumer.go
##########
@@ -140,6 +140,9 @@ type ConsumerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ConsumerInterceptor interface.
 	Interceptors ConsumerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: ultimate)
+	MaxReconnectToBroker *uint

Review comment:
       Thanks @jonyhy96,  @shohi proposal LGTM +, can you help fix 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-client-go] shohi commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r476186223



##########
File path: pulsar/producer_partition.go
##########
@@ -236,8 +236,18 @@ func (p *partitionProducer) ConnectionClosed() {
 }
 
 func (p *partitionProducer) reconnectToBroker() {
-	backoff := internal.Backoff{}
-	for {
+	var (
+		maxRetry int
+		backoff  = internal.Backoff{}
+	)
+
+	if p.options.MaxReconnectToBroker == nil {
+		maxRetry = -1
+	} else {
+		maxRetry = int(*p.options.MaxReconnectToBroker)
+	}

Review comment:
       The handling for uninitialized maxRetry seems not correct, it may be better to extract the retry test into a separate function, e.g `shouldRetry`.




----------------------------------------------------------------
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-client-go] jonyhy96 commented on pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
jonyhy96 commented on pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#issuecomment-679514357


   /cc @wolfstudy 


----------------------------------------------------------------
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-client-go] jonyhy96 commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
jonyhy96 commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r502866773



##########
File path: pulsar/producer.go
##########
@@ -138,6 +138,9 @@ type ProducerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ProducerInterceptor interface
 	Interceptors ProducerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: no retry)
+	MaxReconnectToBroker int

Review comment:
       Sure, neither do I want to break the current behavior. The original commit of this pr will fulfill our request, please check out the [comments](https://github.com/apache/pulsar-client-go/pull/360#discussion_r476231877) above. @shohi may have different opinion on this due to his(her) proposal. Please take a 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.

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



[GitHub] [pulsar-client-go] jonyhy96 commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
jonyhy96 commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r476231877



##########
File path: pulsar/consumer.go
##########
@@ -140,6 +140,9 @@ type ConsumerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ConsumerInterceptor interface.
 	Interceptors ConsumerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: ultimate)
+	MaxReconnectToBroker *uint

Review comment:
       > IMO using `int` type is more friendly, the default value `0` represents unlimited.
   
   How about `0` represents no retry but nil means retry ultimate times? Then this changes won't break current behavior of reconnectToBroker and users can set this to `0` to disable the retry logic.




----------------------------------------------------------------
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-client-go] jonyhy96 commented on pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
jonyhy96 commented on pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#issuecomment-706089867


   > LGTM +1, just a little comment, @jonyhy96 please help check, thanks.
   
   Thanks for review! Already fix 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-client-go] jonyhy96 commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
jonyhy96 commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r476255226



##########
File path: pulsar/producer_partition.go
##########
@@ -236,8 +236,18 @@ func (p *partitionProducer) ConnectionClosed() {
 }
 
 func (p *partitionProducer) reconnectToBroker() {
-	backoff := internal.Backoff{}
-	for {
+	var (
+		maxRetry int
+		backoff  = internal.Backoff{}
+	)
+
+	if p.options.MaxReconnectToBroker == nil {
+		maxRetry = -1
+	} else {
+		maxRetry = int(*p.options.MaxReconnectToBroker)
+	}

Review comment:
       Great, This looks more clear. 
   I think it would be better if we decide the type of p.opts.MaxReconnectToBroker first.
   Besides, i think `shouldRetry()` logic maybe a little bit slower than the origin one, we can reduce the `shouldRetry()` call times by move some common logic outside of the for loop. 




----------------------------------------------------------------
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-client-go] jonyhy96 commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
jonyhy96 commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r476259932



##########
File path: pulsar/consumer.go
##########
@@ -140,6 +140,9 @@ type ConsumerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ConsumerInterceptor interface.
 	Interceptors ConsumerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: ultimate)
+	MaxReconnectToBroker *uint

Review comment:
       > I see your point, but it's common to use int for this kind of configuration, such as [go-redis](https://github.com/go-redis/redis/blob/master/options.go#L59) / [nats.go](https://github.com/nats-io/nats.go/blob/master/nats.go#L266), we can make the following convention for the config to cope with different scenarios,
   > 
   > * 0 for no retry
   > * negative number for retry forever
   > * positive number to set retry attempts
   > 
   > As the current library is still in pre-1.0, i think it's ok to make a break change.
   
   Totally agree, Do we need to ask for other maintainer's opinions?




----------------------------------------------------------------
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-client-go] jonyhy96 commented on pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
jonyhy96 commented on pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#issuecomment-699620996


   > > Thanks @jonyhy96 Here, I have only one confusion. When users use it, they need to set it on the producer and consumer at the same time, right? Can we consider exposing this parameter to the user in ClientOptions so that the user only needs to set it once
   > 
   > That would be more convenience for users to use, But the user loses more detailed control over the program.
   > By the way,do you mean add an option at [here](https://github.com/apache/pulsar-client-go/blob/master/pulsar/client.go#L77) and [here](https://github.com/apache/pulsar-client-go/blob/master/pulsar/client_impl.go#L40),in reconnectToBroker use p.clinet.maxReconnectToBroker to decide the retry logic?
   
   /cc @wolfstudy @shohi  
   
   Any further information about this? Maybe we can implement this feature before octuber 😄 


----------------------------------------------------------------
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-client-go] shohi commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r476253761



##########
File path: pulsar/consumer.go
##########
@@ -140,6 +140,9 @@ type ConsumerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ConsumerInterceptor interface.
 	Interceptors ConsumerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: ultimate)
+	MaxReconnectToBroker *uint

Review comment:
       I see your point, but it's common to use int for this kind of configuration, such as [go-redis](https://github.com/go-redis/redis/blob/master/options.go#L59) / [nats.go](https://github.com/nats-io/nats.go/blob/master/nats.go#L266), we can make the following convention for the config to cope with different scenarios,
   * 0 for no retry
   * negative number for retry forever
   * positive number to set retry attempts
   
   As the current library is still in pre-1.0, i think it's ok to make a break change. Or alternate the meaning of 0 and negative number to make the change compatible with the current implementation.




----------------------------------------------------------------
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-client-go] jonyhy96 commented on pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
jonyhy96 commented on pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#issuecomment-706089867


   > LGTM +1, just a little comment, @jonyhy96 please help check, thanks.
   
   Thanks for review! Already fix 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-client-go] shohi commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r476247043



##########
File path: pulsar/producer_partition.go
##########
@@ -236,8 +236,18 @@ func (p *partitionProducer) ConnectionClosed() {
 }
 
 func (p *partitionProducer) reconnectToBroker() {
-	backoff := internal.Backoff{}
-	for {
+	var (
+		maxRetry int
+		backoff  = internal.Backoff{}
+	)
+
+	if p.options.MaxReconnectToBroker == nil {
+		maxRetry = -1
+	} else {
+		maxRetry = int(*p.options.MaxReconnectToBroker)
+	}

Review comment:
       my bad, the handling logic is correct :-) 
   
   What i mean is to wrap the test logic, like following
   
   ```go
   func (p *partitionProducer) shouldRetry() bool {
   	maxReconnects := p.opts.MaxReconnectToBroker
   
   	switch {
   	case maxReconnects < 0:
   		return true
   	case maxReconnects == 0:
   		return false
   	default:
   		p.reconnectAttempts++
   		if p.reconnectAttempts > maxReconnects {
   			return false
   		}
   
   		return true
   	}
   }
   
   func (p *partitionProducer) reconnectToBroker() {
   	for p.shouldRetry() {
   		// retry logic
   	}
   }
   ```




----------------------------------------------------------------
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-client-go] jonyhy96 commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
jonyhy96 commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r476231877



##########
File path: pulsar/consumer.go
##########
@@ -140,6 +140,9 @@ type ConsumerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ConsumerInterceptor interface.
 	Interceptors ConsumerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: ultimate)
+	MaxReconnectToBroker *uint

Review comment:
       > IMO using `int` type is more friendly, the default value `0` represents unlimited.
   
   How aboud `0` represents no retry but nil means retry ultimate times? Then this changes won't break current behavior of reconnectToBroker.




----------------------------------------------------------------
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-client-go] jonyhy96 commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
jonyhy96 commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r503095391



##########
File path: pulsar/producer.go
##########
@@ -138,6 +138,9 @@ type ProducerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ProducerInterceptor interface
 	Interceptors ProducerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: no retry)
+	MaxReconnectToBroker int

Review comment:
       Should i reset this pr to original commit or keep it this way?




----------------------------------------------------------------
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-client-go] wolfstudy commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r503059284



##########
File path: pulsar/producer.go
##########
@@ -138,6 +138,9 @@ type ProducerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ProducerInterceptor interface
 	Interceptors ProducerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: no retry)
+	MaxReconnectToBroker int

Review comment:
       ok, agree with you




----------------------------------------------------------------
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-client-go] wolfstudy commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r503044671



##########
File path: pulsar/producer.go
##########
@@ -138,6 +138,9 @@ type ProducerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ProducerInterceptor interface
 	Interceptors ProducerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: no retry)
+	MaxReconnectToBroker int

Review comment:
       We can consider setting -1 as the default value, and then check whether MaxReconnectToBroker is set, and if so, use the user-defined value.




----------------------------------------------------------------
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-client-go] shohi commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r476253761



##########
File path: pulsar/consumer.go
##########
@@ -140,6 +140,9 @@ type ConsumerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ConsumerInterceptor interface.
 	Interceptors ConsumerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: ultimate)
+	MaxReconnectToBroker *uint

Review comment:
       I see your point, but it's common to use int for this kind of configuration, such as [go-redis](https://github.com/go-redis/redis/blob/master/options.go#L59) / [nats.go](https://github.com/nats-io/nats.go/blob/master/nats.go#L266). 
   
   * 0 for no-try
   * negative number for retry forever
   * positive number for retry attempts
   
   As current library is still in pre-1.0, i think it's ok to make a break 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.

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



[GitHub] [pulsar-client-go] jonyhy96 edited a comment on pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
jonyhy96 edited a comment on pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#issuecomment-699620996


   > > Thanks @jonyhy96 Here, I have only one confusion. When users use it, they need to set it on the producer and consumer at the same time, right? Can we consider exposing this parameter to the user in ClientOptions so that the user only needs to set it once
   > 
   > That would be more convenience for users to use, But the user loses more detailed control over the program.
   > By the way,do you mean add an option at [here](https://github.com/apache/pulsar-client-go/blob/master/pulsar/client.go#L77) and [here](https://github.com/apache/pulsar-client-go/blob/master/pulsar/client_impl.go#L40),in reconnectToBroker use p.clinet.maxReconnectToBroker to decide the retry logic?
   
   /cc @wolfstudy @shohi  
   
   Any further information about this? Maybe we can implement this feature before october 😄 


----------------------------------------------------------------
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-client-go] wolfstudy commented on a change in pull request #360: [Issue 257][feat] Support limit the retry number of reconnectToBroker

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #360:
URL: https://github.com/apache/pulsar-client-go/pull/360#discussion_r503045073



##########
File path: pulsar/producer.go
##########
@@ -138,6 +138,9 @@ type ProducerOptions struct {
 
 	// A chain of interceptors, These interceptors will be called at some points defined in ProducerInterceptor interface
 	Interceptors ProducerInterceptors
+
+	// MaxReconnectToBroker set the maximum retry number of reconnectToBroker. (default: no retry)
+	MaxReconnectToBroker int

Review comment:
       At present, I see that you have added the option of MaxReconnectToBroker to the relevant test cases. This may not be very user-friendly. We need to ensure that the interface is as simple as possible.




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