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/05/06 15:50:41 UTC

[GitHub] [pulsar-client-go] megarajtm opened a new issue #515: Producer Send and SendAsyn is blocked for forever when pulsar is down

megarajtm opened a new issue #515:
URL: https://github.com/apache/pulsar-client-go/issues/515


   Producer Send and SendAsyn is blocked forever when pulsar is down if the MaxReconnectToBroker is set to ultimate. In case of pulsar down scenarios, within `runEventsLoop`in `producer_partition.go`, the call enters `reconnectToBroker` and remains in a forever loop until the pulsar broker connection is established. Due to this, no more events are consumed from `eventsChan` channel cause both Send and SendAsyn to be blocked. Due to this even the `SendTimeout` would also be not honoured. 
   
   #### Expected behavior
   Producer Send must not be blocked forever when the pulsar broker is down. It must honour the SendTimeout and return back with an error. 
   Producer SendAsyn must never be blocked when the pulsar broker is down. It must honour the SendTimeout and call the callback function.
   
   #### Actual behavior
   Due the above mentioned issue Producer Send/SendAsyn blocks forever when the pulsar broker is down
   
   #### Steps to reproduce
   1. Create a pulsar producer and set the MaxReconnectToBroker as ultimate and SendTimeout as a fixed value
   2. Send messages to pulsar using Send or SendAsyn API
   3. Bring down pulsar broker or inject a connection error between broker and client
   4. In case of Send, the call is blocker forever. In case of SendAsyn, the callback is never called and once the `pendingQueue` is filled, the call is blocked forever. 
   
   #### System configuration
   Pulsar client version - v0.4.0
   


-- 
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] megarajtm commented on issue #515: Producer Send and SendAsyn is blocked for forever when pulsar is down

Posted by GitBox <gi...@apache.org>.
megarajtm commented on issue #515:
URL: https://github.com/apache/pulsar-client-go/issues/515#issuecomment-833636150


   **Proposed fix** : In `runEventsLoop`, have a seperate go-routine working on `connectClosedCh` channel. This was `eventsChan` is never blocked. 
   
   {code}
   func (p *partitionProducer) runEventsLoop() {
   	go func() {
   		for {
   			select {
   			case <-p.closeCh:
   				return
   			case <-p.connectClosedCh:
   				p.log.Debug("runEventsLoop will reconnect")
   				p.reconnectToBroker()
   			}
   		}
   	}()
   	for {
   		select {
   		case i := <-p.eventsChan:
   			switch v := i.(type) {
   			case *sendRequest:
   				p.internalSend(v)
   			case *flushRequest:
   				p.internalFlush(v)
   			case *closeProducer:
   				p.internalClose(v)
   				return
   			}
   		case <-p.batchFlushTicker.C:
   			if p.batchBuilder.IsMultiBatches() {
   				p.internalFlushCurrentBatches()
   			} else {
   				p.internalFlushCurrentBatch()
   			}
   		}
   	}
   }
   {\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.

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



[GitHub] [pulsar-client-go] megarajtm commented on issue #515: Producer Send and SendAsyn is blocked for forever when pulsar is down

Posted by GitBox <gi...@apache.org>.
megarajtm commented on issue #515:
URL: https://github.com/apache/pulsar-client-go/issues/515#issuecomment-840321578


   #496 Issue seems to be with the consumers. Also, does the proposed fix look good. If yes, I can create a PR for the same


-- 
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] megarajtm edited a comment on issue #515: Producer Send and SendAsyn is blocked for forever when pulsar is down

Posted by GitBox <gi...@apache.org>.
megarajtm edited a comment on issue #515:
URL: https://github.com/apache/pulsar-client-go/issues/515#issuecomment-833636150


   **Proposed fix** : In `runEventsLoop`, have a seperate go-routine working on `connectClosedCh` channel. This was `eventsChan` is never blocked. 
   
   ```
   func (p *partitionProducer) runEventsLoop() {
   	go func() {
   		for {
   			select {
   			case <-p.closeCh:
   				return
   			case <-p.connectClosedCh:
   				p.log.Debug("runEventsLoop will reconnect")
   				p.reconnectToBroker()
   			}
   		}
   	}()
   	for {
   		select {
   		case i := <-p.eventsChan:
   			switch v := i.(type) {
   			case *sendRequest:
   				p.internalSend(v)
   			case *flushRequest:
   				p.internalFlush(v)
   			case *closeProducer:
   				p.internalClose(v)
   				return
   			}
   		case <-p.batchFlushTicker.C:
   			if p.batchBuilder.IsMultiBatches() {
   				p.internalFlushCurrentBatches()
   			} else {
   				p.internalFlushCurrentBatch()
   			}
   		}
   	}
   }
   ```
   


-- 
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] megarajtm edited a comment on issue #515: Producer Send and SendAsyn is blocked for forever when pulsar is down

Posted by GitBox <gi...@apache.org>.
megarajtm edited a comment on issue #515:
URL: https://github.com/apache/pulsar-client-go/issues/515#issuecomment-833636150


   **Proposed fix** : In `runEventsLoop`, have a seperate go-routine working on `connectClosedCh` channel. This way `eventsChan` is never blocked. 
   
   ```
   func (p *partitionProducer) runEventsLoop() {
   	go func() {
   		for {
   			select {
   			case <-p.closeCh:
   				return
   			case <-p.connectClosedCh:
   				p.log.Debug("runEventsLoop will reconnect")
   				p.reconnectToBroker()
   			}
   		}
   	}()
   	for {
   		select {
   		case i := <-p.eventsChan:
   			switch v := i.(type) {
   			case *sendRequest:
   				p.internalSend(v)
   			case *flushRequest:
   				p.internalFlush(v)
   			case *closeProducer:
   				p.internalClose(v)
   				return
   			}
   		case <-p.batchFlushTicker.C:
   			if p.batchBuilder.IsMultiBatches() {
   				p.internalFlushCurrentBatches()
   			} else {
   				p.internalFlushCurrentBatch()
   			}
   		}
   	}
   }
   ```
   


-- 
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] omnilight commented on issue #515: Producer Send and SendAsyn is blocked for forever when pulsar is down

Posted by GitBox <gi...@apache.org>.
omnilight commented on issue #515:
URL: https://github.com/apache/pulsar-client-go/issues/515#issuecomment-839854264


   This is a duplication of #496 


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