Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Send a produce request in handshake state #2666

Closed
6 of 7 tasks
xzxxzx401 opened this issue Dec 29, 2019 · 5 comments
Closed
6 of 7 tasks

Send a produce request in handshake state #2666

xzxxzx401 opened this issue Dec 29, 2019 · 5 comments
Milestone

Comments

@xzxxzx401
Copy link

xzxxzx401 commented Dec 29, 2019

Read the FAQ first: https://github.com/edenhill/librdkafka/wiki/FAQ

Description

We found that rdkafka may send ProduceRequest or MetadataRequest when new connection just created, which should send APIVersionQuery or SaslHandShakeRequests first.

Our 0.10.1 broker has these debug logs:

[2019-12-27 18:04:21,452] DEBUG Connection with /<ClientIPA> disconnected (org.apache.kafka.common.network.Selector)
java.io.IOException: org.apache.kafka.common.errors.IllegalSaslStateException: Unexpected Kafka request of type METADATA during SASL handshake.
        at org.apache.kafka.common.security.authenticator.SaslServerAuthenticator.authenticate(SaslServerAuthenticator.java:243)
        at org.apache.kafka.common.network.KafkaChannel.prepare(KafkaChannel.java:64)
        at org.apache.kafka.common.network.Selector.pollSelectionKeys(Selector.java:340)
        at org.apache.kafka.common.network.Selector.poll(Selector.java:293)
        at kafka.network.Processor.poll(SocketServer.scala:476)
        at kafka.network.Processor.run(SocketServer.scala:416)
        at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.kafka.common.errors.IllegalSaslStateException: Unexpected Kafka request of type METADATA during SASL handshake.

[2019-12-27 18:05:58,544] DEBUG Connection with /<ClientIPB> disconnected (org.apache.kafka.common.network.Selector)
java.io.IOException: org.apache.kafka.common.errors.IllegalSaslStateException: Unexpected Kafka request of type PRODUCE during SASL handshake.
        at org.apache.kafka.common.security.authenticator.SaslServerAuthenticator.authenticate(SaslServerAuthenticator.java:243)
        at org.apache.kafka.common.network.KafkaChannel.prepare(KafkaChannel.java:64)
        at org.apache.kafka.common.network.Selector.pollSelectionKeys(Selector.java:340)
        at org.apache.kafka.common.network.Selector.poll(Selector.java:293)
        at kafka.network.Processor.poll(SocketServer.scala:476)
        at kafka.network.Processor.run(SocketServer.scala:416)
        at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.kafka.common.errors.IllegalSaslStateException: Unexpected Kafka request of type PRODUCE during SASL handshake.

In 2.3 broker, similar log also appears. But 2.3 add check for the length of an expected HandshakeRequest, which will reject requests longer than 500KB in handshake state. Our ProduceRequests is approximatly 1MB and will be rejected.

Deep into rdkafka code(both 0.11.5 and master), we found that in rd_kafka_bufq_connection_reset, rdkafka will only set buffer send ptr to 0, but will not reset rkbuf_corrid to 0. And in rd_kafka_broker_buf_enq0, rdkafka will insert a ApiVersionQuery or SaslHandShakeRequests after all requests which rkbuf_corrid is not 0.

So if a broker restart when a ProduceRequest is partially sent, then this ProduceRequest has a corrid larger than 0. Then after new connection established, it will be sent before any other requests. Is this meet excepted? I think we should finish ApiversionQuery and SaslHandshake first.

How to reproduce

Enable sasl, then restart a broker when rdkafka partially send a request.

IMPORTANT: Always try to reproduce the issue on the latest released version (see https://github.com/edenhill/librdkafka/releases), if it can't be reproduced on the latest version the issue has been fixed.

Checklist

IMPORTANT: We will close issues where the checklist has not been completed.

Please provide the following information:

  • librdkafka version (release number or git tag): v0.11.5 and master
  • Apache Kafka version: 0.10.1 and 2.3.0
  • librdkafka client configuration: SASL_PLAINTEXT enabled, api.version.fallback.ms=0 for our broker support ApiVersionQuery
  • Operating system: debian8 and 9
  • Provide logs (with debug=.. as necessary) from librdkafka
  • Provide broker log excerpts
  • Critical issue
@xzxxzx401
Copy link
Author

Debug log with debug=all.

I1227 20:01:19.561264 767918 |CONNECT|[0]Success:[thrd:sasl_plaintext://<BrokerIp>/bootstrap]: sasl_plaintext://<BrokerIp>/1: broker in state DOWN connecting
I1227 20:01:19.561362 767918 |CONNECT|[0]Success:[thrd:sasl_plaintext://<BrokerIp>/bootstrap]: sasl_plaintext://<BrokerIp>/1: Connecting to ipv4#<BrokerIp> (sasl_plaintext) with socket 1484
I1227 20:01:19.561383 767918 |STATE|[0]Success:[thrd:sasl_plaintext://<BrokerIp>/bootstrap]: sasl_plaintext://<BrokerIp>/1: Broker changed state DOWN -> CONNECT
I1227 20:01:19.561389 767918 |BROADCAST|[0]Success:[thrd:sasl_plaintext://<BrokerIp>/bootstrap]: Broadcasting state change
I1227 20:01:19.564422 767918 |CONNECT|[0]Success:[thrd:sasl_plaintext://<BrokerIp>/bootstrap]: sasl_plaintext://<BrokerIp>/1: Connected to ipv4#<BrokerIp>
I1227 20:01:19.564436 767918 |CONNECTED|[0]Success:[thrd:sasl_plaintext://<BrokerIp>/bootstrap]: sasl_plaintext://<BrokerIp>/1: Connected (#263)
I1227 20:01:19.564447 767918 |FEATURE|[0]Success:[thrd:sasl_plaintext://<BrokerIp>/bootstrap]: sasl_plaintext://<BrokerIp>/1: Updated enabled protocol features +ApiVersion to MsgVer1,ApiVersion,BrokerBalancedConsumer,ThrottleTime,Sasl,SaslHandshake,BrokerGroupCoordinator,LZ4,OffsetTime
I1227 20:01:19.564453 767918 |STATE|[0]Success:[thrd:sasl_plaintext://<BrokerIp>/bootstrap]: sasl_plaintext://<BrokerIp>/1: Broker changed state CONNECT -> APIVERSION_QUERY
I1227 20:01:19.564538 767918 |BROADCAST|[0]Success:[thrd:sasl_plaintext://<BrokerIp>/bootstrap]: Broadcasting state change
I1227 20:01:19.564885 767918 |SEND|[0]Success:[thrd:sasl_plaintext://<BrokerIp>/bootstrap]: sasl_plaintext://<BrokerIp>/1: Sent partial ProduceRequest (v2, 0+638568/1943302 bytes, CorrId 105146)
I1227 20:01:19.567972 767918 |BROKERFAIL|[0]Success:[thrd:sasl_plaintext://<BrokerIp>/bootstrap]: sasl_plaintext://<BrokerIp>/1: failed: err: Local: Broker transport failure: (errno: Connection reset by peer)
I1227 20:01:19.568035 767918 |FEATURE|[0]Success:[thrd:sasl_plaintext://<BrokerIp>/bootstrap]: sasl_plaintext://<BrokerIp>/1: Updated enabled protocol features -ApiVersion to MsgVer1,BrokerBalancedConsumer,ThrottleTime,Sasl,SaslHandshake,BrokerGroupCoordinator,LZ4,OffsetTime
I1227 20:01:19.568045 767918 |STATE|[0]Success:[thrd:sasl_plaintext://<BrokerIp>/bootstrap]: sasl_plaintext://<BrokerIp>/1: Broker changed state APIVERSION_QUERY -> DOWN
I1227 20:01:19.568050 767918 |BROADCAST|[0]Success:[thrd:sasl_plaintext://<BrokerIp>/bootstrap]: Broadcasting state change

@xzxxzx401
Copy link
Author

Hi @edenhill , any progress?

@xzxxzx401
Copy link
Author

Hi @edenhill , make some spare time to keep an eye on this question?

@edenhill edenhill added this to the v1.4.0 milestone Jan 20, 2020
@edenhill
Copy link
Contributor

We'll fix this for the upcoming v1.4.0 release.

Thank you

@edenhill
Copy link
Contributor

edenhill commented Feb 4, 2020

Great root cause analysis!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

No branches or pull requests

2 participants