[jira] [Commented] (KAFKA-14402) Transactions Server Side Defense

2023-10-30 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-14402?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17781144#comment-17781144
 ] 

Travis Bischel commented on KAFKA-14402:


Sorry, I may be a bit confused then: when it says "for old clients to verify 
the partitions are in a given transaction" -- this can only happen in v4. How 
can a client / an admin verify that a partition is a part of the transaction if 
v4 is meant to be broker<=>broker only?

> Transactions Server Side Defense
> 
>
> Key: KAFKA-14402
> URL: https://issues.apache.org/jira/browse/KAFKA-14402
> Project: Kafka
>  Issue Type: Improvement
>Affects Versions: 3.5.0
>Reporter: Justine Olshan
>Assignee: Justine Olshan
>Priority: Major
>
> We have seen hanging transactions in Kafka where the last stable offset (LSO) 
> does not update, we can’t clean the log (if the topic is compacted), and 
> read_committed consumers get stuck.
> This can happen when a message gets stuck or delayed due to networking issues 
> or a network partition, the transaction aborts, and then the delayed message 
> finally comes in. The delayed message case can also violate EOS if the 
> delayed message comes in after the next addPartitionsToTxn request comes in. 
> Effectively we may see a message from a previous (aborted) transaction become 
> part of the next transaction.
> Another way hanging transactions can occur is that a client is buggy and may 
> somehow try to write to a partition before it adds the partition to the 
> transaction. In both of these cases, we want the server to have some control 
> to prevent these incorrect records from being written and either causing 
> hanging transactions or violating Exactly once semantics (EOS) by including 
> records in the wrong transaction.
> The best way to avoid this issue is to:
>  # *Uniquely identify transactions by bumping the producer epoch after every 
> commit/abort marker. That way, each transaction can be identified by 
> (producer id, epoch).* 
>  # {*}Remove the addPartitionsToTxn call and implicitly just add partitions 
> to the transaction on the first produce request during a transaction{*}.
> We avoid the late arrival case because the transaction is uniquely identified 
> and fenced AND we avoid the buggy client case because we remove the need for 
> the client to explicitly add partitions to begin the transaction.
> Of course, 1 and 2 require client-side changes, so for older clients, those 
> approaches won’t apply.
> 3. *To cover older clients, we will ensure a transaction is ongoing before we 
> write to a transaction. We can do this by querying the transaction 
> coordinator and caching the result.*
>  
> See KIP-890 for more information: ** 
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (KAFKA-14402) Transactions Server Side Defense

2023-10-30 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-14402?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17781122#comment-17781122
 ] 

Travis Bischel commented on KAFKA-14402:


Another addendum, after your update: v4 of the AddPartitionsToTxn API requires 
CLUSTER_ACTION to semi-enforce broker-to-broker requests. Can the KIP be 
updated to document this? Alternatively, can requiring CLUSTER_ACTION be 
dropped? The KIP itself indicates that admins will send AddPartitionsToTxn to 
see whether a partition is in the transaction, and admins do not need 
CLUSTER_ACTION. As well, as a client library -- if I send the request with one 
transaction (i.e. use v4 the same as I use v3), there's no reason I should be 
limited to v3.

> Transactions Server Side Defense
> 
>
> Key: KAFKA-14402
> URL: https://issues.apache.org/jira/browse/KAFKA-14402
> Project: Kafka
>  Issue Type: Improvement
>Affects Versions: 3.5.0
>Reporter: Justine Olshan
>Assignee: Justine Olshan
>Priority: Major
>
> We have seen hanging transactions in Kafka where the last stable offset (LSO) 
> does not update, we can’t clean the log (if the topic is compacted), and 
> read_committed consumers get stuck.
> This can happen when a message gets stuck or delayed due to networking issues 
> or a network partition, the transaction aborts, and then the delayed message 
> finally comes in. The delayed message case can also violate EOS if the 
> delayed message comes in after the next addPartitionsToTxn request comes in. 
> Effectively we may see a message from a previous (aborted) transaction become 
> part of the next transaction.
> Another way hanging transactions can occur is that a client is buggy and may 
> somehow try to write to a partition before it adds the partition to the 
> transaction. In both of these cases, we want the server to have some control 
> to prevent these incorrect records from being written and either causing 
> hanging transactions or violating Exactly once semantics (EOS) by including 
> records in the wrong transaction.
> The best way to avoid this issue is to:
>  # *Uniquely identify transactions by bumping the producer epoch after every 
> commit/abort marker. That way, each transaction can be identified by 
> (producer id, epoch).* 
>  # {*}Remove the addPartitionsToTxn call and implicitly just add partitions 
> to the transaction on the first produce request during a transaction{*}.
> We avoid the late arrival case because the transaction is uniquely identified 
> and fenced AND we avoid the buggy client case because we remove the need for 
> the client to explicitly add partitions to begin the transaction.
> Of course, 1 and 2 require client-side changes, so for older clients, those 
> approaches won’t apply.
> 3. *To cover older clients, we will ensure a transaction is ongoing before we 
> write to a transaction. We can do this by querying the transaction 
> coordinator and caching the result.*
>  
> See KIP-890 for more information: ** 
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-15656) Frequent INVALID_RECORD on Kafka 3.6

2023-10-23 Thread Travis Bischel (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15656?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Travis Bischel resolved KAFKA-15656.

Resolution: Duplicate

Closing this as a different manifestation of (and thus, duplicate of) 
KAFKA-15653

> Frequent INVALID_RECORD on Kafka 3.6
> 
>
> Key: KAFKA-15656
> URL: https://issues.apache.org/jira/browse/KAFKA-15656
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 3.6.0
>Reporter: Travis Bischel
>Priority: Major
> Attachments: invalid_record.log
>
>
> Using this docker-compose.yml:
> {noformat}
> version: "3.7"
> services:
>   kafka:
>     image: bitnami/kafka:latest
>     network_mode: host
>     environment:
>       KAFKA_ENABLE_KRAFT: yes
>       KAFKA_CFG_PROCESS_ROLES: controller,broker
>       KAFKA_CFG_CONTROLLER_LISTENER_NAMES: CONTROLLER
>       KAFKA_CFG_LISTENERS: PLAINTEXT://:9092,CONTROLLER://:9093
>       KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP: 
> CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT
>       KAFKA_CFG_CONTROLLER_QUORUM_VOTERS: 1@127.0.0.1:9093
>       # Set this to "PLAINTEXT://127.0.0.1:9092" if you want to run this 
> container on localhost via Docker
>       KAFKA_CFG_ADVERTISED_LISTENERS: PLAINTEXT://localhost:9092
>       KAFKA_CFG_NODE_ID: 1
>       ALLOW_PLAINTEXT_LISTENER: yes
>       KAFKA_KRAFT_CLUSTER_ID: XkpGZQ27R3eTl3OdTm2LYA # 16 byte base64-encoded 
> UUID{noformat}
> And running franz-go integration tests with KGO_TEST_RF=1, I consistently 
> receive INVALID_RECORD errors.
>  
> Looking at the container logs, I see these problematic log lines:
> {noformat}
> 2023-10-19 23:33:47,942] ERROR [ReplicaManager broker=1] Error processing 
> append operation on partition 
> 0cf2f3faaafd3f906ea848b684b04833ca162bcd19ecae2cab36767a54f248c7-0 
> (kafka.server.ReplicaManager) 
> org.apache.kafka.common.InvalidRecordException: Invalid negative header key 
> size -25
> [2023-10-19 23:33:47,942] ERROR [ReplicaManager broker=1] Error processing 
> append operation on partition 
> 0cf2f3faaafd3f906ea848b684b04833ca162bcd19ecae2cab36767a54f248c7-6 
> (kafka.server.ReplicaManager) 
> org.apache.kafka.common.InvalidRecordException: Reached end of input stream 
> before skipping all bytes. Remaining bytes:94
> [2023-10-19 23:33:47,942] ERROR [ReplicaManager broker=1] Error processing 
> append operation on partition 
> 0cf2f3faaafd3f906ea848b684b04833ca162bcd19ecae2cab36767a54f248c7-1 
> (kafka.server.ReplicaManager) 
> org.apache.kafka.common.InvalidRecordException: Found invalid number of 
> record headers -26
> [2023-10-19 23:33:47,948] ERROR [ReplicaManager broker=1] Error processing 
> append operation on partition 
> 0cf2f3faaafd3f906ea848b684b04833ca162bcd19ecae2cab36767a54f248c7-6 
> (kafka.server.ReplicaManager) 
> org.apache.kafka.common.InvalidRecordException: Found invalid number of 
> record headers -27
> [2023-10-19 23:33:47,950] ERROR [ReplicaManager broker=1] Error processing 
> append operation on partition 
> 0cf2f3faaafd3f906ea848b684b04833ca162bcd19ecae2cab36767a54f248c7-22 
> (kafka.server.ReplicaManager)
> org.apache.kafka.common.InvalidRecordException: Invalid negative header key 
> size -25
> [2023-10-19 23:33:47,947] ERROR [ReplicaManager broker=1] Error processing 
> append operation on partition 
> c63b6e30987317fad18815effb8d432b6df677d2ab56cf6da517bb93fa49b74b-25 
> (kafka.server.ReplicaManager)
> org.apache.kafka.common.InvalidRecordException: Found invalid number of 
> record headers -50
> [2023-10-19 23:33:47,959] ERROR [ReplicaManager broker=1] Error processing 
> append operation on partition 
> c63b6e30987317fad18815effb8d432b6df677d2ab56cf6da517bb93fa49b74b-25 
> (kafka.server.ReplicaManager) 
>  {noformat}
>  
> I modified franz-go with a diff to print the request that was written to the 
> wire once this error occurs. Attached is a v9 produce request. I deserialized 
> it locally and am not seeing the corrupt data that Kafka is printing. It's 
> possible there is a bug in the client, but again, these tests have never 
> received this error pre-Kafka 3.6. It _looks like_ there is either corruption 
> when processing the incoming data, or there is some problematic race 
> condition in the broker - I'm not sure which.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-15657) Unexpected errors when producing transactionally in 3.6

2023-10-23 Thread Travis Bischel (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15657?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Travis Bischel resolved KAFKA-15657.

Resolution: Duplicate

Closing this as a different manifestation (and thus, duplicate of) KAFKA-15653

> Unexpected errors when producing transactionally in 3.6
> ---
>
> Key: KAFKA-15657
> URL: https://issues.apache.org/jira/browse/KAFKA-15657
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 3.6.0
>Reporter: Travis Bischel
>Priority: Major
>
> In loop-testing the franz-go client, I am frequently receiving INVALID_RECORD 
> (which I created a separate issue for), and INVALID_TXN_STATE and 
> UNKNOWN_SERVER_ERROR.
> INVALID_TXN_STATE is being returned even though the partitions have been 
> added to the transaction (AddPartitionsToTxn). Nothing about the code has 
> changed between 3.5 and 3.6, and I have loop-integration-tested this code 
> against 3.5 thousands of times. 3.6 is newly - and always - returning 
> INVALID_TXN_STATE. If I change the code to retry on INVALID_TXN_STATE, I 
> eventually quickly (always) receive UNKNOWN_SERVER_ERROR. In looking at the 
> broker logs, the broker indicates that sequence numbers are out of order - 
> but (a) I am repeating requests that were in order (so something on the 
> broker got a little haywire maybe? or maybe this is due to me ignoring 
> invalid_txn_state?), _and_ I am not receiving OUT_OF_ORDER_SEQUENCE_NUMBER, I 
> am receiving UNKNOWN_SERVER_ERROR.
> I think the main problem is the client unexpectedly receiving 
> INVALID_TXN_STATE, but a second problem here is that OOOSN is being mapped to 
> USE on return for some reason.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (KAFKA-15653) NPE in ChunkedByteStream

2023-10-23 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-15653?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17778827#comment-17778827
 ] 

Travis Bischel commented on KAFKA-15653:


I've loop tested this a few times at this point, and so far the configuration 
option `transaction.partition.verification.enable=false` prevents test 
failures. I'm going to go ahead and close KAFKA-15657 and KAFKA-15656 which 
both seem like different manifestations of this issue.

> NPE in ChunkedByteStream
> 
>
> Key: KAFKA-15653
> URL: https://issues.apache.org/jira/browse/KAFKA-15653
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 3.6.0
> Environment: Docker container on a Linux laptop, using the latest 
> release.
>Reporter: Travis Bischel
>Priority: Major
> Attachments: repro.sh
>
>
> When looping franz-go integration tests, I received an UNKNOWN_SERVER_ERROR 
> from producing. The broker logs for the failing request:
>  
> {noformat}
> [2023-10-19 22:29:58,160] ERROR [ReplicaManager broker=2] Error processing 
> append operation on partition 
> 2fa8995d8002fbfe68a96d783f26aa2c5efc15368bf44ed8f2ab7e24b41b9879-24 
> (kafka.server.ReplicaManager)
> java.lang.NullPointerException
>   at 
> org.apache.kafka.common.utils.ChunkedBytesStream.(ChunkedBytesStream.java:89)
>   at 
> org.apache.kafka.common.record.CompressionType$3.wrapForInput(CompressionType.java:105)
>   at 
> org.apache.kafka.common.record.DefaultRecordBatch.recordInputStream(DefaultRecordBatch.java:273)
>   at 
> org.apache.kafka.common.record.DefaultRecordBatch.compressedIterator(DefaultRecordBatch.java:277)
>   at 
> org.apache.kafka.common.record.DefaultRecordBatch.skipKeyValueIterator(DefaultRecordBatch.java:352)
>   at 
> org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsetsCompressed(LogValidator.java:358)
>   at 
> org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsets(LogValidator.java:165)
>   at kafka.log.UnifiedLog.append(UnifiedLog.scala:805)
>   at kafka.log.UnifiedLog.appendAsLeader(UnifiedLog.scala:719)
>   at 
> kafka.cluster.Partition.$anonfun$appendRecordsToLeader$1(Partition.scala:1313)
>   at kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:1301)
>   at 
> kafka.server.ReplicaManager.$anonfun$appendToLocalLog$6(ReplicaManager.scala:1210)
>   at 
> scala.collection.StrictOptimizedMapOps.map(StrictOptimizedMapOps.scala:28)
>   at 
> scala.collection.StrictOptimizedMapOps.map$(StrictOptimizedMapOps.scala:27)
>   at scala.collection.mutable.HashMap.map(HashMap.scala:35)
>   at 
> kafka.server.ReplicaManager.appendToLocalLog(ReplicaManager.scala:1198)
>   at kafka.server.ReplicaManager.appendEntries$1(ReplicaManager.scala:754)
>   at 
> kafka.server.ReplicaManager.$anonfun$appendRecords$18(ReplicaManager.scala:874)
>   at 
> kafka.server.ReplicaManager.$anonfun$appendRecords$18$adapted(ReplicaManager.scala:874)
>   at 
> kafka.server.KafkaRequestHandler$.$anonfun$wrap$3(KafkaRequestHandler.scala:73)
>   at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:130)
>   at java.base/java.lang.Thread.run(Unknown Source)
> {noformat}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (KAFKA-15653) NPE in ChunkedByteStream

2023-10-20 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-15653?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=1846#comment-1846
 ] 

Travis Bischel commented on KAFKA-15653:


Sorry, one more note: I'm also occasionally getting InvalidRecordException 
frequently without NPE. The attached repro script ignores 
InvalidRecordException because it's easier to get a failure that contains NPE 
when ignoring that error, when not ignoring that error, frequently the test 
fails with InvalidRecordException without an NPE (so, different sort of problem 
-- but perhaps related).

Attached is a script that will clone the franz-go repo, optionally install Go, 
and loop docker compose up/down with integration tests. Once the script stops, 
container logs are in CONTAINER_LOGS, and franz-go debug logs are in 
FRANZ_FAIL. In FRANZ_FAIL, you can search for `--- FAIL` to see the error that 
caused the test to stop (usually UNKNOWN_SERVER_ERROR) and then search for the 
first instance of that error -- it will be returned from producing. In 
CONTAINER_LOGS, you can look for NullPointerException or InvalidRecordException.

 [^repro.sh] 

> NPE in ChunkedByteStream
> 
>
> Key: KAFKA-15653
> URL: https://issues.apache.org/jira/browse/KAFKA-15653
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 3.6.0
> Environment: Docker container on a Linux laptop, using the latest 
> release.
>Reporter: Travis Bischel
>Assignee: Divij Vaidya
>Priority: Major
> Attachments: repro.sh
>
>
> When looping franz-go integration tests, I received an UNKNOWN_SERVER_ERROR 
> from producing. The broker logs for the failing request:
>  
> {noformat}
> [2023-10-19 22:29:58,160] ERROR [ReplicaManager broker=2] Error processing 
> append operation on partition 
> 2fa8995d8002fbfe68a96d783f26aa2c5efc15368bf44ed8f2ab7e24b41b9879-24 
> (kafka.server.ReplicaManager)
> java.lang.NullPointerException
>   at 
> org.apache.kafka.common.utils.ChunkedBytesStream.(ChunkedBytesStream.java:89)
>   at 
> org.apache.kafka.common.record.CompressionType$3.wrapForInput(CompressionType.java:105)
>   at 
> org.apache.kafka.common.record.DefaultRecordBatch.recordInputStream(DefaultRecordBatch.java:273)
>   at 
> org.apache.kafka.common.record.DefaultRecordBatch.compressedIterator(DefaultRecordBatch.java:277)
>   at 
> org.apache.kafka.common.record.DefaultRecordBatch.skipKeyValueIterator(DefaultRecordBatch.java:352)
>   at 
> org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsetsCompressed(LogValidator.java:358)
>   at 
> org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsets(LogValidator.java:165)
>   at kafka.log.UnifiedLog.append(UnifiedLog.scala:805)
>   at kafka.log.UnifiedLog.appendAsLeader(UnifiedLog.scala:719)
>   at 
> kafka.cluster.Partition.$anonfun$appendRecordsToLeader$1(Partition.scala:1313)
>   at kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:1301)
>   at 
> kafka.server.ReplicaManager.$anonfun$appendToLocalLog$6(ReplicaManager.scala:1210)
>   at 
> scala.collection.StrictOptimizedMapOps.map(StrictOptimizedMapOps.scala:28)
>   at 
> scala.collection.StrictOptimizedMapOps.map$(StrictOptimizedMapOps.scala:27)
>   at scala.collection.mutable.HashMap.map(HashMap.scala:35)
>   at 
> kafka.server.ReplicaManager.appendToLocalLog(ReplicaManager.scala:1198)
>   at kafka.server.ReplicaManager.appendEntries$1(ReplicaManager.scala:754)
>   at 
> kafka.server.ReplicaManager.$anonfun$appendRecords$18(ReplicaManager.scala:874)
>   at 
> kafka.server.ReplicaManager.$anonfun$appendRecords$18$adapted(ReplicaManager.scala:874)
>   at 
> kafka.server.KafkaRequestHandler$.$anonfun$wrap$3(KafkaRequestHandler.scala:73)
>   at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:130)
>   at java.base/java.lang.Thread.run(Unknown Source)
> {noformat}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (KAFKA-15653) NPE in ChunkedByteStream

2023-10-20 Thread Travis Bischel (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15653?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Travis Bischel updated KAFKA-15653:
---
Attachment: repro.sh

> NPE in ChunkedByteStream
> 
>
> Key: KAFKA-15653
> URL: https://issues.apache.org/jira/browse/KAFKA-15653
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 3.6.0
> Environment: Docker container on a Linux laptop, using the latest 
> release.
>Reporter: Travis Bischel
>Assignee: Divij Vaidya
>Priority: Major
> Attachments: repro.sh
>
>
> When looping franz-go integration tests, I received an UNKNOWN_SERVER_ERROR 
> from producing. The broker logs for the failing request:
>  
> {noformat}
> [2023-10-19 22:29:58,160] ERROR [ReplicaManager broker=2] Error processing 
> append operation on partition 
> 2fa8995d8002fbfe68a96d783f26aa2c5efc15368bf44ed8f2ab7e24b41b9879-24 
> (kafka.server.ReplicaManager)
> java.lang.NullPointerException
>   at 
> org.apache.kafka.common.utils.ChunkedBytesStream.(ChunkedBytesStream.java:89)
>   at 
> org.apache.kafka.common.record.CompressionType$3.wrapForInput(CompressionType.java:105)
>   at 
> org.apache.kafka.common.record.DefaultRecordBatch.recordInputStream(DefaultRecordBatch.java:273)
>   at 
> org.apache.kafka.common.record.DefaultRecordBatch.compressedIterator(DefaultRecordBatch.java:277)
>   at 
> org.apache.kafka.common.record.DefaultRecordBatch.skipKeyValueIterator(DefaultRecordBatch.java:352)
>   at 
> org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsetsCompressed(LogValidator.java:358)
>   at 
> org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsets(LogValidator.java:165)
>   at kafka.log.UnifiedLog.append(UnifiedLog.scala:805)
>   at kafka.log.UnifiedLog.appendAsLeader(UnifiedLog.scala:719)
>   at 
> kafka.cluster.Partition.$anonfun$appendRecordsToLeader$1(Partition.scala:1313)
>   at kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:1301)
>   at 
> kafka.server.ReplicaManager.$anonfun$appendToLocalLog$6(ReplicaManager.scala:1210)
>   at 
> scala.collection.StrictOptimizedMapOps.map(StrictOptimizedMapOps.scala:28)
>   at 
> scala.collection.StrictOptimizedMapOps.map$(StrictOptimizedMapOps.scala:27)
>   at scala.collection.mutable.HashMap.map(HashMap.scala:35)
>   at 
> kafka.server.ReplicaManager.appendToLocalLog(ReplicaManager.scala:1198)
>   at kafka.server.ReplicaManager.appendEntries$1(ReplicaManager.scala:754)
>   at 
> kafka.server.ReplicaManager.$anonfun$appendRecords$18(ReplicaManager.scala:874)
>   at 
> kafka.server.ReplicaManager.$anonfun$appendRecords$18$adapted(ReplicaManager.scala:874)
>   at 
> kafka.server.KafkaRequestHandler$.$anonfun$wrap$3(KafkaRequestHandler.scala:73)
>   at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:130)
>   at java.base/java.lang.Thread.run(Unknown Source)
> {noformat}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (KAFKA-15653) NPE in ChunkedByteStream

2023-10-20 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-15653?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=1835#comment-1835
 ] 

Travis Bischel commented on KAFKA-15653:


Maybe related?
{noformat}
org.apache.kafka.common.errors.InvalidRequestException: Error getting request 
for apiKey: PRODUCE, apiVersion: 9, connectionId: 
127.0.0.1:9092-127.0.0.1:46394-145, listenerName: ListenerName(PLAINTEXT), 
principal: User:ANONYMOUS
Caused by: java.nio.BufferUnderflowException
at java.base/java.nio.Buffer.nextGetIndex(Buffer.java:699)
at java.base/java.nio.HeapByteBuffer.get(HeapByteBuffer.java:165)
at 
org.apache.kafka.common.utils.ByteUtils.readUnsignedVarint(ByteUtils.java:160)
at 
org.apache.kafka.common.protocol.ByteBufferAccessor.readUnsignedVarint(ByteBufferAccessor.java:70)
at 
org.apache.kafka.common.message.ProduceRequestData.read(ProduceRequestData.java:195)
at 
org.apache.kafka.common.message.ProduceRequestData.(ProduceRequestData.java:114)
at 
org.apache.kafka.common.requests.ProduceRequest.parse(ProduceRequest.java:256)
at 
org.apache.kafka.common.requests.AbstractRequest.doParseRequest(AbstractRequest.java:178)
at 
org.apache.kafka.common.requests.AbstractRequest.parseRequest(AbstractRequest.java:172)
at 
org.apache.kafka.common.requests.RequestContext.parseRequest(RequestContext.java:95)
at kafka.network.RequestChannel$Request.(RequestChannel.scala:108)
at 
kafka.network.Processor.$anonfun$processCompletedReceives$1(SocketServer.scala:1148)
at 
java.base/java.util.LinkedHashMap$LinkedValues.forEach(LinkedHashMap.java:647)
at 
kafka.network.Processor.processCompletedReceives(SocketServer.scala:1126)
at kafka.network.Processor.run(SocketServer.scala:1012)
at java.base/java.lang.Thread.run(Thread.java:833)
{noformat}

Next comment I'll attach a small script that can be used to repro; I'm testing 
it a few times to iron out bash kinks

> NPE in ChunkedByteStream
> 
>
> Key: KAFKA-15653
> URL: https://issues.apache.org/jira/browse/KAFKA-15653
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 3.6.0
> Environment: Docker container on a Linux laptop, using the latest 
> release.
>Reporter: Travis Bischel
>Assignee: Divij Vaidya
>Priority: Major
>
> When looping franz-go integration tests, I received an UNKNOWN_SERVER_ERROR 
> from producing. The broker logs for the failing request:
>  
> {noformat}
> [2023-10-19 22:29:58,160] ERROR [ReplicaManager broker=2] Error processing 
> append operation on partition 
> 2fa8995d8002fbfe68a96d783f26aa2c5efc15368bf44ed8f2ab7e24b41b9879-24 
> (kafka.server.ReplicaManager)
> java.lang.NullPointerException
>   at 
> org.apache.kafka.common.utils.ChunkedBytesStream.(ChunkedBytesStream.java:89)
>   at 
> org.apache.kafka.common.record.CompressionType$3.wrapForInput(CompressionType.java:105)
>   at 
> org.apache.kafka.common.record.DefaultRecordBatch.recordInputStream(DefaultRecordBatch.java:273)
>   at 
> org.apache.kafka.common.record.DefaultRecordBatch.compressedIterator(DefaultRecordBatch.java:277)
>   at 
> org.apache.kafka.common.record.DefaultRecordBatch.skipKeyValueIterator(DefaultRecordBatch.java:352)
>   at 
> org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsetsCompressed(LogValidator.java:358)
>   at 
> org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsets(LogValidator.java:165)
>   at kafka.log.UnifiedLog.append(UnifiedLog.scala:805)
>   at kafka.log.UnifiedLog.appendAsLeader(UnifiedLog.scala:719)
>   at 
> kafka.cluster.Partition.$anonfun$appendRecordsToLeader$1(Partition.scala:1313)
>   at kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:1301)
>   at 
> kafka.server.ReplicaManager.$anonfun$appendToLocalLog$6(ReplicaManager.scala:1210)
>   at 
> scala.collection.StrictOptimizedMapOps.map(StrictOptimizedMapOps.scala:28)
>   at 
> scala.collection.StrictOptimizedMapOps.map$(StrictOptimizedMapOps.scala:27)
>   at scala.collection.mutable.HashMap.map(HashMap.scala:35)
>   at 
> kafka.server.ReplicaManager.appendToLocalLog(ReplicaManager.scala:1198)
>   at kafka.server.ReplicaManager.appendEntries$1(ReplicaManager.scala:754)
>   at 
> kafka.server.ReplicaManager.$anonfun$appendRecords$18(ReplicaManager.scala:874)
>   at 
> kafka.server.ReplicaManager.$anonfun$appendRecords$18$adapted(ReplicaManager.scala:874)
>   at 
> kafka.server.KafkaRequestHandler$.$anonfun$wrap$3(KafkaRequestHandler.scala:73)
>   at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:130)
>   at java.base/java.lang.Thread.run(Unknown Source)
> {noformat}



--
This 

[jira] [Commented] (KAFKA-15657) Unexpected errors when producing transactionally in 3.6

2023-10-19 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-15657?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=1565#comment-1565
 ] 

Travis Bischel commented on KAFKA-15657:


I'm beginning to suspect that KAFKA-15653 may eventually lead to this, I never 
experience this bug without first experiencing the NPEs while appending. I'll 
wait until 15653 is addressed and loop-test seeing if this still occurs.

> Unexpected errors when producing transactionally in 3.6
> ---
>
> Key: KAFKA-15657
> URL: https://issues.apache.org/jira/browse/KAFKA-15657
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 3.6.0
>Reporter: Travis Bischel
>Priority: Major
>
> In loop-testing the franz-go client, I am frequently receiving INVALID_RECORD 
> (which I created a separate issue for), and INVALID_TXN_STATE and 
> UNKNOWN_SERVER_ERROR.
> INVALID_TXN_STATE is being returned even though the partitions have been 
> added to the transaction (AddPartitionsToTxn). Nothing about the code has 
> changed between 3.5 and 3.6, and I have loop-integration-tested this code 
> against 3.5 thousands of times. 3.6 is newly - and always - returning 
> INVALID_TXN_STATE. If I change the code to retry on INVALID_TXN_STATE, I 
> eventually quickly (always) receive UNKNOWN_SERVER_ERROR. In looking at the 
> broker logs, the broker indicates that sequence numbers are out of order - 
> but (a) I am repeating requests that were in order (so something on the 
> broker got a little haywire maybe? or maybe this is due to me ignoring 
> invalid_txn_state?), _and_ I am not receiving OUT_OF_ORDER_SEQUENCE_NUMBER, I 
> am receiving UNKNOWN_SERVER_ERROR.
> I think the main problem is the client unexpectedly receiving 
> INVALID_TXN_STATE, but a second problem here is that OOOSN is being mapped to 
> USE on return for some reason.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Comment Edited] (KAFKA-15653) NPE in ChunkedByteStream

2023-10-19 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-15653?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=1542#comment-1542
 ] 

Travis Bischel edited comment on KAFKA-15653 at 10/20/23 2:55 AM:
--

{noformat}
[2023-10-20 02:31:00,204] ERROR [ReplicaManager broker=1] Error processing 
append operation on partition 
2c69b88eab8670ef1fd0e55b81b9e000995386afd8756ea342494d36911e6f01-29 
(kafka.server.ReplicaManager)
java.lang.NullPointerException: Cannot invoke "java.nio.ByteBuffer.hasArray()" 
because "this.intermediateBufRef" is null 
at 
org.apache.kafka.common.utils.ChunkedBytesStream.(ChunkedBytesStream.java:89)
at 
org.apache.kafka.common.record.CompressionType$3.wrapForInput(CompressionType.java:105)
at 
org.apache.kafka.common.record.DefaultRecordBatch.recordInputStream(DefaultRecordBatch.java:273)
at 
org.apache.kafka.common.record.DefaultRecordBatch.compressedIterator(DefaultRecordBatch.java:277)
at 
org.apache.kafka.common.record.DefaultRecordBatch.skipKeyValueIterator(DefaultRecordBatch.java:352)
at 
org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsetsCompressed(LogValidator.java:358)
at 
org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsets(LogValidator.java:165)
at kafka.log.UnifiedLog.$anonfun$append$2(UnifiedLog.scala:805)
at kafka.log.UnifiedLog.append(UnifiedLog.scala:1845)
at kafka.log.UnifiedLog.appendAsLeader(UnifiedLog.scala:719)
at 
kafka.cluster.Partition.$anonfun$appendRecordsToLeader$1(Partition.scala:1313)
at kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:1301)
at 
kafka.server.ReplicaManager.$anonfun$appendToLocalLog$6(ReplicaManager.scala:1210)
at 
scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:286)
at 
scala.collection.immutable.HashMap$HashMap1.foreach(HashMap.scala:400)
at 
scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:728)
at 
scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:728)
at scala.collection.TraversableLike.map(TraversableLike.scala:286)
at scala.collection.TraversableLike.map$(TraversableLike.scala:279)
at scala.collection.AbstractTraversable.map(Traversable.scala:108)
at 
kafka.server.ReplicaManager.appendToLocalLog(ReplicaManager.scala:1198)
at kafka.server.ReplicaManager.appendRecords(ReplicaManager.scala:754)
at kafka.server.KafkaApis.handleProduceRequest(KafkaApis.scala:686)
at kafka.server.KafkaApis.handle(KafkaApis.scala:180)
at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:149)
at java.base/java.lang.Thread.run(Thread.java:833)
{noformat}


was (Author: twmb):
Not just :
{noformat}
[2023-10-20 02:31:00,204] ERROR [ReplicaManager broker=1] Error processing 
append operation on partition 
2c69b88eab8670ef1fd0e55b81b9e000995386afd8756ea342494d36911e6f01-29 
(kafka.server.ReplicaManager)
java.lang.NullPointerException: Cannot invoke "java.nio.ByteBuffer.hasArray()" 
because "this.intermediateBufRef" is null 
at 
org.apache.kafka.common.utils.ChunkedBytesStream.(ChunkedBytesStream.java:89)
at 
org.apache.kafka.common.record.CompressionType$3.wrapForInput(CompressionType.java:105)
at 
org.apache.kafka.common.record.DefaultRecordBatch.recordInputStream(DefaultRecordBatch.java:273)
at 
org.apache.kafka.common.record.DefaultRecordBatch.compressedIterator(DefaultRecordBatch.java:277)
at 
org.apache.kafka.common.record.DefaultRecordBatch.skipKeyValueIterator(DefaultRecordBatch.java:352)
at 
org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsetsCompressed(LogValidator.java:358)
at 
org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsets(LogValidator.java:165)
at kafka.log.UnifiedLog.$anonfun$append$2(UnifiedLog.scala:805)
at kafka.log.UnifiedLog.append(UnifiedLog.scala:1845)
at kafka.log.UnifiedLog.appendAsLeader(UnifiedLog.scala:719)
at 
kafka.cluster.Partition.$anonfun$appendRecordsToLeader$1(Partition.scala:1313)
at kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:1301)
at 
kafka.server.ReplicaManager.$anonfun$appendToLocalLog$6(ReplicaManager.scala:1210)
at 
scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:286)
at 
scala.collection.immutable.HashMap$HashMap1.foreach(HashMap.scala:400)
at 
scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:728)
at 
scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:728)
at scala.collection.TraversableLike.map(TraversableLike.scala:286)
at scala.collection.TraversableLike.map$(TraversableLike.scala:279)
   

[jira] [Comment Edited] (KAFKA-15657) Unexpected errors when producing transactionally in 3.6

2023-10-19 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-15657?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=1537#comment-1537
 ] 

Travis Bischel edited comment on KAFKA-15657 at 10/20/23 2:35 AM:
--

re: first comment – the client doesn't advance to producing unless 
AddPartitionsToTxn succeeds. If the request partially succeeds, failed 
partitions are stripped and only successfully added partitions are produced. 
The logic is definitely hard to follow if you're not familiar with the code, 
but here's issuing/stripping: 
[here|https://github.com/twmb/franz-go/blob/ae169a1f35c2ee6b130c4e520632b33e6c491e0b/pkg/kgo/sink.go#L442-L498,]
 and here's where the request is issued (in the same function as producing – 
before the produce request is issued): 
[here|https://github.com/twmb/franz-go/blob/ae169a1f35c2ee6b130c4e520632b33e6c491e0b/pkg/kgo/sink.go#L316-L357]

Also wrt race condition – these tests also pass against the redpanda binary, 
which has always had KIP-890 semantics / has never allowed transactional 
produce requests unless the partition has been added to the transaction (in 
fact this is part of how I caught some early redpanda bugs with _that_ 
implementation).

 

re: second comment, I'll capture some debug logs so you can see both the client 
logs and the container. The tests currently are using v3. I'm currently running 
this in a loop:

```

docker compose down; sleep 1; docker compose up -d ; sleep 5 ; while go test 
-run Txn/cooperative > logs; do echo whoo; docker compose down; sleep 1; docker 
compose up -d. sleep 5; done

```

Once this fails, I'll upload the logs. This is currently ignoring 
INVALID_RECORD, which I more regularly run into. I may remove gating this to 
just the cooperative test and instead run it against all balancers at once (it 
seems heavier load runs into the problem more frequently).

 

Also this does remind me though, somebody had a feature request that 
deliberately abused the ability to produce before AddPartitionsToTxn was done, 
I need to remove support of this for 3.6+. This _is_ exercised in franz-go's CI 
right now and will fail CI for 3.6+ (see the doc comment on 
[EndBeginTxnUnsafe|https://pkg.go.dev/github.com/twmb/franz-go/pkg/kgo#EndBeginTxnHow]).

Edit: KAFKA-15653 may be complicating the investigation here, too.


was (Author: twmb):
re: first comment – the client doesn't advance to producing unless 
AddPartitionsToTxn succeeds. If the request partially succeeds, failed 
partitions are stripped and only successfully added partitions are produced. 
The logic is definitely hard to follow if you're not familiar with the code, 
but here's issuing/stripping: 
[here|https://github.com/twmb/franz-go/blob/ae169a1f35c2ee6b130c4e520632b33e6c491e0b/pkg/kgo/sink.go#L442-L498,]
 and here's where the request is issued (in the same function as producing – 
before the produce request is issued): 
[here|https://github.com/twmb/franz-go/blob/ae169a1f35c2ee6b130c4e520632b33e6c491e0b/pkg/kgo/sink.go#L316-L357]

Also wrt race condition – these tests also pass against the redpanda binary, 
which has always had KIP-890 semantics / has never allowed transactional 
produce requests unless the partition has been added to the transaction (in 
fact this is part of how I caught some early redpanda bugs with _that_ 
implementation).

 

re: second comment, I'll capture some debug logs so you can see both the client 
logs and the container. The tests currently are using v3. I'm currently running 
this in a loop:

```

docker compose down; sleep 1; docker compose up -d ; sleep 5 ; while go test 
-run Txn/cooperative > logs; do echo whoo; docker compose down; sleep 1; docker 
compose up -d. sleep 5; done

```

Once this fails, I'll upload the logs. This is currently ignoring 
INVALID_RECORD, which I more regularly run into. I may remove gating this to 
just the cooperative test and instead run it against all balancers at once (it 
seems heavier load runs into the problem more frequently).

 

Also this does remind me though, somebody had a feature request that 
deliberately abused the ability to produce before AddPartitionsToTxn was done, 
I need to remove support of this for 3.6+. This _is_ exercised in franz-go's CI 
right now and will fail CI for 3.6+ (see the doc comment on 
[EndBeginTxnUnsafe|https://pkg.go.dev/github.com/twmb/franz-go/pkg/kgo#EndBeginTxnHow]).

> Unexpected errors when producing transactionally in 3.6
> ---
>
> Key: KAFKA-15657
> URL: https://issues.apache.org/jira/browse/KAFKA-15657
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 3.6.0
>Reporter: Travis Bischel
>Priority: Major
>
> In loop-testing the franz-go client, I am frequently receiving INVALID_RECORD 
> (which I created a separate issue 

[jira] [Commented] (KAFKA-15653) NPE in ChunkedByteStream

2023-10-19 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-15653?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=1542#comment-1542
 ] 

Travis Bischel commented on KAFKA-15653:


Not just :
{noformat}
[2023-10-20 02:31:00,204] ERROR [ReplicaManager broker=1] Error processing 
append operation on partition 
2c69b88eab8670ef1fd0e55b81b9e000995386afd8756ea342494d36911e6f01-29 
(kafka.server.ReplicaManager)
java.lang.NullPointerException: Cannot invoke "java.nio.ByteBuffer.hasArray()" 
because "this.intermediateBufRef" is null 
at 
org.apache.kafka.common.utils.ChunkedBytesStream.(ChunkedBytesStream.java:89)
at 
org.apache.kafka.common.record.CompressionType$3.wrapForInput(CompressionType.java:105)
at 
org.apache.kafka.common.record.DefaultRecordBatch.recordInputStream(DefaultRecordBatch.java:273)
at 
org.apache.kafka.common.record.DefaultRecordBatch.compressedIterator(DefaultRecordBatch.java:277)
at 
org.apache.kafka.common.record.DefaultRecordBatch.skipKeyValueIterator(DefaultRecordBatch.java:352)
at 
org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsetsCompressed(LogValidator.java:358)
at 
org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsets(LogValidator.java:165)
at kafka.log.UnifiedLog.$anonfun$append$2(UnifiedLog.scala:805)
at kafka.log.UnifiedLog.append(UnifiedLog.scala:1845)
at kafka.log.UnifiedLog.appendAsLeader(UnifiedLog.scala:719)
at 
kafka.cluster.Partition.$anonfun$appendRecordsToLeader$1(Partition.scala:1313)
at kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:1301)
at 
kafka.server.ReplicaManager.$anonfun$appendToLocalLog$6(ReplicaManager.scala:1210)
at 
scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:286)
at 
scala.collection.immutable.HashMap$HashMap1.foreach(HashMap.scala:400)
at 
scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:728)
at 
scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:728)
at scala.collection.TraversableLike.map(TraversableLike.scala:286)
at scala.collection.TraversableLike.map$(TraversableLike.scala:279)
at scala.collection.AbstractTraversable.map(Traversable.scala:108)
at 
kafka.server.ReplicaManager.appendToLocalLog(ReplicaManager.scala:1198)
at kafka.server.ReplicaManager.appendRecords(ReplicaManager.scala:754)
at kafka.server.KafkaApis.handleProduceRequest(KafkaApis.scala:686)
at kafka.server.KafkaApis.handle(KafkaApis.scala:180)
at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:149)
at java.base/java.lang.Thread.run(Thread.java:833)
{noformat}

> NPE in ChunkedByteStream
> 
>
> Key: KAFKA-15653
> URL: https://issues.apache.org/jira/browse/KAFKA-15653
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 3.6.0
> Environment: Docker container on a Linux laptop, using the latest 
> release.
>Reporter: Travis Bischel
>Priority: Major
>
> When looping franz-go integration tests, I received an UNKNOWN_SERVER_ERROR 
> from producing. The broker logs for the failing request:
>  
> {noformat}
> [2023-10-19 22:29:58,160] ERROR [ReplicaManager broker=2] Error processing 
> append operation on partition 
> 2fa8995d8002fbfe68a96d783f26aa2c5efc15368bf44ed8f2ab7e24b41b9879-24 
> (kafka.server.ReplicaManager)
> java.lang.NullPointerException
>   at 
> org.apache.kafka.common.utils.ChunkedBytesStream.(ChunkedBytesStream.java:89)
>   at 
> org.apache.kafka.common.record.CompressionType$3.wrapForInput(CompressionType.java:105)
>   at 
> org.apache.kafka.common.record.DefaultRecordBatch.recordInputStream(DefaultRecordBatch.java:273)
>   at 
> org.apache.kafka.common.record.DefaultRecordBatch.compressedIterator(DefaultRecordBatch.java:277)
>   at 
> org.apache.kafka.common.record.DefaultRecordBatch.skipKeyValueIterator(DefaultRecordBatch.java:352)
>   at 
> org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsetsCompressed(LogValidator.java:358)
>   at 
> org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsets(LogValidator.java:165)
>   at kafka.log.UnifiedLog.append(UnifiedLog.scala:805)
>   at kafka.log.UnifiedLog.appendAsLeader(UnifiedLog.scala:719)
>   at 
> kafka.cluster.Partition.$anonfun$appendRecordsToLeader$1(Partition.scala:1313)
>   at kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:1301)
>   at 
> kafka.server.ReplicaManager.$anonfun$appendToLocalLog$6(ReplicaManager.scala:1210)
>   at 
> scala.collection.StrictOptimizedMapOps.map(StrictOptimizedMapOps.scala:28)
>   at 
> 

[jira] [Updated] (KAFKA-15653) NPE in ChunkedByteStream

2023-10-19 Thread Travis Bischel (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15653?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Travis Bischel updated KAFKA-15653:
---
Summary: NPE in ChunkedByteStream  (was: NPE in ChunkedByteStream.)

> NPE in ChunkedByteStream
> 
>
> Key: KAFKA-15653
> URL: https://issues.apache.org/jira/browse/KAFKA-15653
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 3.6.0
> Environment: Docker container on a Linux laptop, using the latest 
> release.
>Reporter: Travis Bischel
>Priority: Major
>
> When looping franz-go integration tests, I received an UNKNOWN_SERVER_ERROR 
> from producing. The broker logs for the failing request:
>  
> {noformat}
> [2023-10-19 22:29:58,160] ERROR [ReplicaManager broker=2] Error processing 
> append operation on partition 
> 2fa8995d8002fbfe68a96d783f26aa2c5efc15368bf44ed8f2ab7e24b41b9879-24 
> (kafka.server.ReplicaManager)
> java.lang.NullPointerException
>   at 
> org.apache.kafka.common.utils.ChunkedBytesStream.(ChunkedBytesStream.java:89)
>   at 
> org.apache.kafka.common.record.CompressionType$3.wrapForInput(CompressionType.java:105)
>   at 
> org.apache.kafka.common.record.DefaultRecordBatch.recordInputStream(DefaultRecordBatch.java:273)
>   at 
> org.apache.kafka.common.record.DefaultRecordBatch.compressedIterator(DefaultRecordBatch.java:277)
>   at 
> org.apache.kafka.common.record.DefaultRecordBatch.skipKeyValueIterator(DefaultRecordBatch.java:352)
>   at 
> org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsetsCompressed(LogValidator.java:358)
>   at 
> org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsets(LogValidator.java:165)
>   at kafka.log.UnifiedLog.append(UnifiedLog.scala:805)
>   at kafka.log.UnifiedLog.appendAsLeader(UnifiedLog.scala:719)
>   at 
> kafka.cluster.Partition.$anonfun$appendRecordsToLeader$1(Partition.scala:1313)
>   at kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:1301)
>   at 
> kafka.server.ReplicaManager.$anonfun$appendToLocalLog$6(ReplicaManager.scala:1210)
>   at 
> scala.collection.StrictOptimizedMapOps.map(StrictOptimizedMapOps.scala:28)
>   at 
> scala.collection.StrictOptimizedMapOps.map$(StrictOptimizedMapOps.scala:27)
>   at scala.collection.mutable.HashMap.map(HashMap.scala:35)
>   at 
> kafka.server.ReplicaManager.appendToLocalLog(ReplicaManager.scala:1198)
>   at kafka.server.ReplicaManager.appendEntries$1(ReplicaManager.scala:754)
>   at 
> kafka.server.ReplicaManager.$anonfun$appendRecords$18(ReplicaManager.scala:874)
>   at 
> kafka.server.ReplicaManager.$anonfun$appendRecords$18$adapted(ReplicaManager.scala:874)
>   at 
> kafka.server.KafkaRequestHandler$.$anonfun$wrap$3(KafkaRequestHandler.scala:73)
>   at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:130)
>   at java.base/java.lang.Thread.run(Unknown Source)
> {noformat}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (KAFKA-15657) Unexpected errors when producing transactionally in 3.6

2023-10-19 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-15657?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=1537#comment-1537
 ] 

Travis Bischel commented on KAFKA-15657:


re: first comment – the client doesn't advance to producing unless 
AddPartitionsToTxn succeeds. If the request partially succeeds, failed 
partitions are stripped and only successfully added partitions are produced. 
The logic is definitely hard to follow if you're not familiar with the code, 
but here's issuing/stripping: 
[here|https://github.com/twmb/franz-go/blob/ae169a1f35c2ee6b130c4e520632b33e6c491e0b/pkg/kgo/sink.go#L442-L498,]
 and here's where the request is issued (in the same function as producing – 
before the produce request is issued): 
[here|https://github.com/twmb/franz-go/blob/ae169a1f35c2ee6b130c4e520632b33e6c491e0b/pkg/kgo/sink.go#L316-L357]

Also wrt race condition – these tests also pass against the redpanda binary, 
which has always had KIP-890 semantics / has never allowed transactional 
produce requests unless the partition has been added to the transaction (in 
fact this is part of how I caught some early redpanda bugs with _that_ 
implementation).

 

re: second comment, I'll capture some debug logs so you can see both the client 
logs and the container. The tests currently are using v3. I'm currently running 
this in a loop:

```

docker compose down; sleep 1; docker compose up -d ; sleep 5 ; while go test 
-run Txn/cooperative > logs; do echo whoo; docker compose down; sleep 1; docker 
compose up -d. sleep 5; done

```

Once this fails, I'll upload the logs. This is currently ignoring 
INVALID_RECORD, which I more regularly run into. I may remove gating this to 
just the cooperative test and instead run it against all balancers at once (it 
seems heavier load runs into the problem more frequently).

 

Also this does remind me though, somebody had a feature request that 
deliberately abused the ability to produce before AddPartitionsToTxn was done, 
I need to remove support of this for 3.6+. This _is_ exercised in franz-go's CI 
right now and will fail CI for 3.6+ (see the doc comment on 
[EndBeginTxnUnsafe|https://pkg.go.dev/github.com/twmb/franz-go/pkg/kgo#EndBeginTxnHow]).

> Unexpected errors when producing transactionally in 3.6
> ---
>
> Key: KAFKA-15657
> URL: https://issues.apache.org/jira/browse/KAFKA-15657
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 3.6.0
>Reporter: Travis Bischel
>Priority: Major
>
> In loop-testing the franz-go client, I am frequently receiving INVALID_RECORD 
> (which I created a separate issue for), and INVALID_TXN_STATE and 
> UNKNOWN_SERVER_ERROR.
> INVALID_TXN_STATE is being returned even though the partitions have been 
> added to the transaction (AddPartitionsToTxn). Nothing about the code has 
> changed between 3.5 and 3.6, and I have loop-integration-tested this code 
> against 3.5 thousands of times. 3.6 is newly - and always - returning 
> INVALID_TXN_STATE. If I change the code to retry on INVALID_TXN_STATE, I 
> eventually quickly (always) receive UNKNOWN_SERVER_ERROR. In looking at the 
> broker logs, the broker indicates that sequence numbers are out of order - 
> but (a) I am repeating requests that were in order (so something on the 
> broker got a little haywire maybe? or maybe this is due to me ignoring 
> invalid_txn_state?), _and_ I am not receiving OUT_OF_ORDER_SEQUENCE_NUMBER, I 
> am receiving UNKNOWN_SERVER_ERROR.
> I think the main problem is the client unexpectedly receiving 
> INVALID_TXN_STATE, but a second problem here is that OOOSN is being mapped to 
> USE on return for some reason.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-15657) Unexpected errors when producing transactionally in 3.6

2023-10-19 Thread Travis Bischel (Jira)
Travis Bischel created KAFKA-15657:
--

 Summary: Unexpected errors when producing transactionally in 3.6
 Key: KAFKA-15657
 URL: https://issues.apache.org/jira/browse/KAFKA-15657
 Project: Kafka
  Issue Type: Bug
  Components: producer 
Affects Versions: 3.6.0
Reporter: Travis Bischel


In loop-testing the franz-go client, I am frequently receiving INVALID_RECORD 
(which I created a separate issue for), and INVALID_TXN_STATE and 
UNKNOWN_SERVER_ERROR.

INVALID_TXN_STATE is being returned even though the partitions have been added 
to the transaction (AddPartitionsToTxn). Nothing about the code has changed 
between 3.5 and 3.6, and I have loop-integration-tested this code against 3.5 
thousands of times. 3.6 is newly - and always - returning INVALID_TXN_STATE. If 
I change the code to retry on INVALID_TXN_STATE, I eventually quickly (always) 
receive UNKNOWN_SERVER_ERROR. In looking at the broker logs, the broker 
indicates that sequence numbers are out of order - but (a) I am repeating 
requests that were in order (so something on the broker got a little haywire 
maybe? or maybe this is due to me ignoring invalid_txn_state?), _and_ I am not 
receiving OUT_OF_ORDER_SEQUENCE_NUMBER, I am receiving UNKNOWN_SERVER_ERROR.

I think the main problem is the client unexpectedly receiving 
INVALID_TXN_STATE, but a second problem here is that OOOSN is being mapped to 
USE on return for some reason.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (KAFKA-15656) Frequent INVALID_RECORD on Kafka 3.6

2023-10-19 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-15656?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=1515#comment-1515
 ] 

Travis Bischel commented on KAFKA-15656:


Note that if I change my code to retry on INVALID_RECORD – and repeat the same 
exact serialization – the produce request will succeed when repeated.

> Frequent INVALID_RECORD on Kafka 3.6
> 
>
> Key: KAFKA-15656
> URL: https://issues.apache.org/jira/browse/KAFKA-15656
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 3.6.0
>Reporter: Travis Bischel
>Priority: Major
> Attachments: invalid_record.log
>
>
> Using this docker-compose.yml:
> {noformat}
> version: "3.7"
> services:
>   kafka:
>     image: bitnami/kafka:latest
>     network_mode: host
>     environment:
>       KAFKA_ENABLE_KRAFT: yes
>       KAFKA_CFG_PROCESS_ROLES: controller,broker
>       KAFKA_CFG_CONTROLLER_LISTENER_NAMES: CONTROLLER
>       KAFKA_CFG_LISTENERS: PLAINTEXT://:9092,CONTROLLER://:9093
>       KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP: 
> CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT
>       KAFKA_CFG_CONTROLLER_QUORUM_VOTERS: 1@127.0.0.1:9093
>       # Set this to "PLAINTEXT://127.0.0.1:9092" if you want to run this 
> container on localhost via Docker
>       KAFKA_CFG_ADVERTISED_LISTENERS: PLAINTEXT://localhost:9092
>       KAFKA_CFG_NODE_ID: 1
>       ALLOW_PLAINTEXT_LISTENER: yes
>       KAFKA_KRAFT_CLUSTER_ID: XkpGZQ27R3eTl3OdTm2LYA # 16 byte base64-encoded 
> UUID{noformat}
> And running franz-go integration tests with KGO_TEST_RF=1, I consistently 
> receive INVALID_RECORD errors.
>  
> Looking at the container logs, I see these problematic log lines:
> {noformat}
> 2023-10-19 23:33:47,942] ERROR [ReplicaManager broker=1] Error processing 
> append operation on partition 
> 0cf2f3faaafd3f906ea848b684b04833ca162bcd19ecae2cab36767a54f248c7-0 
> (kafka.server.ReplicaManager) 
> org.apache.kafka.common.InvalidRecordException: Invalid negative header key 
> size -25
> [2023-10-19 23:33:47,942] ERROR [ReplicaManager broker=1] Error processing 
> append operation on partition 
> 0cf2f3faaafd3f906ea848b684b04833ca162bcd19ecae2cab36767a54f248c7-6 
> (kafka.server.ReplicaManager) 
> org.apache.kafka.common.InvalidRecordException: Reached end of input stream 
> before skipping all bytes. Remaining bytes:94
> [2023-10-19 23:33:47,942] ERROR [ReplicaManager broker=1] Error processing 
> append operation on partition 
> 0cf2f3faaafd3f906ea848b684b04833ca162bcd19ecae2cab36767a54f248c7-1 
> (kafka.server.ReplicaManager) 
> org.apache.kafka.common.InvalidRecordException: Found invalid number of 
> record headers -26
> [2023-10-19 23:33:47,948] ERROR [ReplicaManager broker=1] Error processing 
> append operation on partition 
> 0cf2f3faaafd3f906ea848b684b04833ca162bcd19ecae2cab36767a54f248c7-6 
> (kafka.server.ReplicaManager) 
> org.apache.kafka.common.InvalidRecordException: Found invalid number of 
> record headers -27
> [2023-10-19 23:33:47,950] ERROR [ReplicaManager broker=1] Error processing 
> append operation on partition 
> 0cf2f3faaafd3f906ea848b684b04833ca162bcd19ecae2cab36767a54f248c7-22 
> (kafka.server.ReplicaManager)
> org.apache.kafka.common.InvalidRecordException: Invalid negative header key 
> size -25
> [2023-10-19 23:33:47,947] ERROR [ReplicaManager broker=1] Error processing 
> append operation on partition 
> c63b6e30987317fad18815effb8d432b6df677d2ab56cf6da517bb93fa49b74b-25 
> (kafka.server.ReplicaManager)
> org.apache.kafka.common.InvalidRecordException: Found invalid number of 
> record headers -50
> [2023-10-19 23:33:47,959] ERROR [ReplicaManager broker=1] Error processing 
> append operation on partition 
> c63b6e30987317fad18815effb8d432b6df677d2ab56cf6da517bb93fa49b74b-25 
> (kafka.server.ReplicaManager) 
>  {noformat}
>  
> I modified franz-go with a diff to print the request that was written to the 
> wire once this error occurs. Attached is a v9 produce request. I deserialized 
> it locally and am not seeing the corrupt data that Kafka is printing. It's 
> possible there is a bug in the client, but again, these tests have never 
> received this error pre-Kafka 3.6. It _looks like_ there is either corruption 
> when processing the incoming data, or there is some problematic race 
> condition in the broker - I'm not sure which.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-15656) Frequent INVALID_RECORD on Kafka 3.6

2023-10-19 Thread Travis Bischel (Jira)
Travis Bischel created KAFKA-15656:
--

 Summary: Frequent INVALID_RECORD on Kafka 3.6
 Key: KAFKA-15656
 URL: https://issues.apache.org/jira/browse/KAFKA-15656
 Project: Kafka
  Issue Type: Bug
  Components: producer 
Affects Versions: 3.6.0
Reporter: Travis Bischel
 Attachments: invalid_record.log

Using this docker-compose.yml:
{noformat}
version: "3.7"
services:
  kafka:
    image: bitnami/kafka:latest
    network_mode: host
    environment:
      KAFKA_ENABLE_KRAFT: yes
      KAFKA_CFG_PROCESS_ROLES: controller,broker
      KAFKA_CFG_CONTROLLER_LISTENER_NAMES: CONTROLLER
      KAFKA_CFG_LISTENERS: PLAINTEXT://:9092,CONTROLLER://:9093
      KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP: 
CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT
      KAFKA_CFG_CONTROLLER_QUORUM_VOTERS: 1@127.0.0.1:9093
      # Set this to "PLAINTEXT://127.0.0.1:9092" if you want to run this 
container on localhost via Docker
      KAFKA_CFG_ADVERTISED_LISTENERS: PLAINTEXT://localhost:9092
      KAFKA_CFG_NODE_ID: 1
      ALLOW_PLAINTEXT_LISTENER: yes
      KAFKA_KRAFT_CLUSTER_ID: XkpGZQ27R3eTl3OdTm2LYA # 16 byte base64-encoded 
UUID{noformat}
And running franz-go integration tests with KGO_TEST_RF=1, I consistently 
receive INVALID_RECORD errors.

 

Looking at the container logs, I see these problematic log lines:
{noformat}
2023-10-19 23:33:47,942] ERROR [ReplicaManager broker=1] Error processing 
append operation on partition 
0cf2f3faaafd3f906ea848b684b04833ca162bcd19ecae2cab36767a54f248c7-0 
(kafka.server.ReplicaManager) 
org.apache.kafka.common.InvalidRecordException: Invalid negative header key 
size -25
[2023-10-19 23:33:47,942] ERROR [ReplicaManager broker=1] Error processing 
append operation on partition 
0cf2f3faaafd3f906ea848b684b04833ca162bcd19ecae2cab36767a54f248c7-6 
(kafka.server.ReplicaManager) 
org.apache.kafka.common.InvalidRecordException: Reached end of input stream 
before skipping all bytes. Remaining bytes:94
[2023-10-19 23:33:47,942] ERROR [ReplicaManager broker=1] Error processing 
append operation on partition 
0cf2f3faaafd3f906ea848b684b04833ca162bcd19ecae2cab36767a54f248c7-1 
(kafka.server.ReplicaManager) 
org.apache.kafka.common.InvalidRecordException: Found invalid number of record 
headers -26
[2023-10-19 23:33:47,948] ERROR [ReplicaManager broker=1] Error processing 
append operation on partition 
0cf2f3faaafd3f906ea848b684b04833ca162bcd19ecae2cab36767a54f248c7-6 
(kafka.server.ReplicaManager) 
org.apache.kafka.common.InvalidRecordException: Found invalid number of record 
headers -27
[2023-10-19 23:33:47,950] ERROR [ReplicaManager broker=1] Error processing 
append operation on partition 
0cf2f3faaafd3f906ea848b684b04833ca162bcd19ecae2cab36767a54f248c7-22 
(kafka.server.ReplicaManager)
org.apache.kafka.common.InvalidRecordException: Invalid negative header key 
size -25
[2023-10-19 23:33:47,947] ERROR [ReplicaManager broker=1] Error processing 
append operation on partition 
c63b6e30987317fad18815effb8d432b6df677d2ab56cf6da517bb93fa49b74b-25 
(kafka.server.ReplicaManager)
org.apache.kafka.common.InvalidRecordException: Found invalid number of record 
headers -50
[2023-10-19 23:33:47,959] ERROR [ReplicaManager broker=1] Error processing 
append operation on partition 
c63b6e30987317fad18815effb8d432b6df677d2ab56cf6da517bb93fa49b74b-25 
(kafka.server.ReplicaManager) 
 {noformat}
 

I modified franz-go with a diff to print the request that was written to the 
wire once this error occurs. Attached is a v9 produce request. I deserialized 
it locally and am not seeing the corrupt data that Kafka is printing. It's 
possible there is a bug in the client, but again, these tests have never 
received this error pre-Kafka 3.6. It _looks like_ there is either corruption 
when processing the incoming data, or there is some problematic race condition 
in the broker - I'm not sure which.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (KAFKA-14402) Transactions Server Side Defense

2023-10-19 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-14402?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=1512#comment-1512
 ] 

Travis Bischel commented on KAFKA-14402:


Thanks, that is clearer. I note that clients also work with v4 – I'm testing 
that locally – but also I notice locally that occasionally I get 
INVALID_TXN_STATE when I should not (note this test failure: 
[https://github.com/twmb/franz-go/actions/runs/6581193303/job/17880673632?pr=599]).
 In the linked test, the client itself is still sending v3; there is no change 
in the client from the tests that pass on 3.5 and this test that is failing on 
3.6.

 

However, trying to reproduce this locally, I'm running into INVALID_RECORD 
problems as well only on 3.6 which I'll create a separate issue for :|.

> Transactions Server Side Defense
> 
>
> Key: KAFKA-14402
> URL: https://issues.apache.org/jira/browse/KAFKA-14402
> Project: Kafka
>  Issue Type: Improvement
>Affects Versions: 3.5.0
>Reporter: Justine Olshan
>Assignee: Justine Olshan
>Priority: Major
>
> We have seen hanging transactions in Kafka where the last stable offset (LSO) 
> does not update, we can’t clean the log (if the topic is compacted), and 
> read_committed consumers get stuck.
> This can happen when a message gets stuck or delayed due to networking issues 
> or a network partition, the transaction aborts, and then the delayed message 
> finally comes in. The delayed message case can also violate EOS if the 
> delayed message comes in after the next addPartitionsToTxn request comes in. 
> Effectively we may see a message from a previous (aborted) transaction become 
> part of the next transaction.
> Another way hanging transactions can occur is that a client is buggy and may 
> somehow try to write to a partition before it adds the partition to the 
> transaction. In both of these cases, we want the server to have some control 
> to prevent these incorrect records from being written and either causing 
> hanging transactions or violating Exactly once semantics (EOS) by including 
> records in the wrong transaction.
> The best way to avoid this issue is to:
>  # *Uniquely identify transactions by bumping the producer epoch after every 
> commit/abort marker. That way, each transaction can be identified by 
> (producer id, epoch).* 
>  # {*}Remove the addPartitionsToTxn call and implicitly just add partitions 
> to the transaction on the first produce request during a transaction{*}.
> We avoid the late arrival case because the transaction is uniquely identified 
> and fenced AND we avoid the buggy client case because we remove the need for 
> the client to explicitly add partitions to begin the transaction.
> Of course, 1 and 2 require client-side changes, so for older clients, those 
> approaches won’t apply.
> 3. *To cover older clients, we will ensure a transaction is ongoing before we 
> write to a transaction. We can do this by querying the transaction 
> coordinator and caching the result.*
>  
> See KIP-890 for more information: ** 
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-15653) NPE in ChunkedByteStream.

2023-10-19 Thread Travis Bischel (Jira)
Travis Bischel created KAFKA-15653:
--

 Summary: NPE in ChunkedByteStream.
 Key: KAFKA-15653
 URL: https://issues.apache.org/jira/browse/KAFKA-15653
 Project: Kafka
  Issue Type: Bug
  Components: producer 
Affects Versions: 3.6.0
 Environment: Docker container on a Linux laptop, using the latest 
release.
Reporter: Travis Bischel


When looping franz-go integration tests, I received an UNKNOWN_SERVER_ERROR 
from producing. The broker logs for the failing request:

 
{noformat}
[2023-10-19 22:29:58,160] ERROR [ReplicaManager broker=2] Error processing 
append operation on partition 
2fa8995d8002fbfe68a96d783f26aa2c5efc15368bf44ed8f2ab7e24b41b9879-24 
(kafka.server.ReplicaManager)
java.lang.NullPointerException
at 
org.apache.kafka.common.utils.ChunkedBytesStream.(ChunkedBytesStream.java:89)
at 
org.apache.kafka.common.record.CompressionType$3.wrapForInput(CompressionType.java:105)
at 
org.apache.kafka.common.record.DefaultRecordBatch.recordInputStream(DefaultRecordBatch.java:273)
at 
org.apache.kafka.common.record.DefaultRecordBatch.compressedIterator(DefaultRecordBatch.java:277)
at 
org.apache.kafka.common.record.DefaultRecordBatch.skipKeyValueIterator(DefaultRecordBatch.java:352)
at 
org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsetsCompressed(LogValidator.java:358)
at 
org.apache.kafka.storage.internals.log.LogValidator.validateMessagesAndAssignOffsets(LogValidator.java:165)
at kafka.log.UnifiedLog.append(UnifiedLog.scala:805)
at kafka.log.UnifiedLog.appendAsLeader(UnifiedLog.scala:719)
at 
kafka.cluster.Partition.$anonfun$appendRecordsToLeader$1(Partition.scala:1313)
at kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:1301)
at 
kafka.server.ReplicaManager.$anonfun$appendToLocalLog$6(ReplicaManager.scala:1210)
at 
scala.collection.StrictOptimizedMapOps.map(StrictOptimizedMapOps.scala:28)
at 
scala.collection.StrictOptimizedMapOps.map$(StrictOptimizedMapOps.scala:27)
at scala.collection.mutable.HashMap.map(HashMap.scala:35)
at 
kafka.server.ReplicaManager.appendToLocalLog(ReplicaManager.scala:1198)
at kafka.server.ReplicaManager.appendEntries$1(ReplicaManager.scala:754)
at 
kafka.server.ReplicaManager.$anonfun$appendRecords$18(ReplicaManager.scala:874)
at 
kafka.server.ReplicaManager.$anonfun$appendRecords$18$adapted(ReplicaManager.scala:874)
at 
kafka.server.KafkaRequestHandler$.$anonfun$wrap$3(KafkaRequestHandler.scala:73)
at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:130)
at java.base/java.lang.Thread.run(Unknown Source)

{noformat}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (KAFKA-14402) Transactions Server Side Defense

2023-10-19 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-14402?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=1489#comment-1489
 ] 

Travis Bischel commented on KAFKA-14402:


Can the KIP be updated to refer that v5 of AddPartitionsToTxn will be the 
version where clients do _not_ send the request? v4 still requires sending the 
request to the broker. I consistently receive INVALID_TXN_STATE if I do not 
send AddPartitionsToTxn.

> Transactions Server Side Defense
> 
>
> Key: KAFKA-14402
> URL: https://issues.apache.org/jira/browse/KAFKA-14402
> Project: Kafka
>  Issue Type: Improvement
>Affects Versions: 3.5.0
>Reporter: Justine Olshan
>Assignee: Justine Olshan
>Priority: Major
>
> We have seen hanging transactions in Kafka where the last stable offset (LSO) 
> does not update, we can’t clean the log (if the topic is compacted), and 
> read_committed consumers get stuck.
> This can happen when a message gets stuck or delayed due to networking issues 
> or a network partition, the transaction aborts, and then the delayed message 
> finally comes in. The delayed message case can also violate EOS if the 
> delayed message comes in after the next addPartitionsToTxn request comes in. 
> Effectively we may see a message from a previous (aborted) transaction become 
> part of the next transaction.
> Another way hanging transactions can occur is that a client is buggy and may 
> somehow try to write to a partition before it adds the partition to the 
> transaction. In both of these cases, we want the server to have some control 
> to prevent these incorrect records from being written and either causing 
> hanging transactions or violating Exactly once semantics (EOS) by including 
> records in the wrong transaction.
> The best way to avoid this issue is to:
>  # *Uniquely identify transactions by bumping the producer epoch after every 
> commit/abort marker. That way, each transaction can be identified by 
> (producer id, epoch).* 
>  # {*}Remove the addPartitionsToTxn call and implicitly just add partitions 
> to the transaction on the first produce request during a transaction{*}.
> We avoid the late arrival case because the transaction is uniquely identified 
> and fenced AND we avoid the buggy client case because we remove the need for 
> the client to explicitly add partitions to begin the transaction.
> Of course, 1 and 2 require client-side changes, so for older clients, those 
> approaches won’t apply.
> 3. *To cover older clients, we will ensure a transaction is ongoing before we 
> write to a transaction. We can do this by querying the transaction 
> coordinator and caching the result.*
>  
> See KIP-890 for more information: ** 
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (KAFKA-7739) Kafka Tiered Storage

2023-03-29 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-7739?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17706496#comment-17706496
 ] 

Travis Bischel commented on KAFKA-7739:
---

[~showuon] thanks. How does that affect the KIP? If I'm to use the KIP as a 
client author, currently, I'll implement something incorrectly.

> Kafka Tiered Storage
> 
>
> Key: KAFKA-7739
> URL: https://issues.apache.org/jira/browse/KAFKA-7739
> Project: Kafka
>  Issue Type: New Feature
>  Components: core
>Reporter: Harsha
>Assignee: Satish Duggana
>Priority: Major
>  Labels: needs-kip
> Fix For: 3.5.0
>
>
> KIP: 
> [https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (KAFKA-9803) Allow producers to recover gracefully from transaction timeouts

2023-02-08 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9803?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17686015#comment-17686015
 ] 

Travis Bischel commented on KAFKA-9803:
---

Would it be possible to move this forward? It looks like timed out transactions 
are still unrecoverable as of 3.4, due to the TRANSACTION_TIMED_OUT error 
return from EndTransactionRequest not yet being implemented.

> Allow producers to recover gracefully from transaction timeouts
> ---
>
> Key: KAFKA-9803
> URL: https://issues.apache.org/jira/browse/KAFKA-9803
> Project: Kafka
>  Issue Type: Improvement
>  Components: producer , streams
>Reporter: Jason Gustafson
>Assignee: Boyang Chen
>Priority: Major
>  Labels: needs-kip
>
> Transaction timeouts are detected by the transaction coordinator. When the 
> coordinator detects a timeout, it bumps the producer epoch and aborts the 
> transaction. The epoch bump is necessary in order to prevent the current 
> producer from being able to begin writing to a new transaction which was not 
> started through the coordinator.  
> Transactions may also be aborted if a new producer with the same 
> `transactional.id` starts up. Similarly this results in an epoch bump. 
> Currently the coordinator does not distinguish these two cases. Both will end 
> up as a `ProducerFencedException`, which means the producer needs to shut 
> itself down. 
> We can improve this with the new APIs from KIP-360. When the coordinator 
> times out a transaction, it can remember that fact and allow the existing 
> producer to claim the bumped epoch and continue. Roughly the logic would work 
> like this:
> 1. When a transaction times out, set lastProducerEpoch to the current epoch 
> and do the normal bump.
> 2. Any transactional requests from the old epoch result in a new 
> TRANSACTION_TIMED_OUT error code, which is propagated to the application.
> 3. The producer recovers by sending InitProducerId with the current epoch. 
> The coordinator returns the bumped epoch.
> One issue that needs to be addressed is how to handle INVALID_PRODUCER_EPOCH 
> from Produce requests. Partition leaders will not generally know if a bumped 
> epoch was the result of a timed out transaction or a fenced producer. 
> Possibly the producer can treat these errors as abortable when they come from 
> Produce responses. In that case, the user would try to abort the transaction 
> and then we can see if it was due to a timeout or otherwise.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (KAFKA-7739) Kafka Tiered Storage

2023-01-02 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-7739?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17653624#comment-17653624
 ] 

Travis Bischel commented on KAFKA-7739:
---

Hi there, this work looks excellent. Can the KIP be updated to change earliest 
log start offset from -3 to -4 (since -3 was taken for max timestamp)?

> Kafka Tiered Storage
> 
>
> Key: KAFKA-7739
> URL: https://issues.apache.org/jira/browse/KAFKA-7739
> Project: Kafka
>  Issue Type: New Feature
>  Components: core
>Reporter: Harsha
>Assignee: Satish Duggana
>Priority: Major
>  Labels: needs-kip
> Fix For: 3.5.0
>
>
> KIP: 
> [https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (KAFKA-10442) Tooling to detect and abort hanging transactions (KIP-664)

2022-11-26 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-10442?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17639566#comment-17639566
 ] 

Travis Bischel commented on KAFKA-10442:


Can the KIP be updated to remove references to WriteTxnMarker changes that are 
no longer necessary?

> Tooling to detect and abort hanging transactions (KIP-664)
> --
>
> Key: KAFKA-10442
> URL: https://issues.apache.org/jira/browse/KAFKA-10442
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Jason Gustafson
>Assignee: Jason Gustafson
>Priority: Major
>
> This JIRA tracks the implementation of KIP-664: 
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-664%3A+Provide+tooling+to+detect+and+abort+hanging+transactions.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14377) Expose Broker's Name and Version to Clients

2022-11-10 Thread Travis Bischel (Jira)
Travis Bischel created KAFKA-14377:
--

 Summary: Expose Broker's Name and Version to Clients
 Key: KAFKA-14377
 URL: https://issues.apache.org/jira/browse/KAFKA-14377
 Project: Kafka
  Issue Type: Improvement
Reporter: Travis Bischel


Implement KIP-885 as documented here: 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-885%3A+Expose+Broker%27s+Name+and+Version+to+Clients



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (KAFKA-13435) Static membership protocol should let the leader skip assignment (KIP-814)

2022-10-19 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-13435?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17620653#comment-17620653
 ] 

Travis Bischel commented on KAFKA-13435:


KIP-814 doesn't have a solution to KAFKA-12759 mentioned above. If the leader 
is turned off, has its assignment switched, and rejoins, the leader will 
receive its old assignment. KIP-814 says that a rebalance should not occur.

If a leader is assigned "foo", turns off, changes its configuration to be 
assigned "bar", and turns back on, the broker will tell the leader that its 
current assignment is "foo". Worse, the JoinGroupResponse will indicate that 
"bar" is the member assignment, and then the SyncGroupResponse will say "you 
own foo" even though the leader is not interested.

Rather than returning leader's new protocol metadata, the broker should return 
the old leader protocol metadata. This would allow the leader to detect if its 
metadata has changed and if it should continue with a full balance and 
assignment.

I'll open a new issue for this.

> Static membership protocol should let the leader skip assignment (KIP-814)
> --
>
> Key: KAFKA-13435
> URL: https://issues.apache.org/jira/browse/KAFKA-13435
> Project: Kafka
>  Issue Type: Bug
>  Components: consumer
>Affects Versions: 3.0.0
>Reporter: Ryan Leslie
>Assignee: David Jacot
>Priority: Critical
>  Labels: new-rebalance-should-fix
> Fix For: 3.2.0
>
>
> When using consumer groups with static membership, if the consumer marked as 
> leader has restarted, then metadata changes such as partition increase are 
> not triggering expected rebalances.
> To reproduce this issue, simply:
>  # Create a static consumer subscribed to a single topic
>  # Close the consumer and create a new one with the same group instance id
>  # Increase partitions for the topic
>  # Observe that no rebalance occurs and the new partitions are not assigned
> I have only tested this in 2.7, but it may apply to newer versions as well.
> h3. Analysis
> In {_}ConsumerCoordinator{_}, one responsibility of the leader consumer is to 
> track metadata and trigger a rebalance if there are changes such as new 
> partitions added:
> [https://github.com/apache/kafka/blob/43bcc5682da82a602a4c0a000dc7433d0507b450/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java#L793]
> {code:java}
> if (assignmentSnapshot != null && 
> !assignmentSnapshot.matches(metadataSnapshot)) {
> ...
> requestRejoinIfNecessary(reason);
> return true;
> }
> {code}
> Note that _assignmentSnapshot_ is currently only set if the consumer is the 
> leader:
> [https://github.com/apache/kafka/blob/43bcc5682da82a602a4c0a000dc7433d0507b450/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java#L353]
> {code:java}
> // Only the leader is responsible for monitoring for metadata changes (i.e. 
> partition changes)
> if (!isLeader)
> assignmentSnapshot = null;
> {code}
> And _isLeader_ is only true after an assignment is performed during a 
> rebalance:
> [https://github.com/apache/kafka/blob/43bcc5682da82a602a4c0a000dc7433d0507b450/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java#L634]
> That is, when a consumer group forms, exactly one consumer in the group 
> should have _isLeader == True_ and be responsible for triggering rebalances 
> on metadata changes.
> However, in the case of static membership, if the leader has been restarted 
> and rejoined the group, the group essentially no longer has a current leader. 
> Even though the metadata changes are fetched, no rebalance will be triggered. 
> That is, _isLeader_ will be false for all members.
> This issue does not resolve until after an actual group change that causes a 
> proper rebalance. In order to safely make a partition increase when using 
> static membership, consumers must be stopped and have timed out, or forcibly 
> removed with {_}AdminClient.removeMembersFromConsumerGroup(){_}.
> Correcting this in the client probably also requires help from the broker. 
> Currently, when a static consumer that is leader is restarted, the 
> coordinator does recognize the change:
> e.g. leader _bbfcb930-61a3-4d21-945c-85f4576490ff_ was restarted
> {noformat}
> [2021-11-04 13:53:13,487] INFO [GroupCoordinator 4]: Static member 
> Some(1GK7DRJPHZ0LRV91Y4D3SYHS5928XHXJQ6263GT26V5P70QX0) of group ryan_test 
> with unknown member id rejoins, assigning new member id 
> 1GK7DRJPHZ0LRV91Y4D3SYHS5928XHXJQ6263GT26V5P70QX0-af88ecf2-
> 6ebf-47da-95ef-c54fef17ab74, while old member id 
> 1GK7DRJPHZ0LRV91Y4D3SYHS5928XHXJQ6263GT26V5P70QX0-bbfcb930-61a3-4d21-945c-85f4576490ff
>  will be removed. (
> 

[jira] [Commented] (KAFKA-14315) Kraft: 1 broker setup, broker took 34 seconds to transition from PrepareCommit to CompleteCommit

2022-10-18 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-14315?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17619729#comment-17619729
 ] 

Travis Bischel commented on KAFKA-14315:


Note that I've also had requests repeatedly failing with 
CONCURRENT_TRANSACTIONS – for upwards of 40 seconds – my guess is that these 
are related.

> Kraft: 1 broker setup, broker took 34 seconds to transition from 
> PrepareCommit to CompleteCommit
> 
>
> Key: KAFKA-14315
> URL: https://issues.apache.org/jira/browse/KAFKA-14315
> Project: Kafka
>  Issue Type: Bug
>  Components: kraft
>Reporter: Travis Bischel
>Priority: Minor
>
> I'm still looking into a PR failure in [my 
> client|https://github.com/twmb/franz-go/pull/223] and noticed something a bit 
> strange. I know that _technically_ I should be using 
> RequireStableFetchOffsets in my transaction tests to prevent rebalances while 
> a transaction is not finalized. I'll be adding that.
> However, these tests have never failed against zookeeper mode. The client 
> goes through a lot of efforts to avoid needing KIP-447 behavior, and the 
> assumption with localhost testing is that things run fast enough (and that 
> there are enough guards) that problems would not be encountered.
> That looks to not be true with a kraft broker, but looking at 
> __transaction_state, the following looks to be especially problematic:
>  
> {{__transaction_state partition 33 offset 7 at [2022-10-18 11:15:37.821]}}
> {{TxnMetadataKey(0) 
> 9f87dc04dc3f4d5b15ef3072c531cf46327278307df8e149fa966462cd40c10b}}
> {{TxnMetadataValue(0)}}
> {{      ProducerID           41}}
> {{      ProducerEpoch        0}}
> {{      TimeoutMillis        12}}
> {{      State                PrepareCommit}}
> {{      Topics               __consumer_offsets=>[13] 
> e7c7d971626fbaf4bfb33975e57089167939e6acabb4c4fc534eb148462e45cc=>[4 5 12 16] 
>  }}
> {{      LastUpdateTimestamp  1666113337821}}
> {{      StartTimestamp       1666113335311}}
> {{__transaction_state partition 33 offset 8 at [2022-10-18 11:16:11.419]}}
> {{TxnMetadataKey(0) 
> 9f87dc04dc3f4d5b15ef3072c531cf46327278307df8e149fa966462cd40c10b}}
> {{TxnMetadataValue(0)}}
> {{      ProducerID           41}}
> {{      ProducerEpoch        0}}
> {{      TimeoutMillis        12}}
> {{      State                CompleteCommit}}
> {{      Topics     }}
> {{      LastUpdateTimestamp  1666113337821}}
> {{      StartTimestamp       1666113335311}}
>  
> I've captured that using my kcl tool.
> Note that the transaction enters PrepareCommit at 11:15:37.821, and then 
> enters CompleteCommit at 11:16:11.419. AFAICT, this means that in my single 
> node kraft setup, the broker took 34 seconds to transition commit states 
> internally.
> I noticed this in tests because a rebalance happened between those 34 
> seconds, which caused duplicate consumption because transactional offset 
> commits were not finalized and the old commits were picked up.
> This ticket is related to KAFKA-14312, in that this failure is cropping up as 
> I've worked around KAFKA-14312 within the client itself.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14315) Kraft: 1 broker setup, broker took 34 seconds to transition from PrepareCommit to CompleteCommit

2022-10-18 Thread Travis Bischel (Jira)
Travis Bischel created KAFKA-14315:
--

 Summary: Kraft: 1 broker setup, broker took 34 seconds to 
transition from PrepareCommit to CompleteCommit
 Key: KAFKA-14315
 URL: https://issues.apache.org/jira/browse/KAFKA-14315
 Project: Kafka
  Issue Type: Bug
  Components: kraft
Reporter: Travis Bischel


I'm still looking into a PR failure in [my 
client|https://github.com/twmb/franz-go/pull/223] and noticed something a bit 
strange. I know that _technically_ I should be using RequireStableFetchOffsets 
in my transaction tests to prevent rebalances while a transaction is not 
finalized. I'll be adding that.

However, these tests have never failed against zookeeper mode. The client goes 
through a lot of efforts to avoid needing KIP-447 behavior, and the assumption 
with localhost testing is that things run fast enough (and that there are 
enough guards) that problems would not be encountered.

That looks to not be true with a kraft broker, but looking at 
__transaction_state, the following looks to be especially problematic:

 

{{__transaction_state partition 33 offset 7 at [2022-10-18 11:15:37.821]}}
{{TxnMetadataKey(0) 
9f87dc04dc3f4d5b15ef3072c531cf46327278307df8e149fa966462cd40c10b}}
{{TxnMetadataValue(0)}}
{{      ProducerID           41}}
{{      ProducerEpoch        0}}
{{      TimeoutMillis        12}}
{{      State                PrepareCommit}}
{{      Topics               __consumer_offsets=>[13] 
e7c7d971626fbaf4bfb33975e57089167939e6acabb4c4fc534eb148462e45cc=>[4 5 12 16]  
}}
{{      LastUpdateTimestamp  1666113337821}}
{{      StartTimestamp       1666113335311}}
{{__transaction_state partition 33 offset 8 at [2022-10-18 11:16:11.419]}}
{{TxnMetadataKey(0) 
9f87dc04dc3f4d5b15ef3072c531cf46327278307df8e149fa966462cd40c10b}}
{{TxnMetadataValue(0)}}
{{      ProducerID           41}}
{{      ProducerEpoch        0}}
{{      TimeoutMillis        12}}
{{      State                CompleteCommit}}
{{      Topics     }}
{{      LastUpdateTimestamp  1666113337821}}
{{      StartTimestamp       1666113335311}}

 

I've captured that using my kcl tool.

Note that the transaction enters PrepareCommit at 11:15:37.821, and then enters 
CompleteCommit at 11:16:11.419. AFAICT, this means that in my single node kraft 
setup, the broker took 34 seconds to transition commit states internally.

I noticed this in tests because a rebalance happened between those 34 seconds, 
which caused duplicate consumption because transactional offset commits were 
not finalized and the old commits were picked up.

This ticket is related to KAFKA-14312, in that this failure is cropping up as 
I've worked around KAFKA-14312 within the client itself.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (KAFKA-14312) Kraft + ProducerStateManager: produce requests to new partitions with a non-zero sequence number should be rejected

2022-10-17 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-14312?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17619197#comment-17619197
 ] 

Travis Bischel commented on KAFKA-14312:


Agreed, the comment indicates another such scenario where a non-zero sequence 
number can occur.

Unfortunately, it looks to interact badly with what looks like Kraft's slowish 
staged topic setup. Perhaps solving KAFKA-14313 would largely mitigate this 
issue.

The theoretical problem remains:
 * Client sends two requests in flight to a broker
 * Broker is not leader while receiving the first request, is leader while 
receiving the second
 * Broker appends the later request, does not append the former request
 * Client repeats both requests, receives out of order sequence number

 

 

> Kraft + ProducerStateManager: produce requests to new partitions with a 
> non-zero sequence number should be rejected
> ---
>
> Key: KAFKA-14312
> URL: https://issues.apache.org/jira/browse/KAFKA-14312
> Project: Kafka
>  Issue Type: Bug
>  Components: kraft, producer 
>Reporter: Travis Bischel
>Priority: Major
>
> h1. Background
> In Kraft mode, if I create a topic, I am occasionally seeing MetadataResponse 
> with a valid leader, and if I immediately produce to that topic, I am seeing 
> NOT_LEADER_FOR_PARTITION. There may be another bug causing Kraft to return a 
> leader in metadata but reject requests to that leader, _but_ this is showing 
> a bigger problem.
> Kafka currently accepts produce requests to new partitions with a non-zero 
> sequence number. I have confirmed this locally by modifying my client to 
> start producing with a sequence number of 10. Producing three records 
> sequentially back to back (seq 10, 11, 12) are all successful. I _think_ this 
> [comment|https://github.com/apache/kafka/blob/3e7eddecd6a63ea6a9793d3270bef6d0be5c9021/core/src/main/scala/kafka/log/ProducerStateManager.scala#L235-L236]
>  in the Kafka source also indicates roughly the same thing.
> h1. Problem
>  * Client initializes producer ID
>  * Client creates topic "foo" (for the problem, we will ignore partitions – 
> there is just one partition)
>  * Client sends produce request A with 5 records
>  * Client sends produce request B with 5 records before receiving a response 
> for A
>  * Broker returns NOT_LEADER_FOR_PARTITION to produce request A
>  * Broker finally initializes, becomes leader before seeing request B
>  * Broker accepts request B as the first request
>  * Broker believes sequence number 5 is ok, and is expecting the next 
> sequence to be 10
>  * Client retries requests A and B, because A failed
>  * Broker sees request A with sequence 0, returns OutOfOrderSequenceException
>  * Client enters a fatal state, because OOOSN is not retryable
> h1. Reproducing
> I can reliably reproduce this error using Kraft mode with 1 broker. I am 
> using the following docker compose:
> {{version: "3.7"}}
> {{services:}}
> {{  kafka:}}
> {{    image: bitnami/kafka:latest}}
> {{    network_mode: host}}
> {{    environment:}}
> {{      KAFKA_ENABLE_KRAFT: yes}}
> {{      KAFKA_CFG_PROCESS_ROLES: controller,broker}}
> {{      KAFKA_CFG_CONTROLLER_LISTENER_NAMES: CONTROLLER}}
> {{      KAFKA_CFG_LISTENERS: PLAINTEXT://:9092,CONTROLLER://:9093}}
> {{      KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP: 
> CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT}}
> {{      KAFKA_CFG_CONTROLLER_QUORUM_VOTERS: 1@127.0.0.1:9093}}
> {{      # Set this to "PLAINTEXT://127.0.0.1:9092" if you want to run this 
> container on localhost via Docker}}
> {{      KAFKA_CFG_ADVERTISED_LISTENERS: PLAINTEXT://127.0.0.1:9092}}
> {{      KAFKA_CFG_BROKER_ID: 1}}
> {{      ALLOW_PLAINTEXT_LISTENER: yes}}
> {{      KAFKA_KRAFT_CLUSTER_ID: XkpGZQ27R3eTl3OdTm2LYA # 16 byte 
> base64-encoded UUID}}
> {{      BITNAMI_DEBUG: true # Enable this to get more info on startup 
> failures}}
>  
> I am running the franz-go integration tests to trigger this (frequently, but 
> not all of the time). However, these tests are not required. The behavior 
> described above can occasionally reproduce this.
> I have never experienced this against the zookeeper version. It seems that 
> the zk version always fully initializes a topic immediately and does not 
> return NOT_LEADER_FOR_PARTITION on the first produce request. This is a 
> separate problem – but the main problem described above exists in all 
> versions, and _can_ be experienced in zk in very strange circumstances.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (KAFKA-14312) Kraft + ProducerStateManager: produce requests to new partitions with a non-zero sequence number should be rejected

2022-10-17 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-14312?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17619169#comment-17619169
 ] 

Travis Bischel commented on KAFKA-14312:


My investigation for this issue comes from my own repo's issue, 
[https://github.com/twmb/franz-go/pull/223.] 

> Kraft + ProducerStateManager: produce requests to new partitions with a 
> non-zero sequence number should be rejected
> ---
>
> Key: KAFKA-14312
> URL: https://issues.apache.org/jira/browse/KAFKA-14312
> Project: Kafka
>  Issue Type: Bug
>  Components: kraft, producer 
>Reporter: Travis Bischel
>Priority: Major
>
> h1. Background
> In Kraft mode, if I create a topic, I am occasionally seeing MetadataResponse 
> with a valid leader, and if I immediately produce to that topic, I am seeing 
> NOT_LEADER_FOR_PARTITION. There may be another bug causing Kraft to return a 
> leader in metadata but reject requests to that leader, _but_ this is showing 
> a bigger problem.
> Kafka currently accepts produce requests to new partitions with a non-zero 
> sequence number. I have confirmed this locally by modifying my client to 
> start producing with a sequence number of 10. Producing three records 
> sequentially back to back (seq 10, 11, 12) are all successful. I _think_ this 
> [comment|https://github.com/apache/kafka/blob/3e7eddecd6a63ea6a9793d3270bef6d0be5c9021/core/src/main/scala/kafka/log/ProducerStateManager.scala#L235-L236]
>  in the Kafka source also indicates roughly the same thing.
> h1. Problem
>  * Client initializes producer ID
>  * Client creates topic "foo" (for the problem, we will ignore partitions – 
> there is just one partition)
>  * Client sends produce request A with 5 records
>  * Client sends produce request B with 5 records before receiving a response 
> for A
>  * Broker returns NOT_LEADER_FOR_PARTITION to produce request A
>  * Broker finally initializes, becomes leader before seeing request B
>  * Broker accepts request B as the first request
>  * Broker believes sequence number 5 is ok, and is expecting the next 
> sequence to be 10
>  * Client retries requests A and B, because A failed
>  * Broker sees request A with sequence 0, returns OutOfOrderSequenceException
>  * Client enters a fatal state, because OOOSN is not retryable
> h1. Reproducing
> I can reliably reproduce this error using Kraft mode with 1 broker. I am 
> using the following docker compose:
> {{version: "3.7"}}
> {{services:}}
> {{  kafka:}}
> {{    image: bitnami/kafka:latest}}
> {{    network_mode: host}}
> {{    environment:}}
> {{      KAFKA_ENABLE_KRAFT: yes}}
> {{      KAFKA_CFG_PROCESS_ROLES: controller,broker}}
> {{      KAFKA_CFG_CONTROLLER_LISTENER_NAMES: CONTROLLER}}
> {{      KAFKA_CFG_LISTENERS: PLAINTEXT://:9092,CONTROLLER://:9093}}
> {{      KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP: 
> CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT}}
> {{      KAFKA_CFG_CONTROLLER_QUORUM_VOTERS: 1@127.0.0.1:9093}}
> {{      # Set this to "PLAINTEXT://127.0.0.1:9092" if you want to run this 
> container on localhost via Docker}}
> {{      KAFKA_CFG_ADVERTISED_LISTENERS: PLAINTEXT://127.0.0.1:9092}}
> {{      KAFKA_CFG_BROKER_ID: 1}}
> {{      ALLOW_PLAINTEXT_LISTENER: yes}}
> {{      KAFKA_KRAFT_CLUSTER_ID: XkpGZQ27R3eTl3OdTm2LYA # 16 byte 
> base64-encoded UUID}}
> {{      BITNAMI_DEBUG: true # Enable this to get more info on startup 
> failures}}
>  
> I am running the franz-go integration tests to trigger this (frequently, but 
> not all of the time). However, these tests are not required. The behavior 
> described above can occasionally reproduce this.
> I have never experienced this against the zookeeper version. It seems that 
> the zk version always fully initializes a topic immediately and does not 
> return NOT_LEADER_FOR_PARTITION on the first produce request. This is a 
> separate problem – but the main problem described above exists in all 
> versions, and _can_ be experienced in zk in very strange circumstances.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14313) Kraft: immediately producing to new topics occasionally returns NOT_LEADER_FOR_PARTITION

2022-10-17 Thread Travis Bischel (Jira)
Travis Bischel created KAFKA-14313:
--

 Summary: Kraft: immediately producing to new topics occasionally 
returns NOT_LEADER_FOR_PARTITION
 Key: KAFKA-14313
 URL: https://issues.apache.org/jira/browse/KAFKA-14313
 Project: Kafka
  Issue Type: Bug
  Components: kraft, producer 
Reporter: Travis Bischel


Related issue: KAFKA-14312

See the related issue for the full problem description. This issue is to track 
a _slightly_ less important issue.

In Kraft mode, if I create a topic, sometimes immediate produce requests are 
rejected with NOT_LEADER_FOR_PARTITION

 

Scenario:
 * Client creates topic
 * Client loads metadata for topic, receives leader 1
 * Client produces to broker 1
 * Client receives NOT_LEADER_FOR_PARTITION

If the client waits a little bit, the broker eventually does become the leader.

The above is tested in a 1 broker scenario; my docker-compose.yaml is in the 
linked ticket.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14312) Kraft + ProducerStateManager: produce requests to new partitions with a non-zero sequence number should be rejected

2022-10-17 Thread Travis Bischel (Jira)
Travis Bischel created KAFKA-14312:
--

 Summary: Kraft + ProducerStateManager: produce requests to new 
partitions with a non-zero sequence number should be rejected
 Key: KAFKA-14312
 URL: https://issues.apache.org/jira/browse/KAFKA-14312
 Project: Kafka
  Issue Type: Bug
  Components: kraft, producer 
Reporter: Travis Bischel


h1. Background

In Kraft mode, if I create a topic, I am occasionally seeing MetadataResponse 
with a valid leader, and if I immediately produce to that topic, I am seeing 
NOT_LEADER_FOR_PARTITION. There may be another bug causing Kraft to return a 
leader in metadata but reject requests to that leader, _but_ this is showing a 
bigger problem.

Kafka currently accepts produce requests to new partitions with a non-zero 
sequence number. I have confirmed this locally by modifying my client to start 
producing with a sequence number of 10. Producing three records sequentially 
back to back (seq 10, 11, 12) are all successful. I _think_ this 
[comment|https://github.com/apache/kafka/blob/3e7eddecd6a63ea6a9793d3270bef6d0be5c9021/core/src/main/scala/kafka/log/ProducerStateManager.scala#L235-L236]
 in the Kafka source also indicates roughly the same thing.
h1. Problem
 * Client initializes producer ID
 * Client creates topic "foo" (for the problem, we will ignore partitions – 
there is just one partition)
 * Client sends produce request A with 5 records
 * Client sends produce request B with 5 records before receiving a response 
for A
 * Broker returns NOT_LEADER_FOR_PARTITION to produce request A
 * Broker finally initializes, becomes leader before seeing request B
 * Broker accepts request B as the first request
 * Broker believes sequence number 5 is ok, and is expecting the next sequence 
to be 10
 * Client retries requests A and B, because A failed
 * Broker sees request A with sequence 0, returns OutOfOrderSequenceException
 * Client enters a fatal state, because OOOSN is not retryable

h1. Reproducing

I can reliably reproduce this error using Kraft mode with 1 broker. I am using 
the following docker compose:

{{version: "3.7"}}
{{services:}}
{{  kafka:}}
{{    image: bitnami/kafka:latest}}
{{    network_mode: host}}
{{    environment:}}
{{      KAFKA_ENABLE_KRAFT: yes}}
{{      KAFKA_CFG_PROCESS_ROLES: controller,broker}}
{{      KAFKA_CFG_CONTROLLER_LISTENER_NAMES: CONTROLLER}}
{{      KAFKA_CFG_LISTENERS: PLAINTEXT://:9092,CONTROLLER://:9093}}
{{      KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP: 
CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT}}
{{      KAFKA_CFG_CONTROLLER_QUORUM_VOTERS: 1@127.0.0.1:9093}}
{{      # Set this to "PLAINTEXT://127.0.0.1:9092" if you want to run this 
container on localhost via Docker}}
{{      KAFKA_CFG_ADVERTISED_LISTENERS: PLAINTEXT://127.0.0.1:9092}}
{{      KAFKA_CFG_BROKER_ID: 1}}
{{      ALLOW_PLAINTEXT_LISTENER: yes}}
{{      KAFKA_KRAFT_CLUSTER_ID: XkpGZQ27R3eTl3OdTm2LYA # 16 byte base64-encoded 
UUID}}
{{      BITNAMI_DEBUG: true # Enable this to get more info on startup failures}}

 

I am running the franz-go integration tests to trigger this (frequently, but 
not all of the time). However, these tests are not required. The behavior 
described above can occasionally reproduce this.

I have never experienced this against the zookeeper version. It seems that the 
zk version always fully initializes a topic immediately and does not return 
NOT_LEADER_FOR_PARTITION on the first produce request. This is a separate 
problem – but the main problem described above exists in all versions, and 
_can_ be experienced in zk in very strange circumstances.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (KAFKA-14048) The Next Generation of the Consumer Rebalance Protocol

2022-07-07 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-14048?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17563831#comment-17563831
 ] 

Travis Bischel commented on KAFKA-14048:


>From a technical perspective, this is a very interesting and compelling 
>proposal.

Can the KIP list be updated to include this? I've noticed the KIP list has not 
been updated much and there are a good few KIPs missing. Also, there is no 
public discussion link: has there been any public discussion on this proposal?

> The Next Generation of the Consumer Rebalance Protocol
> --
>
> Key: KAFKA-14048
> URL: https://issues.apache.org/jira/browse/KAFKA-14048
> Project: Kafka
>  Issue Type: Improvement
>Reporter: David Jacot
>Assignee: David Jacot
>Priority: Major
>
> This Jira tracks the development of KIP-848: 
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-848%3A+The+Next+Generation+of+the+Consumer+Rebalance+Protocol.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (KAFKA-12671) Out of order processing with a transactional producer can lead to a stuck LastStableOffset

2022-05-25 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12671?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17542294#comment-17542294
 ] 

Travis Bischel commented on KAFKA-12671:


I did a bit of  followup while adding a niche end-and-begin transaction feature 
to my client. This is what was found:

 
 * If a transactional produce happens outside of the context of 
AddPartitionsToTxn, a transaction is begun within Kafka's ProducerStateManager, 
but not within the TxnCoordinator. This is what begins a zombie transaction. 
There are fixes, though.
 * If the client _eventually_ adds the partitions to a transaction, the 
transaction is begun in the TxnCoordinator and the previous zombie transaction 
is cleared.
 * If the client restarts and re-initializes a producer ID and then begins a 
transaction through AddPartitionsToTxn, the previous zombie transaction is 
cleared.

 

Basically, this issue is isolated to the case where a client takes the 
following path:
 * InitProducerID, with transactional ID
 * Produce ({_}skipping AddPartitionsToTxn){_}
 * Quit, and forever abandon the transactional ID

 

If the transactional ID is ever used again, this _should_ clear any previous 
zombie transaction state. Until the zombie transaction state is cleared, the 
partition cannot be consumed in a READ_COMMITTED manner.

> Out of order processing with a transactional producer can lead to a stuck 
> LastStableOffset
> --
>
> Key: KAFKA-12671
> URL: https://issues.apache.org/jira/browse/KAFKA-12671
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.4.0, 2.5.0, 2.6.0, 2.7.0, 2.8.0
>Reporter: Travis Bischel
>Priority: Major
>  Labels: Transactions
>
> If there is pathological processing of incoming produce requests and EndTxn 
> requests, then the LastStableOffset can get stuck, which will block consuming 
> in READ_COMMITTED mode.
> To transactionally produce, the standard flow is to InitProducerId, and then 
> loop AddPartitionsToTxn -> Produce+ -> EndTxn. The AddPartitionsToTxn is 
> responsible for fencing and adding partitions to a transaction, and the end 
> transaction is responsible for finishing the transaction. Producing itself is 
> mostly uninvolved with the proper fencing / ending flow, but produce requests 
> are required to be after AddPartitionsToTxn and before EndTxn.
> When a ProduceRequest is handled, Kafka uses an internal ProducerStateManager 
> to mildly manage transactions. The ProducerStateManager is completely 
> independent of the TxnCoordinator, and its guarantees are relatively weak. 
> The ProducerStateManager handles two types of "batches" being added: a data 
> batch and a transaction marker. When a data batch is added, a "transaction" 
> is begun and tied to the producer ID that is producing the batch. When a 
> transaction marker is handled, the ProducerStateManager removes the 
> transaction for the producer ID (roughly).
> EndTxn is what triggers transaction markers to be sent to the 
> ProducerStateManager. In essence, EndTxn is the one part of the transactional 
> producer flow that talks across both the TxnCoordinator and the 
> ProducerStateManager.
> If a ProduceRequest is issued before EndTxn, but handled internally in Kafka 
> after EndTxn, then the ProduceRequest will begin a new transaction in the 
> ProducerStateManager. If the client was disconnecting, and the EndTxn was the 
> final request issued, the new transaction created in ProducerStateManager is 
> orphaned and nothing can clean it up. The LastStableOffset then hangs based 
> off of this hung transaction.
> This same problem can be triggered by a produce request that is issued with a 
> transactional ID outside of the context of a transaction at all (no 
> AddPartitionsToTxn). This problem cannot be triggered by producing for so 
> long that the transaction expires; the difference here is that the 
> transaction coordinator bumps the epoch for the producer ID, thus producing 
> again with the old epoch does not work.
> Theoretically, we are supposed have unlimited retries on produce requests, 
> but in the context of wanting to abort everything and shut down, this is not 
> always feasible. As it currently stands, I'm not sure there's a truly safe 
> way to shut down _without_ flushing and receiving responses for every record 
> produced, even if I want to abort everything and quit. The safest approach I 
> can think of is to actually avoid issuing an EndTxn so that instead we rely 
> on Kafka internally to time things out after a period of time.
> —
> For some context, here's my request logs from the client. Note that I write 
> two ProduceRequests, read one, and then issue EndTxn (because I know I want 
> to quit). The second ProduceRequest is read successfully before 

[jira] [Commented] (KAFKA-13464) SCRAM does not validate client-final-message's nonce

2021-11-18 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-13464?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17446311#comment-17446311
 ] 

Travis Bischel commented on KAFKA-13464:


AFAICT this compromises no integrity. I'm not sure the purpose of this final 
check. The important part is the client-proof, which hashes the `c-nonce 
s-nonce` into it. Sending the c-nonce s-nonce back in plaintext doesn't seem to 
provide much benefit.

> SCRAM does not validate client-final-message's nonce
> 
>
> Key: KAFKA-13464
> URL: https://issues.apache.org/jira/browse/KAFKA-13464
> Project: Kafka
>  Issue Type: Bug
>Reporter: Travis Bischel
>Assignee: Luke Chen
>Priority: Minor
>
> [https://datatracker.ietf.org/doc/html/rfc5802#section-5.1]
> Relevant part, in "r="
>   nonce it initially specified.  The server MUST verify that the
>   nonce sent by the client in the second message is the same as the
>   one sent by the server in its first message.
> [https://github.com/apache/kafka/blob/8a1fcee86e42c8bd1f26309dde8748927109056e/clients/src/main/java/org/apache/kafka/common/security/scram/internals/ScramSaslServer.java#L149-L161]
> The only verification of client-final-message is verifyClientProof:
> [https://github.com/apache/kafka/blob/8a1fcee86e42c8bd1f26309dde8748927109056e/clients/src/main/java/org/apache/kafka/common/security/scram/internals/ScramSaslServer.java#L225-L235]
> This function only looks at the key itself. It does not ensure that the 
> gs2-header is "biws" (base64("n,,")), meaning the user can erroneously 
> specify channel binding. This also does not check that the client's nonce is 
> correct (c-nonce + s-nonce).
>  
> While I'm not 100% sure on what security concerns an invalid nonce could 
> result in _at this stage_ of the auth flow (it's clearer in the first stage 
> w.r.t. replay attacks), it's likely still important to validate.
>  
> I noticed this validation is missing because my own client erroneously 
> replies with only the original c-nonce, not c-nonce s-nonce. The scram flow 
> has always worked, though. Today I changed the client-final-reply to always 
> return nonce "foo", which still successfully talks to Kafka.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Created] (KAFKA-13464) SCRAM does not validate client-final-message's nonce

2021-11-18 Thread Travis Bischel (Jira)
Travis Bischel created KAFKA-13464:
--

 Summary: SCRAM does not validate client-final-message's nonce
 Key: KAFKA-13464
 URL: https://issues.apache.org/jira/browse/KAFKA-13464
 Project: Kafka
  Issue Type: Bug
Reporter: Travis Bischel


[https://datatracker.ietf.org/doc/html/rfc5802#section-5.1]

Relevant part, in "r="
  nonce it initially specified.  The server MUST verify that the
  nonce sent by the client in the second message is the same as the
  one sent by the server in its first message.
[https://github.com/apache/kafka/blob/8a1fcee86e42c8bd1f26309dde8748927109056e/clients/src/main/java/org/apache/kafka/common/security/scram/internals/ScramSaslServer.java#L149-L161]

The only verification of client-final-message is verifyClientProof:

[https://github.com/apache/kafka/blob/8a1fcee86e42c8bd1f26309dde8748927109056e/clients/src/main/java/org/apache/kafka/common/security/scram/internals/ScramSaslServer.java#L225-L235]

This function only looks at the key itself. It does not ensure that the 
gs2-header is "biws" (base64("n,,")), meaning the user can erroneously specify 
channel binding. This also does not check that the client's nonce is correct 
(c-nonce + s-nonce).

 

While I'm not 100% sure on what security concerns an invalid nonce could result 
in _at this stage_ of the auth flow (it's clearer in the first stage w.r.t. 
replay attacks), it's likely still important to validate.

 

I noticed this validation is missing because my own client erroneously replies 
with only the original c-nonce, not c-nonce s-nonce. The scram flow has always 
worked, though. Today I changed the client-final-reply to always return nonce 
"foo", which still successfully talks to Kafka.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (KAFKA-12671) Out of order processing with a transactional producer can lead to a stuck LastStableOffset

2021-05-19 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12671?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17347983#comment-17347983
 ] 

Travis Bischel commented on KAFKA-12671:


For anybody stuck on this later, I added a command to my 
[kcl|https://github.com/twmb/kcl/blob/master/CHANGELOG.md#v030] binary that may 
help: {{kcl admin txn unhang-lso }} can un-stick a LastStableOffset.

> Out of order processing with a transactional producer can lead to a stuck 
> LastStableOffset
> --
>
> Key: KAFKA-12671
> URL: https://issues.apache.org/jira/browse/KAFKA-12671
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.4.0, 2.5.0, 2.6.0, 2.7.0, 2.8.0
>Reporter: Travis Bischel
>Priority: Major
>  Labels: Transactions
>
> If there is pathological processing of incoming produce requests and EndTxn 
> requests, then the LastStableOffset can get stuck, which will block consuming 
> in READ_COMMITTED mode.
> To transactionally produce, the standard flow is to InitProducerId, and then 
> loop AddPartitionsToTxn -> Produce+ -> EndTxn. The AddPartitionsToTxn is 
> responsible for fencing and adding partitions to a transaction, and the end 
> transaction is responsible for finishing the transaction. Producing itself is 
> mostly uninvolved with the proper fencing / ending flow, but produce requests 
> are required to be after AddPartitionsToTxn and before EndTxn.
> When a ProduceRequest is handled, Kafka uses an internal ProducerStateManager 
> to mildly manage transactions. The ProducerStateManager is completely 
> independent of the TxnCoordinator, and its guarantees are relatively weak. 
> The ProducerStateManager handles two types of "batches" being added: a data 
> batch and a transaction marker. When a data batch is added, a "transaction" 
> is begun and tied to the producer ID that is producing the batch. When a 
> transaction marker is handled, the ProducerStateManager removes the 
> transaction for the producer ID (roughly).
> EndTxn is what triggers transaction markers to be sent to the 
> ProducerStateManager. In essence, EndTxn is the one part of the transactional 
> producer flow that talks across both the TxnCoordinator and the 
> ProducerStateManager.
> If a ProduceRequest is issued before EndTxn, but handled internally in Kafka 
> after EndTxn, then the ProduceRequest will begin a new transaction in the 
> ProducerStateManager. If the client was disconnecting, and the EndTxn was the 
> final request issued, the new transaction created in ProducerStateManager is 
> orphaned and nothing can clean it up. The LastStableOffset then hangs based 
> off of this hung transaction.
> This same problem can be triggered by a produce request that is issued with a 
> transactional ID outside of the context of a transaction at all (no 
> AddPartitionsToTxn). This problem cannot be triggered by producing for so 
> long that the transaction expires; the difference here is that the 
> transaction coordinator bumps the epoch for the producer ID, thus producing 
> again with the old epoch does not work.
> Theoretically, we are supposed have unlimited retries on produce requests, 
> but in the context of wanting to abort everything and shut down, this is not 
> always feasible. As it currently stands, I'm not sure there's a truly safe 
> way to shut down _without_ flushing and receiving responses for every record 
> produced, even if I want to abort everything and quit. The safest approach I 
> can think of is to actually avoid issuing an EndTxn so that instead we rely 
> on Kafka internally to time things out after a period of time.
> —
> For some context, here's my request logs from the client. Note that I write 
> two ProduceRequests, read one, and then issue EndTxn (because I know I want 
> to quit). The second ProduceRequest is read successfully before shutdown, but 
> I ignore the results because I am shutting down. I've taken out logs related 
> to consuming, but the order of the logs is unchanged:
> {noformat}
> [INFO] done waiting for unknown topic, metadata was successful; topic: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765
> [INFO] initializing producer id
> [DEBUG] wrote FindCoordinator v3; err: 
> [DEBUG] read FindCoordinator v3; err: 
> [DEBUG] wrote InitProducerID v4; err: 
> [DEBUG] read InitProducerID v4; err: 
> [INFO] producer id initialization success; id: 1463, epoch: 0
> [DEBUG] wrote AddPartitionsToTxn v2; err: 
> [DEBUG] read AddPartitionsToTxn v2; err: 
> [DEBUG] wrote Produce v8; err: 
> [DEBUG] read Produce v8; err: 
> [DEBUG] produced; to: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765[1{15589=>19686}]
> [DEBUG] wrote Produce v8; err: 
> [DEBUG] wrote EndTxn v2; err: 
> [DEBUG] read EndTxn v2; err: 
> [DEBUG] read 

[jira] [Commented] (KAFKA-12671) Out of order processing with a transactional producer can lead to a stuck LastStableOffset

2021-05-12 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12671?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17343401#comment-17343401
 ] 

Travis Bischel commented on KAFKA-12671:


Hi [~Peyer],

I believe that the Java producer has taken the same approach that I ultimately 
ended up on, which is to disallow an EndTxn request if there are any unresolved 
records. By forcing all records to be in a known-to-the-client stable state, 
and ensuring that no Produce request is in flight, this issue is largely 
avoided.

However, that does not mean the issue does not exist, and a faulty client or 
potentially some unforeseen bugs, or worse a malicious client, can easily 
trigger this: if you manually issue an InitProducerID request and then a 
ProduceRequest with the producer id, this issue happens. Partitions will then 
have a stuck LastStableOffset.

Again, this is because a Produce request begins a transaction in the 
ProduceStateManager, yet no transaction was begun in the TxnCoordinator, and 
nothing will ever end the transaction that was begun in the ProduceStateManager.

> Out of order processing with a transactional producer can lead to a stuck 
> LastStableOffset
> --
>
> Key: KAFKA-12671
> URL: https://issues.apache.org/jira/browse/KAFKA-12671
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.4.0, 2.5.0, 2.6.0, 2.7.0, 2.8.0
>Reporter: Travis Bischel
>Priority: Major
>  Labels: Transactions
>
> If there is pathological processing of incoming produce requests and EndTxn 
> requests, then the LastStableOffset can get stuck, which will block consuming 
> in READ_COMMITTED mode.
> To transactionally produce, the standard flow is to InitProducerId, and then 
> loop AddPartitionsToTxn -> Produce+ -> EndTxn. The AddPartitionsToTxn is 
> responsible for fencing and adding partitions to a transaction, and the end 
> transaction is responsible for finishing the transaction. Producing itself is 
> mostly uninvolved with the proper fencing / ending flow, but produce requests 
> are required to be after AddPartitionsToTxn and before EndTxn.
> When a ProduceRequest is handled, Kafka uses an internal ProducerStateManager 
> to mildly manage transactions. The ProducerStateManager is completely 
> independent of the TxnCoordinator, and its guarantees are relatively weak. 
> The ProducerStateManager handles two types of "batches" being added: a data 
> batch and a transaction marker. When a data batch is added, a "transaction" 
> is begun and tied to the producer ID that is producing the batch. When a 
> transaction marker is handled, the ProducerStateManager removes the 
> transaction for the producer ID (roughly).
> EndTxn is what triggers transaction markers to be sent to the 
> ProducerStateManager. In essence, EndTxn is the one part of the transactional 
> producer flow that talks across both the TxnCoordinator and the 
> ProducerStateManager.
> If a ProduceRequest is issued before EndTxn, but handled internally in Kafka 
> after EndTxn, then the ProduceRequest will begin a new transaction in the 
> ProducerStateManager. If the client was disconnecting, and the EndTxn was the 
> final request issued, the new transaction created in ProducerStateManager is 
> orphaned and nothing can clean it up. The LastStableOffset then hangs based 
> off of this hung transaction.
> This same problem can be triggered by a produce request that is issued with a 
> transactional ID outside of the context of a transaction at all (no 
> AddPartitionsToTxn). This problem cannot be triggered by producing for so 
> long that the transaction expires; the difference here is that the 
> transaction coordinator bumps the epoch for the producer ID, thus producing 
> again with the old epoch does not work.
> Theoretically, we are supposed have unlimited retries on produce requests, 
> but in the context of wanting to abort everything and shut down, this is not 
> always feasible. As it currently stands, I'm not sure there's a truly safe 
> way to shut down _without_ flushing and receiving responses for every record 
> produced, even if I want to abort everything and quit. The safest approach I 
> can think of is to actually avoid issuing an EndTxn so that instead we rely 
> on Kafka internally to time things out after a period of time.
> —
> For some context, here's my request logs from the client. Note that I write 
> two ProduceRequests, read one, and then issue EndTxn (because I know I want 
> to quit). The second ProduceRequest is read successfully before shutdown, but 
> I ignore the results because I am shutting down. I've taken out logs related 
> to consuming, but the order of the logs is unchanged:
> {noformat}
> [INFO] done waiting for unknown topic, metadata was successful; topic: 
> 

[jira] [Comment Edited] (KAFKA-12671) Out of order processing with a transactional producer can lead to a stuck LastStableOffset

2021-05-12 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12671?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17340576#comment-17340576
 ] 

Travis Bischel edited comment on KAFKA-12671 at 5/12/21, 5:25 PM:
--

After much investigating, a potential fix is to issue an AddPartitionsToTxn 
request with the producer ID, epoch, transactional ID that caused partitions to 
get stuck, as well as _all_ topics/partitions that are stuck, and then follow 
that with an EndTxn request (w/ pid, producer epoch, txn id).

Finding the pid, producer epoch, and transactional ID is the hard part.

My process to find those three was as follows:

- kcl consume --read-uncommitted -p  -o  
-num 3 -f '%x %y %|\n'
- kcl consume __transaction_state | grep 'ProducerID.*\b\b' -A 7 -B 4



was (Author: twmb):
After much investigating, a potential fix is to issue an AddPartitionsToTxn 
request with the producer ID, epoch, transactional ID that caused partitions to 
get stuck, as well as _all_ topics/partitions that are stuck, and then follow 
that with an EndTxn request (w/ pid, producer epoch, txn id).

Finding the pid, producer epoch, and transactional ID is the hard part.

My process to find those three was as follows:

- kcl consume --read-uncommitted -p  -o  
-num 3 -f '%x %y %|\n'
- kcl consume __transaction_state | grep 'ProducerID.*\b\b -A 7 -B 4


> Out of order processing with a transactional producer can lead to a stuck 
> LastStableOffset
> --
>
> Key: KAFKA-12671
> URL: https://issues.apache.org/jira/browse/KAFKA-12671
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.4.0, 2.5.0, 2.6.0, 2.7.0, 2.8.0
>Reporter: Travis Bischel
>Priority: Major
>  Labels: Transactions
>
> If there is pathological processing of incoming produce requests and EndTxn 
> requests, then the LastStableOffset can get stuck, which will block consuming 
> in READ_COMMITTED mode.
> To transactionally produce, the standard flow is to InitProducerId, and then 
> loop AddPartitionsToTxn -> Produce+ -> EndTxn. The AddPartitionsToTxn is 
> responsible for fencing and adding partitions to a transaction, and the end 
> transaction is responsible for finishing the transaction. Producing itself is 
> mostly uninvolved with the proper fencing / ending flow, but produce requests 
> are required to be after AddPartitionsToTxn and before EndTxn.
> When a ProduceRequest is handled, Kafka uses an internal ProducerStateManager 
> to mildly manage transactions. The ProducerStateManager is completely 
> independent of the TxnCoordinator, and its guarantees are relatively weak. 
> The ProducerStateManager handles two types of "batches" being added: a data 
> batch and a transaction marker. When a data batch is added, a "transaction" 
> is begun and tied to the producer ID that is producing the batch. When a 
> transaction marker is handled, the ProducerStateManager removes the 
> transaction for the producer ID (roughly).
> EndTxn is what triggers transaction markers to be sent to the 
> ProducerStateManager. In essence, EndTxn is the one part of the transactional 
> producer flow that talks across both the TxnCoordinator and the 
> ProducerStateManager.
> If a ProduceRequest is issued before EndTxn, but handled internally in Kafka 
> after EndTxn, then the ProduceRequest will begin a new transaction in the 
> ProducerStateManager. If the client was disconnecting, and the EndTxn was the 
> final request issued, the new transaction created in ProducerStateManager is 
> orphaned and nothing can clean it up. The LastStableOffset then hangs based 
> off of this hung transaction.
> This same problem can be triggered by a produce request that is issued with a 
> transactional ID outside of the context of a transaction at all (no 
> AddPartitionsToTxn). This problem cannot be triggered by producing for so 
> long that the transaction expires; the difference here is that the 
> transaction coordinator bumps the epoch for the producer ID, thus producing 
> again with the old epoch does not work.
> Theoretically, we are supposed have unlimited retries on produce requests, 
> but in the context of wanting to abort everything and shut down, this is not 
> always feasible. As it currently stands, I'm not sure there's a truly safe 
> way to shut down _without_ flushing and receiving responses for every record 
> produced, even if I want to abort everything and quit. The safest approach I 
> can think of is to actually avoid issuing an EndTxn so that instead we rely 
> on Kafka internally to time things out after a period of time.
> —
> For some context, here's my request logs from the client. Note that I write 
> two ProduceRequests, read one, and then issue EndTxn (because I know I want 
> to quit). The second 

[jira] [Updated] (KAFKA-12671) Out of order processing with a transactional producer can lead to a stuck LastStableOffset

2021-05-12 Thread Travis Bischel (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-12671?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Travis Bischel updated KAFKA-12671:
---
Priority: Major  (was: Blocker)

> Out of order processing with a transactional producer can lead to a stuck 
> LastStableOffset
> --
>
> Key: KAFKA-12671
> URL: https://issues.apache.org/jira/browse/KAFKA-12671
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.4.0, 2.5.0, 2.6.0, 2.7.0, 2.8.0
>Reporter: Travis Bischel
>Priority: Major
>  Labels: Transactions
>
> If there is pathological processing of incoming produce requests and EndTxn 
> requests, then the LastStableOffset can get stuck, which will block consuming 
> in READ_COMMITTED mode.
> To transactionally produce, the standard flow is to InitProducerId, and then 
> loop AddPartitionsToTxn -> Produce+ -> EndTxn. The AddPartitionsToTxn is 
> responsible for fencing and adding partitions to a transaction, and the end 
> transaction is responsible for finishing the transaction. Producing itself is 
> mostly uninvolved with the proper fencing / ending flow, but produce requests 
> are required to be after AddPartitionsToTxn and before EndTxn.
> When a ProduceRequest is handled, Kafka uses an internal ProducerStateManager 
> to mildly manage transactions. The ProducerStateManager is completely 
> independent of the TxnCoordinator, and its guarantees are relatively weak. 
> The ProducerStateManager handles two types of "batches" being added: a data 
> batch and a transaction marker. When a data batch is added, a "transaction" 
> is begun and tied to the producer ID that is producing the batch. When a 
> transaction marker is handled, the ProducerStateManager removes the 
> transaction for the producer ID (roughly).
> EndTxn is what triggers transaction markers to be sent to the 
> ProducerStateManager. In essence, EndTxn is the one part of the transactional 
> producer flow that talks across both the TxnCoordinator and the 
> ProducerStateManager.
> If a ProduceRequest is issued before EndTxn, but handled internally in Kafka 
> after EndTxn, then the ProduceRequest will begin a new transaction in the 
> ProducerStateManager. If the client was disconnecting, and the EndTxn was the 
> final request issued, the new transaction created in ProducerStateManager is 
> orphaned and nothing can clean it up. The LastStableOffset then hangs based 
> off of this hung transaction.
> This same problem can be triggered by a produce request that is issued with a 
> transactional ID outside of the context of a transaction at all (no 
> AddPartitionsToTxn). This problem cannot be triggered by producing for so 
> long that the transaction expires; the difference here is that the 
> transaction coordinator bumps the epoch for the producer ID, thus producing 
> again with the old epoch does not work.
> Theoretically, we are supposed have unlimited retries on produce requests, 
> but in the context of wanting to abort everything and shut down, this is not 
> always feasible. As it currently stands, I'm not sure there's a truly safe 
> way to shut down _without_ flushing and receiving responses for every record 
> produced, even if I want to abort everything and quit. The safest approach I 
> can think of is to actually avoid issuing an EndTxn so that instead we rely 
> on Kafka internally to time things out after a period of time.
> —
> For some context, here's my request logs from the client. Note that I write 
> two ProduceRequests, read one, and then issue EndTxn (because I know I want 
> to quit). The second ProduceRequest is read successfully before shutdown, but 
> I ignore the results because I am shutting down. I've taken out logs related 
> to consuming, but the order of the logs is unchanged:
> {noformat}
> [INFO] done waiting for unknown topic, metadata was successful; topic: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765
> [INFO] initializing producer id
> [DEBUG] wrote FindCoordinator v3; err: 
> [DEBUG] read FindCoordinator v3; err: 
> [DEBUG] wrote InitProducerID v4; err: 
> [DEBUG] read InitProducerID v4; err: 
> [INFO] producer id initialization success; id: 1463, epoch: 0
> [DEBUG] wrote AddPartitionsToTxn v2; err: 
> [DEBUG] read AddPartitionsToTxn v2; err: 
> [DEBUG] wrote Produce v8; err: 
> [DEBUG] read Produce v8; err: 
> [DEBUG] produced; to: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765[1{15589=>19686}]
> [DEBUG] wrote Produce v8; err: 
> [DEBUG] wrote EndTxn v2; err: 
> [DEBUG] read EndTxn v2; err: 
> [DEBUG] read from broker errored, killing connection; addr: localhost:9092, 
> id: 1, successful_reads: 1, err: context canceled
> [DEBUG] read Produce v8; err: 
> [DEBUG] produced; to: 
> 

[jira] [Comment Edited] (KAFKA-12671) Out of order processing with a transactional producer can lead to a stuck LastStableOffset

2021-05-06 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12671?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17340576#comment-17340576
 ] 

Travis Bischel edited comment on KAFKA-12671 at 5/7/21, 5:37 AM:
-

After much investigating, a potential fix is to issue an AddPartitionsToTxn 
request with the producer ID, epoch, transactional ID that caused partitions to 
get stuck, as well as _all_ topics/partitions that are stuck, and then follow 
that with an EndTxn request (w/ pid, producer epoch, txn id).

Finding the pid, producer epoch, and transactional ID is the hard part.

My process to find those three was as follows:

- kcl consume --read-uncommitted -p  -o  
-num 3 -f '%x %y %|\n'
- kcl consume __transaction_state | grep 'ProducerID.*\b\b -A 7 -B 4



was (Author: twmb):
After much investigating, a potential fix is to issue an AddPartitionsToTxn 
request with the producer ID, epoch, transactional ID that caused partitions to 
get stuck, as well as _all_ topics/partitions that are stuck, and then follow 
that with an EndTxn request (w/ pid, producer epoch, txn id).

Finding the pid, producer epoch, and transactional ID is the hard part.

> Out of order processing with a transactional producer can lead to a stuck 
> LastStableOffset
> --
>
> Key: KAFKA-12671
> URL: https://issues.apache.org/jira/browse/KAFKA-12671
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.4.0, 2.5.0, 2.6.0, 2.7.0, 2.8.0
>Reporter: Travis Bischel
>Priority: Blocker
>  Labels: Transactions
>
> If there is pathological processing of incoming produce requests and EndTxn 
> requests, then the LastStableOffset can get stuck, which will block consuming 
> in READ_COMMITTED mode.
> To transactionally produce, the standard flow is to InitProducerId, and then 
> loop AddPartitionsToTxn -> Produce+ -> EndTxn. The AddPartitionsToTxn is 
> responsible for fencing and adding partitions to a transaction, and the end 
> transaction is responsible for finishing the transaction. Producing itself is 
> mostly uninvolved with the proper fencing / ending flow, but produce requests 
> are required to be after AddPartitionsToTxn and before EndTxn.
> When a ProduceRequest is handled, Kafka uses an internal ProducerStateManager 
> to mildly manage transactions. The ProducerStateManager is completely 
> independent of the TxnCoordinator, and its guarantees are relatively weak. 
> The ProducerStateManager handles two types of "batches" being added: a data 
> batch and a transaction marker. When a data batch is added, a "transaction" 
> is begun and tied to the producer ID that is producing the batch. When a 
> transaction marker is handled, the ProducerStateManager removes the 
> transaction for the producer ID (roughly).
> EndTxn is what triggers transaction markers to be sent to the 
> ProducerStateManager. In essence, EndTxn is the one part of the transactional 
> producer flow that talks across both the TxnCoordinator and the 
> ProducerStateManager.
> If a ProduceRequest is issued before EndTxn, but handled internally in Kafka 
> after EndTxn, then the ProduceRequest will begin a new transaction in the 
> ProducerStateManager. If the client was disconnecting, and the EndTxn was the 
> final request issued, the new transaction created in ProducerStateManager is 
> orphaned and nothing can clean it up. The LastStableOffset then hangs based 
> off of this hung transaction.
> This same problem can be triggered by a produce request that is issued with a 
> transactional ID outside of the context of a transaction at all (no 
> AddPartitionsToTxn). This problem cannot be triggered by producing for so 
> long that the transaction expires; the difference here is that the 
> transaction coordinator bumps the epoch for the producer ID, thus producing 
> again with the old epoch does not work.
> Theoretically, we are supposed have unlimited retries on produce requests, 
> but in the context of wanting to abort everything and shut down, this is not 
> always feasible. As it currently stands, I'm not sure there's a truly safe 
> way to shut down _without_ flushing and receiving responses for every record 
> produced, even if I want to abort everything and quit. The safest approach I 
> can think of is to actually avoid issuing an EndTxn so that instead we rely 
> on Kafka internally to time things out after a period of time.
> —
> For some context, here's my request logs from the client. Note that I write 
> two ProduceRequests, read one, and then issue EndTxn (because I know I want 
> to quit). The second ProduceRequest is read successfully before shutdown, but 
> I ignore the results because I am shutting down. I've taken out logs related 
> to consuming, but the order of the logs is unchanged:
> 

[jira] [Commented] (KAFKA-12671) Out of order processing with a transactional producer can lead to a stuck LastStableOffset

2021-05-06 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12671?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17340576#comment-17340576
 ] 

Travis Bischel commented on KAFKA-12671:


After much investigating, a potential fix is to issue an AddPartitionsToTxn 
request with the producer ID, epoch, transactional ID that caused partitions to 
get stuck, as well as _all_ topics/partitions that are stuck, and then follow 
that with an EndTxn request (w/ pid, producer epoch, txn id).

Finding the pid, producer epoch, and transactional ID is the hard part.

> Out of order processing with a transactional producer can lead to a stuck 
> LastStableOffset
> --
>
> Key: KAFKA-12671
> URL: https://issues.apache.org/jira/browse/KAFKA-12671
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.4.0, 2.5.0, 2.6.0, 2.7.0, 2.8.0
>Reporter: Travis Bischel
>Priority: Blocker
>  Labels: Transactions
>
> If there is pathological processing of incoming produce requests and EndTxn 
> requests, then the LastStableOffset can get stuck, which will block consuming 
> in READ_COMMITTED mode.
> To transactionally produce, the standard flow is to InitProducerId, and then 
> loop AddPartitionsToTxn -> Produce+ -> EndTxn. The AddPartitionsToTxn is 
> responsible for fencing and adding partitions to a transaction, and the end 
> transaction is responsible for finishing the transaction. Producing itself is 
> mostly uninvolved with the proper fencing / ending flow, but produce requests 
> are required to be after AddPartitionsToTxn and before EndTxn.
> When a ProduceRequest is handled, Kafka uses an internal ProducerStateManager 
> to mildly manage transactions. The ProducerStateManager is completely 
> independent of the TxnCoordinator, and its guarantees are relatively weak. 
> The ProducerStateManager handles two types of "batches" being added: a data 
> batch and a transaction marker. When a data batch is added, a "transaction" 
> is begun and tied to the producer ID that is producing the batch. When a 
> transaction marker is handled, the ProducerStateManager removes the 
> transaction for the producer ID (roughly).
> EndTxn is what triggers transaction markers to be sent to the 
> ProducerStateManager. In essence, EndTxn is the one part of the transactional 
> producer flow that talks across both the TxnCoordinator and the 
> ProducerStateManager.
> If a ProduceRequest is issued before EndTxn, but handled internally in Kafka 
> after EndTxn, then the ProduceRequest will begin a new transaction in the 
> ProducerStateManager. If the client was disconnecting, and the EndTxn was the 
> final request issued, the new transaction created in ProducerStateManager is 
> orphaned and nothing can clean it up. The LastStableOffset then hangs based 
> off of this hung transaction.
> This same problem can be triggered by a produce request that is issued with a 
> transactional ID outside of the context of a transaction at all (no 
> AddPartitionsToTxn). This problem cannot be triggered by producing for so 
> long that the transaction expires; the difference here is that the 
> transaction coordinator bumps the epoch for the producer ID, thus producing 
> again with the old epoch does not work.
> Theoretically, we are supposed have unlimited retries on produce requests, 
> but in the context of wanting to abort everything and shut down, this is not 
> always feasible. As it currently stands, I'm not sure there's a truly safe 
> way to shut down _without_ flushing and receiving responses for every record 
> produced, even if I want to abort everything and quit. The safest approach I 
> can think of is to actually avoid issuing an EndTxn so that instead we rely 
> on Kafka internally to time things out after a period of time.
> —
> For some context, here's my request logs from the client. Note that I write 
> two ProduceRequests, read one, and then issue EndTxn (because I know I want 
> to quit). The second ProduceRequest is read successfully before shutdown, but 
> I ignore the results because I am shutting down. I've taken out logs related 
> to consuming, but the order of the logs is unchanged:
> {noformat}
> [INFO] done waiting for unknown topic, metadata was successful; topic: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765
> [INFO] initializing producer id
> [DEBUG] wrote FindCoordinator v3; err: 
> [DEBUG] read FindCoordinator v3; err: 
> [DEBUG] wrote InitProducerID v4; err: 
> [DEBUG] read InitProducerID v4; err: 
> [INFO] producer id initialization success; id: 1463, epoch: 0
> [DEBUG] wrote AddPartitionsToTxn v2; err: 
> [DEBUG] read AddPartitionsToTxn v2; err: 
> [DEBUG] wrote Produce v8; err: 
> [DEBUG] read Produce v8; err: 
> [DEBUG] produced; to: 
> 

[jira] [Comment Edited] (KAFKA-12671) Out of order processing with a transactional producer can lead to a stuck LastStableOffset

2021-05-06 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12671?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17339995#comment-17339995
 ] 

Travis Bischel edited comment on KAFKA-12671 at 5/6/21, 9:01 PM:
-

There is no fix for this other than deleting the topic.

This logic race condition is present in all versions from 0.11 onward.


was (Author: twmb):
The only fix for this if it occurs is to turn off and turn back on the entire 
cluster.

This logic race condition is present in all versions from 0.11 onward.

> Out of order processing with a transactional producer can lead to a stuck 
> LastStableOffset
> --
>
> Key: KAFKA-12671
> URL: https://issues.apache.org/jira/browse/KAFKA-12671
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.4.0, 2.5.0, 2.6.0, 2.7.0, 2.8.0
>Reporter: Travis Bischel
>Priority: Blocker
>  Labels: Transactions
>
> If there is pathological processing of incoming produce requests and EndTxn 
> requests, then the LastStableOffset can get stuck, which will block consuming 
> in READ_COMMITTED mode.
> To transactionally produce, the standard flow is to InitProducerId, and then 
> loop AddPartitionsToTxn -> Produce+ -> EndTxn. The AddPartitionsToTxn is 
> responsible for fencing and adding partitions to a transaction, and the end 
> transaction is responsible for finishing the transaction. Producing itself is 
> mostly uninvolved with the proper fencing / ending flow, but produce requests 
> are required to be after AddPartitionsToTxn and before EndTxn.
> When a ProduceRequest is handled, Kafka uses an internal ProducerStateManager 
> to mildly manage transactions. The ProducerStateManager is completely 
> independent of the TxnCoordinator, and its guarantees are relatively weak. 
> The ProducerStateManager handles two types of "batches" being added: a data 
> batch and a transaction marker. When a data batch is added, a "transaction" 
> is begun and tied to the producer ID that is producing the batch. When a 
> transaction marker is handled, the ProducerStateManager removes the 
> transaction for the producer ID (roughly).
> EndTxn is what triggers transaction markers to be sent to the 
> ProducerStateManager. In essence, EndTxn is the one part of the transactional 
> producer flow that talks across both the TxnCoordinator and the 
> ProducerStateManager.
> If a ProduceRequest is issued before EndTxn, but handled internally in Kafka 
> after EndTxn, then the ProduceRequest will begin a new transaction in the 
> ProducerStateManager. If the client was disconnecting, and the EndTxn was the 
> final request issued, the new transaction created in ProducerStateManager is 
> orphaned and nothing can clean it up. The LastStableOffset then hangs based 
> off of this hung transaction.
> This same problem can be triggered by a produce request that is issued with a 
> transactional ID outside of the context of a transaction at all (no 
> AddPartitionsToTxn). This problem cannot be triggered by producing for so 
> long that the transaction expires; the difference here is that the 
> transaction coordinator bumps the epoch for the producer ID, thus producing 
> again with the old epoch does not work.
> Theoretically, we are supposed have unlimited retries on produce requests, 
> but in the context of wanting to abort everything and shut down, this is not 
> always feasible. As it currently stands, I'm not sure there's a truly safe 
> way to shut down _without_ flushing and receiving responses for every record 
> produced, even if I want to abort everything and quit. The safest approach I 
> can think of is to actually avoid issuing an EndTxn so that instead we rely 
> on Kafka internally to time things out after a period of time.
> —
> For some context, here's my request logs from the client. Note that I write 
> two ProduceRequests, read one, and then issue EndTxn (because I know I want 
> to quit). The second ProduceRequest is read successfully before shutdown, but 
> I ignore the results because I am shutting down. I've taken out logs related 
> to consuming, but the order of the logs is unchanged:
> {noformat}
> [INFO] done waiting for unknown topic, metadata was successful; topic: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765
> [INFO] initializing producer id
> [DEBUG] wrote FindCoordinator v3; err: 
> [DEBUG] read FindCoordinator v3; err: 
> [DEBUG] wrote InitProducerID v4; err: 
> [DEBUG] read InitProducerID v4; err: 
> [INFO] producer id initialization success; id: 1463, epoch: 0
> [DEBUG] wrote AddPartitionsToTxn v2; err: 
> [DEBUG] read AddPartitionsToTxn v2; err: 
> [DEBUG] wrote Produce v8; err: 
> [DEBUG] read Produce v8; err: 
> [DEBUG] produced; to: 
> 

[jira] [Comment Edited] (KAFKA-12671) Out of order processing with a transactional producer can lead to a stuck LastStableOffset

2021-05-06 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12671?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17339995#comment-17339995
 ] 

Travis Bischel edited comment on KAFKA-12671 at 5/6/21, 6:45 AM:
-

The only fix for this if it occurs is to turn off and turn back on the entire 
cluster.

This logic race condition is present in all versions from 0.11 onward.


was (Author: twmb):
The only fix for this if it occurs is to restart a broker.

This logic race condition is present in all versions from 0.11 onward.

> Out of order processing with a transactional producer can lead to a stuck 
> LastStableOffset
> --
>
> Key: KAFKA-12671
> URL: https://issues.apache.org/jira/browse/KAFKA-12671
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.4.0, 2.5.0, 2.6.0, 2.7.0, 2.8.0
>Reporter: Travis Bischel
>Priority: Blocker
>  Labels: Transactions
>
> If there is pathological processing of incoming produce requests and EndTxn 
> requests, then the LastStableOffset can get stuck, which will block consuming 
> in READ_COMMITTED mode.
> To transactionally produce, the standard flow is to InitProducerId, and then 
> loop AddPartitionsToTxn -> Produce+ -> EndTxn. The AddPartitionsToTxn is 
> responsible for fencing and adding partitions to a transaction, and the end 
> transaction is responsible for finishing the transaction. Producing itself is 
> mostly uninvolved with the proper fencing / ending flow, but produce requests 
> are required to be after AddPartitionsToTxn and before EndTxn.
> When a ProduceRequest is handled, Kafka uses an internal ProducerStateManager 
> to mildly manage transactions. The ProducerStateManager is completely 
> independent of the TxnCoordinator, and its guarantees are relatively weak. 
> The ProducerStateManager handles two types of "batches" being added: a data 
> batch and a transaction marker. When a data batch is added, a "transaction" 
> is begun and tied to the producer ID that is producing the batch. When a 
> transaction marker is handled, the ProducerStateManager removes the 
> transaction for the producer ID (roughly).
> EndTxn is what triggers transaction markers to be sent to the 
> ProducerStateManager. In essence, EndTxn is the one part of the transactional 
> producer flow that talks across both the TxnCoordinator and the 
> ProducerStateManager.
> If a ProduceRequest is issued before EndTxn, but handled internally in Kafka 
> after EndTxn, then the ProduceRequest will begin a new transaction in the 
> ProducerStateManager. If the client was disconnecting, and the EndTxn was the 
> final request issued, the new transaction created in ProducerStateManager is 
> orphaned and nothing can clean it up. The LastStableOffset then hangs based 
> off of this hung transaction.
> This same problem can be triggered by a produce request that is issued with a 
> transactional ID outside of the context of a transaction at all (no 
> AddPartitionsToTxn). This problem cannot be triggered by producing for so 
> long that the transaction expires; the difference here is that the 
> transaction coordinator bumps the epoch for the producer ID, thus producing 
> again with the old epoch does not work.
> Theoretically, we are supposed have unlimited retries on produce requests, 
> but in the context of wanting to abort everything and shut down, this is not 
> always feasible. As it currently stands, I'm not sure there's a truly safe 
> way to shut down _without_ flushing and receiving responses for every record 
> produced, even if I want to abort everything and quit. The safest approach I 
> can think of is to actually avoid issuing an EndTxn so that instead we rely 
> on Kafka internally to time things out after a period of time.
> —
> For some context, here's my request logs from the client. Note that I write 
> two ProduceRequests, read one, and then issue EndTxn (because I know I want 
> to quit). The second ProduceRequest is read successfully before shutdown, but 
> I ignore the results because I am shutting down. I've taken out logs related 
> to consuming, but the order of the logs is unchanged:
> {noformat}
> [INFO] done waiting for unknown topic, metadata was successful; topic: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765
> [INFO] initializing producer id
> [DEBUG] wrote FindCoordinator v3; err: 
> [DEBUG] read FindCoordinator v3; err: 
> [DEBUG] wrote InitProducerID v4; err: 
> [DEBUG] read InitProducerID v4; err: 
> [INFO] producer id initialization success; id: 1463, epoch: 0
> [DEBUG] wrote AddPartitionsToTxn v2; err: 
> [DEBUG] read AddPartitionsToTxn v2; err: 
> [DEBUG] wrote Produce v8; err: 
> [DEBUG] read Produce v8; err: 
> [DEBUG] produced; to: 
> 

[jira] [Commented] (KAFKA-10442) Tooling to detect and abort hanging transactions (KIP-664)

2021-05-06 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-10442?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17340029#comment-17340029
 ] 

Travis Bischel commented on KAFKA-10442:


I believe that KAFKA-12671 is related to this issue. Specifically, 
WriteTxnMarker should be possible to close out my issue, but I describe another 
aspect that maybe is worth investigating for the fix: the ProducerStateManager. 
What do you think?

> Tooling to detect and abort hanging transactions (KIP-664)
> --
>
> Key: KAFKA-10442
> URL: https://issues.apache.org/jira/browse/KAFKA-10442
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Jason Gustafson
>Assignee: Jason Gustafson
>Priority: Major
>
> This JIRA tracks the implementation of KIP-664: 
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-664%3A+Provide+tooling+to+detect+and+abort+hanging+transactions.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Updated] (KAFKA-12671) Out of order processing with a transactional producer can lead to a stuck LastStableOffset

2021-05-05 Thread Travis Bischel (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-12671?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Travis Bischel updated KAFKA-12671:
---
Priority: Blocker  (was: Critical)

> Out of order processing with a transactional producer can lead to a stuck 
> LastStableOffset
> --
>
> Key: KAFKA-12671
> URL: https://issues.apache.org/jira/browse/KAFKA-12671
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.4.0, 2.5.0, 2.6.0, 2.7.0, 2.8.0
>Reporter: Travis Bischel
>Priority: Blocker
>  Labels: Transactions
>
> If there is pathological processing of incoming produce requests and EndTxn 
> requests, then the LastStableOffset can get stuck, which will block consuming 
> in READ_COMMITTED mode.
> To transactionally produce, the standard flow is to InitProducerId, and then 
> loop AddPartitionsToTxn -> Produce+ -> EndTxn. The AddPartitionsToTxn is 
> responsible for fencing and adding partitions to a transaction, and the end 
> transaction is responsible for finishing the transaction. Producing itself is 
> mostly uninvolved with the proper fencing / ending flow, but produce requests 
> are required to be after AddPartitionsToTxn and before EndTxn.
> When a ProduceRequest is handled, Kafka uses an internal ProducerStateManager 
> to mildly manage transactions. The ProducerStateManager is completely 
> independent of the TxnCoordinator, and its guarantees are relatively weak. 
> The ProducerStateManager handles two types of "batches" being added: a data 
> batch and a transaction marker. When a data batch is added, a "transaction" 
> is begun and tied to the producer ID that is producing the batch. When a 
> transaction marker is handled, the ProducerStateManager removes the 
> transaction for the producer ID (roughly).
> EndTxn is what triggers transaction markers to be sent to the 
> ProducerStateManager. In essence, EndTxn is the one part of the transactional 
> producer flow that talks across both the TxnCoordinator and the 
> ProducerStateManager.
> If a ProduceRequest is issued before EndTxn, but handled internally in Kafka 
> after EndTxn, then the ProduceRequest will begin a new transaction in the 
> ProducerStateManager. If the client was disconnecting, and the EndTxn was the 
> final request issued, the new transaction created in ProducerStateManager is 
> orphaned and nothing can clean it up. The LastStableOffset then hangs based 
> off of this hung transaction.
> This same problem can be triggered by a produce request that is issued with a 
> transactional ID outside of the context of a transaction at all (no 
> AddPartitionsToTxn). This problem cannot be triggered by producing for so 
> long that the transaction expires; the difference here is that the 
> transaction coordinator bumps the epoch for the producer ID, thus producing 
> again with the old epoch does not work.
> Theoretically, we are supposed have unlimited retries on produce requests, 
> but in the context of wanting to abort everything and shut down, this is not 
> always feasible. As it currently stands, I'm not sure there's a truly safe 
> way to shut down _without_ flushing and receiving responses for every record 
> produced, even if I want to abort everything and quit. The safest approach I 
> can think of is to actually avoid issuing an EndTxn so that instead we rely 
> on Kafka internally to time things out after a period of time.
> —
> For some context, here's my request logs from the client. Note that I write 
> two ProduceRequests, read one, and then issue EndTxn (because I know I want 
> to quit). The second ProduceRequest is read successfully before shutdown, but 
> I ignore the results because I am shutting down. I've taken out logs related 
> to consuming, but the order of the logs is unchanged:
> {noformat}
> [INFO] done waiting for unknown topic, metadata was successful; topic: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765
> [INFO] initializing producer id
> [DEBUG] wrote FindCoordinator v3; err: 
> [DEBUG] read FindCoordinator v3; err: 
> [DEBUG] wrote InitProducerID v4; err: 
> [DEBUG] read InitProducerID v4; err: 
> [INFO] producer id initialization success; id: 1463, epoch: 0
> [DEBUG] wrote AddPartitionsToTxn v2; err: 
> [DEBUG] read AddPartitionsToTxn v2; err: 
> [DEBUG] wrote Produce v8; err: 
> [DEBUG] read Produce v8; err: 
> [DEBUG] produced; to: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765[1{15589=>19686}]
> [DEBUG] wrote Produce v8; err: 
> [DEBUG] wrote EndTxn v2; err: 
> [DEBUG] read EndTxn v2; err: 
> [DEBUG] read from broker errored, killing connection; addr: localhost:9092, 
> id: 1, successful_reads: 1, err: context canceled
> [DEBUG] read Produce v8; err: 
> [DEBUG] produced; to: 
> 

[jira] [Updated] (KAFKA-12671) Out of order processing with a transactional producer can lead to a stuck LastStableOffset

2021-05-05 Thread Travis Bischel (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-12671?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Travis Bischel updated KAFKA-12671:
---
Labels: Transactions  (was: )

> Out of order processing with a transactional producer can lead to a stuck 
> LastStableOffset
> --
>
> Key: KAFKA-12671
> URL: https://issues.apache.org/jira/browse/KAFKA-12671
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.4.0, 2.5.0, 2.6.0, 2.7.0, 2.8.0
>Reporter: Travis Bischel
>Priority: Critical
>  Labels: Transactions
>
> If there is pathological processing of incoming produce requests and EndTxn 
> requests, then the LastStableOffset can get stuck, which will block consuming 
> in READ_COMMITTED mode.
> To transactionally produce, the standard flow is to InitProducerId, and then 
> loop AddPartitionsToTxn -> Produce+ -> EndTxn. The AddPartitionsToTxn is 
> responsible for fencing and adding partitions to a transaction, and the end 
> transaction is responsible for finishing the transaction. Producing itself is 
> mostly uninvolved with the proper fencing / ending flow, but produce requests 
> are required to be after AddPartitionsToTxn and before EndTxn.
> When a ProduceRequest is handled, Kafka uses an internal ProducerStateManager 
> to mildly manage transactions. The ProducerStateManager is completely 
> independent of the TxnCoordinator, and its guarantees are relatively weak. 
> The ProducerStateManager handles two types of "batches" being added: a data 
> batch and a transaction marker. When a data batch is added, a "transaction" 
> is begun and tied to the producer ID that is producing the batch. When a 
> transaction marker is handled, the ProducerStateManager removes the 
> transaction for the producer ID (roughly).
> EndTxn is what triggers transaction markers to be sent to the 
> ProducerStateManager. In essence, EndTxn is the one part of the transactional 
> producer flow that talks across both the TxnCoordinator and the 
> ProducerStateManager.
> If a ProduceRequest is issued before EndTxn, but handled internally in Kafka 
> after EndTxn, then the ProduceRequest will begin a new transaction in the 
> ProducerStateManager. If the client was disconnecting, and the EndTxn was the 
> final request issued, the new transaction created in ProducerStateManager is 
> orphaned and nothing can clean it up. The LastStableOffset then hangs based 
> off of this hung transaction.
> This same problem can be triggered by a produce request that is issued with a 
> transactional ID outside of the context of a transaction at all (no 
> AddPartitionsToTxn). This problem cannot be triggered by producing for so 
> long that the transaction expires; the difference here is that the 
> transaction coordinator bumps the epoch for the producer ID, thus producing 
> again with the old epoch does not work.
> Theoretically, we are supposed have unlimited retries on produce requests, 
> but in the context of wanting to abort everything and shut down, this is not 
> always feasible. As it currently stands, I'm not sure there's a truly safe 
> way to shut down _without_ flushing and receiving responses for every record 
> produced, even if I want to abort everything and quit. The safest approach I 
> can think of is to actually avoid issuing an EndTxn so that instead we rely 
> on Kafka internally to time things out after a period of time.
> —
> For some context, here's my request logs from the client. Note that I write 
> two ProduceRequests, read one, and then issue EndTxn (because I know I want 
> to quit). The second ProduceRequest is read successfully before shutdown, but 
> I ignore the results because I am shutting down. I've taken out logs related 
> to consuming, but the order of the logs is unchanged:
> {noformat}
> [INFO] done waiting for unknown topic, metadata was successful; topic: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765
> [INFO] initializing producer id
> [DEBUG] wrote FindCoordinator v3; err: 
> [DEBUG] read FindCoordinator v3; err: 
> [DEBUG] wrote InitProducerID v4; err: 
> [DEBUG] read InitProducerID v4; err: 
> [INFO] producer id initialization success; id: 1463, epoch: 0
> [DEBUG] wrote AddPartitionsToTxn v2; err: 
> [DEBUG] read AddPartitionsToTxn v2; err: 
> [DEBUG] wrote Produce v8; err: 
> [DEBUG] read Produce v8; err: 
> [DEBUG] produced; to: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765[1{15589=>19686}]
> [DEBUG] wrote Produce v8; err: 
> [DEBUG] wrote EndTxn v2; err: 
> [DEBUG] read EndTxn v2; err: 
> [DEBUG] read from broker errored, killing connection; addr: localhost:9092, 
> id: 1, successful_reads: 1, err: context canceled
> [DEBUG] read Produce v8; err: 
> [DEBUG] produced; to: 
> 

[jira] [Commented] (KAFKA-12671) Out of order processing with a transactional producer can lead to a stuck LastStableOffset

2021-05-05 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12671?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17339995#comment-17339995
 ] 

Travis Bischel commented on KAFKA-12671:


The only fix for this if it occurs is to restart a broker.

This logic race condition is present in all versions from 0.11 onward.

> Out of order processing with a transactional producer can lead to a stuck 
> LastStableOffset
> --
>
> Key: KAFKA-12671
> URL: https://issues.apache.org/jira/browse/KAFKA-12671
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.4.0, 2.5.0, 2.6.0, 2.7.0, 2.8.0
>Reporter: Travis Bischel
>Priority: Critical
>
> If there is pathological processing of incoming produce requests and EndTxn 
> requests, then the LastStableOffset can get stuck, which will block consuming 
> in READ_COMMITTED mode.
> To transactionally produce, the standard flow is to InitProducerId, and then 
> loop AddPartitionsToTxn -> Produce+ -> EndTxn. The AddPartitionsToTxn is 
> responsible for fencing and adding partitions to a transaction, and the end 
> transaction is responsible for finishing the transaction. Producing itself is 
> mostly uninvolved with the proper fencing / ending flow, but produce requests 
> are required to be after AddPartitionsToTxn and before EndTxn.
> When a ProduceRequest is handled, Kafka uses an internal ProducerStateManager 
> to mildly manage transactions. The ProducerStateManager is completely 
> independent of the TxnCoordinator, and its guarantees are relatively weak. 
> The ProducerStateManager handles two types of "batches" being added: a data 
> batch and a transaction marker. When a data batch is added, a "transaction" 
> is begun and tied to the producer ID that is producing the batch. When a 
> transaction marker is handled, the ProducerStateManager removes the 
> transaction for the producer ID (roughly).
> EndTxn is what triggers transaction markers to be sent to the 
> ProducerStateManager. In essence, EndTxn is the one part of the transactional 
> producer flow that talks across both the TxnCoordinator and the 
> ProducerStateManager.
> If a ProduceRequest is issued before EndTxn, but handled internally in Kafka 
> after EndTxn, then the ProduceRequest will begin a new transaction in the 
> ProducerStateManager. If the client was disconnecting, and the EndTxn was the 
> final request issued, the new transaction created in ProducerStateManager is 
> orphaned and nothing can clean it up. The LastStableOffset then hangs based 
> off of this hung transaction.
> This same problem can be triggered by a produce request that is issued with a 
> transactional ID outside of the context of a transaction at all (no 
> AddPartitionsToTxn). This problem cannot be triggered by producing for so 
> long that the transaction expires; the difference here is that the 
> transaction coordinator bumps the epoch for the producer ID, thus producing 
> again with the old epoch does not work.
> Theoretically, we are supposed have unlimited retries on produce requests, 
> but in the context of wanting to abort everything and shut down, this is not 
> always feasible. As it currently stands, I'm not sure there's a truly safe 
> way to shut down _without_ flushing and receiving responses for every record 
> produced, even if I want to abort everything and quit. The safest approach I 
> can think of is to actually avoid issuing an EndTxn so that instead we rely 
> on Kafka internally to time things out after a period of time.
> —
> For some context, here's my request logs from the client. Note that I write 
> two ProduceRequests, read one, and then issue EndTxn (because I know I want 
> to quit). The second ProduceRequest is read successfully before shutdown, but 
> I ignore the results because I am shutting down. I've taken out logs related 
> to consuming, but the order of the logs is unchanged:
> {noformat}
> [INFO] done waiting for unknown topic, metadata was successful; topic: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765
> [INFO] initializing producer id
> [DEBUG] wrote FindCoordinator v3; err: 
> [DEBUG] read FindCoordinator v3; err: 
> [DEBUG] wrote InitProducerID v4; err: 
> [DEBUG] read InitProducerID v4; err: 
> [INFO] producer id initialization success; id: 1463, epoch: 0
> [DEBUG] wrote AddPartitionsToTxn v2; err: 
> [DEBUG] read AddPartitionsToTxn v2; err: 
> [DEBUG] wrote Produce v8; err: 
> [DEBUG] read Produce v8; err: 
> [DEBUG] produced; to: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765[1{15589=>19686}]
> [DEBUG] wrote Produce v8; err: 
> [DEBUG] wrote EndTxn v2; err: 
> [DEBUG] read EndTxn v2; err: 
> [DEBUG] read from broker errored, killing connection; addr: localhost:9092, 
> id: 1, successful_reads: 1, err: context 

[jira] [Updated] (KAFKA-12671) Out of order processing with a transactional producer can lead to a stuck LastStableOffset

2021-05-05 Thread Travis Bischel (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-12671?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Travis Bischel updated KAFKA-12671:
---
Affects Version/s: (was: 2.6.2)
   (was: 2.6.1)
   (was: 2.5.1)
   (was: 2.3.1)
   (was: 2.2.2)

> Out of order processing with a transactional producer can lead to a stuck 
> LastStableOffset
> --
>
> Key: KAFKA-12671
> URL: https://issues.apache.org/jira/browse/KAFKA-12671
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.4.0, 2.5.0, 2.6.0, 2.7.0, 2.8.0
>Reporter: Travis Bischel
>Priority: Critical
>
> If there is pathological processing of incoming produce requests and EndTxn 
> requests, then the LastStableOffset can get stuck, which will block consuming 
> in READ_COMMITTED mode.
> To transactionally produce, the standard flow is to InitProducerId, and then 
> loop AddPartitionsToTxn -> Produce+ -> EndTxn. The AddPartitionsToTxn is 
> responsible for fencing and adding partitions to a transaction, and the end 
> transaction is responsible for finishing the transaction. Producing itself is 
> mostly uninvolved with the proper fencing / ending flow, but produce requests 
> are required to be after AddPartitionsToTxn and before EndTxn.
> When a ProduceRequest is handled, Kafka uses an internal ProducerStateManager 
> to mildly manage transactions. The ProducerStateManager is completely 
> independent of the TxnCoordinator, and its guarantees are relatively weak. 
> The ProducerStateManager handles two types of "batches" being added: a data 
> batch and a transaction marker. When a data batch is added, a "transaction" 
> is begun and tied to the producer ID that is producing the batch. When a 
> transaction marker is handled, the ProducerStateManager removes the 
> transaction for the producer ID (roughly).
> EndTxn is what triggers transaction markers to be sent to the 
> ProducerStateManager. In essence, EndTxn is the one part of the transactional 
> producer flow that talks across both the TxnCoordinator and the 
> ProducerStateManager.
> If a ProduceRequest is issued before EndTxn, but handled internally in Kafka 
> after EndTxn, then the ProduceRequest will begin a new transaction in the 
> ProducerStateManager. If the client was disconnecting, and the EndTxn was the 
> final request issued, the new transaction created in ProducerStateManager is 
> orphaned and nothing can clean it up. The LastStableOffset then hangs based 
> off of this hung transaction.
> This same problem can be triggered by a produce request that is issued with a 
> transactional ID outside of the context of a transaction at all (no 
> AddPartitionsToTxn). This problem cannot be triggered by producing for so 
> long that the transaction expires; the difference here is that the 
> transaction coordinator bumps the epoch for the producer ID, thus producing 
> again with the old epoch does not work.
> Theoretically, we are supposed have unlimited retries on produce requests, 
> but in the context of wanting to abort everything and shut down, this is not 
> always feasible. As it currently stands, I'm not sure there's a truly safe 
> way to shut down _without_ flushing and receiving responses for every record 
> produced, even if I want to abort everything and quit. The safest approach I 
> can think of is to actually avoid issuing an EndTxn so that instead we rely 
> on Kafka internally to time things out after a period of time.
> —
> For some context, here's my request logs from the client. Note that I write 
> two ProduceRequests, read one, and then issue EndTxn (because I know I want 
> to quit). The second ProduceRequest is read successfully before shutdown, but 
> I ignore the results because I am shutting down. I've taken out logs related 
> to consuming, but the order of the logs is unchanged:
> {noformat}
> [INFO] done waiting for unknown topic, metadata was successful; topic: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765
> [INFO] initializing producer id
> [DEBUG] wrote FindCoordinator v3; err: 
> [DEBUG] read FindCoordinator v3; err: 
> [DEBUG] wrote InitProducerID v4; err: 
> [DEBUG] read InitProducerID v4; err: 
> [INFO] producer id initialization success; id: 1463, epoch: 0
> [DEBUG] wrote AddPartitionsToTxn v2; err: 
> [DEBUG] read AddPartitionsToTxn v2; err: 
> [DEBUG] wrote Produce v8; err: 
> [DEBUG] read Produce v8; err: 
> [DEBUG] produced; to: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765[1{15589=>19686}]
> [DEBUG] wrote Produce v8; err: 
> [DEBUG] wrote EndTxn v2; err: 
> [DEBUG] read EndTxn v2; err: 
> [DEBUG] read from broker errored, killing connection; addr: localhost:9092, 
> id: 1, 

[jira] [Updated] (KAFKA-12671) Out of order processing with a transactional producer can lead to a stuck LastStableOffset

2021-05-05 Thread Travis Bischel (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-12671?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Travis Bischel updated KAFKA-12671:
---
Affects Version/s: 2.2.2
   2.4.0
   2.3.1
   2.5.0
   2.6.0
   2.5.1
   2.7.0
   2.6.1
   2.8.0
   2.6.2

> Out of order processing with a transactional producer can lead to a stuck 
> LastStableOffset
> --
>
> Key: KAFKA-12671
> URL: https://issues.apache.org/jira/browse/KAFKA-12671
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.2.2, 2.4.0, 2.3.1, 2.5.0, 2.6.0, 2.5.1, 2.7.0, 2.6.1, 
> 2.8.0, 2.6.2
>Reporter: Travis Bischel
>Priority: Critical
>
> If there is pathological processing of incoming produce requests and EndTxn 
> requests, then the LastStableOffset can get stuck, which will block consuming 
> in READ_COMMITTED mode.
> To transactionally produce, the standard flow is to InitProducerId, and then 
> loop AddPartitionsToTxn -> Produce+ -> EndTxn. The AddPartitionsToTxn is 
> responsible for fencing and adding partitions to a transaction, and the end 
> transaction is responsible for finishing the transaction. Producing itself is 
> mostly uninvolved with the proper fencing / ending flow, but produce requests 
> are required to be after AddPartitionsToTxn and before EndTxn.
> When a ProduceRequest is handled, Kafka uses an internal ProducerStateManager 
> to mildly manage transactions. The ProducerStateManager is completely 
> independent of the TxnCoordinator, and its guarantees are relatively weak. 
> The ProducerStateManager handles two types of "batches" being added: a data 
> batch and a transaction marker. When a data batch is added, a "transaction" 
> is begun and tied to the producer ID that is producing the batch. When a 
> transaction marker is handled, the ProducerStateManager removes the 
> transaction for the producer ID (roughly).
> EndTxn is what triggers transaction markers to be sent to the 
> ProducerStateManager. In essence, EndTxn is the one part of the transactional 
> producer flow that talks across both the TxnCoordinator and the 
> ProducerStateManager.
> If a ProduceRequest is issued before EndTxn, but handled internally in Kafka 
> after EndTxn, then the ProduceRequest will begin a new transaction in the 
> ProducerStateManager. If the client was disconnecting, and the EndTxn was the 
> final request issued, the new transaction created in ProducerStateManager is 
> orphaned and nothing can clean it up. The LastStableOffset then hangs based 
> off of this hung transaction.
> This same problem can be triggered by a produce request that is issued with a 
> transactional ID outside of the context of a transaction at all (no 
> AddPartitionsToTxn). This problem cannot be triggered by producing for so 
> long that the transaction expires; the difference here is that the 
> transaction coordinator bumps the epoch for the producer ID, thus producing 
> again with the old epoch does not work.
> Theoretically, we are supposed have unlimited retries on produce requests, 
> but in the context of wanting to abort everything and shut down, this is not 
> always feasible. As it currently stands, I'm not sure there's a truly safe 
> way to shut down _without_ flushing and receiving responses for every record 
> produced, even if I want to abort everything and quit. The safest approach I 
> can think of is to actually avoid issuing an EndTxn so that instead we rely 
> on Kafka internally to time things out after a period of time.
> —
> For some context, here's my request logs from the client. Note that I write 
> two ProduceRequests, read one, and then issue EndTxn (because I know I want 
> to quit). The second ProduceRequest is read successfully before shutdown, but 
> I ignore the results because I am shutting down. I've taken out logs related 
> to consuming, but the order of the logs is unchanged:
> {noformat}
> [INFO] done waiting for unknown topic, metadata was successful; topic: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765
> [INFO] initializing producer id
> [DEBUG] wrote FindCoordinator v3; err: 
> [DEBUG] read FindCoordinator v3; err: 
> [DEBUG] wrote InitProducerID v4; err: 
> [DEBUG] read InitProducerID v4; err: 
> [INFO] producer id initialization success; id: 1463, epoch: 0
> [DEBUG] wrote AddPartitionsToTxn v2; err: 
> [DEBUG] read AddPartitionsToTxn v2; err: 
> [DEBUG] wrote Produce v8; err: 
> [DEBUG] read Produce v8; err: 
> [DEBUG] produced; to: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765[1{15589=>19686}]
> [DEBUG] wrote Produce v8; err: 
> [DEBUG] wrote EndTxn v2; err: 

[jira] [Updated] (KAFKA-12701) NPE in MetadataRequest when using topic IDs

2021-04-22 Thread Travis Bischel (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-12701?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Travis Bischel updated KAFKA-12701:
---
Description: 
Authorized result checking relies on topic name to not be null, which, when 
using topic IDs, it is.

Unlike the logic in handleDeleteTopicsRequest, handleMetadataRequest does not 
check zk for the names corresponding to topic IDs if topic IDs are present.

{noformat}
[2021-04-21 05:53:01,463] ERROR [KafkaApi-1] Error when handling request: 
clientId=kgo, correlationId=1, api=METADATA, version=11, 
body=MetadataRequestData(topics=[MetadataRequestTopic(topicId=LmqOoFOASnqQp_4-oJgeKA,
 name=null)], allowAutoTopicCreation=false, 
includeClusterAuthorizedOperations=false, 
includeTopicAuthorizedOperations=false) (kafka.server.RequestHandlerHelper)
java.lang.NullPointerException: name
at java.base/java.util.Objects.requireNonNull(Unknown Source)
at 
org.apache.kafka.common.resource.ResourcePattern.(ResourcePattern.java:50)
at 
kafka.server.AuthHelper.$anonfun$filterByAuthorized$3(AuthHelper.scala:121)
at scala.collection.Iterator$$anon$9.next(Iterator.scala:575)
at scala.collection.mutable.Growable.addAll(Growable.scala:62)
at scala.collection.mutable.Growable.addAll$(Growable.scala:57)
at scala.collection.mutable.ArrayBuffer.addAll(ArrayBuffer.scala:142)
at scala.collection.mutable.ArrayBuffer.addAll(ArrayBuffer.scala:42)
at scala.collection.mutable.ArrayBuffer$.from(ArrayBuffer.scala:258)
at scala.collection.mutable.ArrayBuffer$.from(ArrayBuffer.scala:247)
at scala.collection.SeqFactory$Delegate.from(Factory.scala:306)
at scala.collection.IterableOnceOps.toBuffer(IterableOnce.scala:1270)
at scala.collection.IterableOnceOps.toBuffer$(IterableOnce.scala:1270)
at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1288)
at kafka.server.AuthHelper.filterByAuthorized(AuthHelper.scala:120)
at 
kafka.server.KafkaApis.handleTopicMetadataRequest(KafkaApis.scala:1146)
at kafka.server.KafkaApis.handle(KafkaApis.scala:170)
at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:74)
at java.base/java.lang.Thread.run(Unknown Source)
[2021-04-21 05:53:01,464] ERROR [Kafka Request Handler 1 on Broker 1], 
Exception when handling request (kafka.server.KafkaRequestHandler)
java.lang.NullPointerException
at 
org.apache.kafka.common.message.MetadataResponseData$MetadataResponseTopic.addSize(MetadataResponseData.java:1247)
at 
org.apache.kafka.common.message.MetadataResponseData.addSize(MetadataResponseData.java:417)
at 
org.apache.kafka.common.protocol.SendBuilder.buildSend(SendBuilder.java:218)
at 
org.apache.kafka.common.protocol.SendBuilder.buildResponseSend(SendBuilder.java:200)
at 
org.apache.kafka.common.requests.AbstractResponse.toSend(AbstractResponse.java:43)
at 
org.apache.kafka.common.requests.RequestContext.buildResponseSend(RequestContext.java:111)
at 
kafka.network.RequestChannel$Request.buildResponseSend(RequestChannel.scala:132)
at 
kafka.server.RequestHandlerHelper.sendResponse(RequestHandlerHelper.scala:185)
at 
kafka.server.RequestHandlerHelper.sendErrorOrCloseConnection(RequestHandlerHelper.scala:155)
at 
kafka.server.RequestHandlerHelper.sendErrorResponseMaybeThrottle(RequestHandlerHelper.scala:109)
at 
kafka.server.RequestHandlerHelper.handleError(RequestHandlerHelper.scala:79)
at kafka.server.KafkaApis.handle(KafkaApis.scala:229)
at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:74)
at java.base/java.lang.Thread.run(Unknown Source)
{noformat}

  was:
Authorized result checking relies on topic name to not be null, which, when 
using topic IDs, it is.

Unlike the logic in handleDeleteTopicsRequest, hanelMetadataRequest does not 
check zk for the names corresponding to topic IDs if topic IDs are present.

{noformat}
[2021-04-21 05:53:01,463] ERROR [KafkaApi-1] Error when handling request: 
clientId=kgo, correlationId=1, api=METADATA, version=11, 
body=MetadataRequestData(topics=[MetadataRequestTopic(topicId=LmqOoFOASnqQp_4-oJgeKA,
 name=null)], allowAutoTopicCreation=false, 
includeClusterAuthorizedOperations=false, 
includeTopicAuthorizedOperations=false) (kafka.server.RequestHandlerHelper)
java.lang.NullPointerException: name
at java.base/java.util.Objects.requireNonNull(Unknown Source)
at 
org.apache.kafka.common.resource.ResourcePattern.(ResourcePattern.java:50)
at 
kafka.server.AuthHelper.$anonfun$filterByAuthorized$3(AuthHelper.scala:121)
at scala.collection.Iterator$$anon$9.next(Iterator.scala:575)
at scala.collection.mutable.Growable.addAll(Growable.scala:62)
at scala.collection.mutable.Growable.addAll$(Growable.scala:57)
at 

[jira] [Commented] (KAFKA-12701) NPE in MetadataRequest when using topic IDs

2021-04-21 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12701?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17326316#comment-17326316
 ] 

Travis Bischel commented on KAFKA-12701:


Metadata v11 is supported in 2.8.0. I've added support for specifying topic IDs 
in my {{kcl}} command line client 
[here|https://github.com/twmb/kcl/commit/a09f6f8cca4f87b878d943a2bed4ef8ed2e10a7e]
 (you'd need to build off of master if you want to test this yourself).

{noformat}
[01:20:51]
twmb@h4x3r:~
$ kcl admin topic create foo
NAME  IDMESSAGE
foo   6a62c01129a341c8a0231f7b3c21bc9b  OK

[01:20:57]
twmb@h4x3r:~
$ kcl metadata -t --ids 6a62c01129a341c8a0231f7b3c21bc9b
^C

[01:21:11]
twmb@h4x3r:~
$ kcl metadata -t foo
NAME  IDPARTITIONS  REPLICAS
foo   6a62c01129a341c8a0231f7b3c21bc9b  20  1

[01:21:16]
twmb@h4x3r:~
$ kcl admin topic delete --ids 6a62c01129a341c8a0231f7b3c21bc9b
foo   OK
{noformat}

What this is doing is issuing a metadata request that has a single topic where 
the topic name is null and the topic ID is the hex-decoded ID that is passed in.

> NPE in MetadataRequest when using topic IDs
> ---
>
> Key: KAFKA-12701
> URL: https://issues.apache.org/jira/browse/KAFKA-12701
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.8.0
>Reporter: Travis Bischel
>Assignee: dengziming
>Priority: Major
>
> Authorized result checking relies on topic name to not be null, which, when 
> using topic IDs, it is.
> Unlike the logic in handleDeleteTopicsRequest, hanelMetadataRequest does not 
> check zk for the names corresponding to topic IDs if topic IDs are present.
> {noformat}
> [2021-04-21 05:53:01,463] ERROR [KafkaApi-1] Error when handling request: 
> clientId=kgo, correlationId=1, api=METADATA, version=11, 
> body=MetadataRequestData(topics=[MetadataRequestTopic(topicId=LmqOoFOASnqQp_4-oJgeKA,
>  name=null)], allowAutoTopicCreation=false, 
> includeClusterAuthorizedOperations=false, 
> includeTopicAuthorizedOperations=false) (kafka.server.RequestHandlerHelper)
> java.lang.NullPointerException: name
>   at java.base/java.util.Objects.requireNonNull(Unknown Source)
>   at 
> org.apache.kafka.common.resource.ResourcePattern.(ResourcePattern.java:50)
>   at 
> kafka.server.AuthHelper.$anonfun$filterByAuthorized$3(AuthHelper.scala:121)
>   at scala.collection.Iterator$$anon$9.next(Iterator.scala:575)
>   at scala.collection.mutable.Growable.addAll(Growable.scala:62)
>   at scala.collection.mutable.Growable.addAll$(Growable.scala:57)
>   at scala.collection.mutable.ArrayBuffer.addAll(ArrayBuffer.scala:142)
>   at scala.collection.mutable.ArrayBuffer.addAll(ArrayBuffer.scala:42)
>   at scala.collection.mutable.ArrayBuffer$.from(ArrayBuffer.scala:258)
>   at scala.collection.mutable.ArrayBuffer$.from(ArrayBuffer.scala:247)
>   at scala.collection.SeqFactory$Delegate.from(Factory.scala:306)
>   at scala.collection.IterableOnceOps.toBuffer(IterableOnce.scala:1270)
>   at scala.collection.IterableOnceOps.toBuffer$(IterableOnce.scala:1270)
>   at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1288)
>   at kafka.server.AuthHelper.filterByAuthorized(AuthHelper.scala:120)
>   at 
> kafka.server.KafkaApis.handleTopicMetadataRequest(KafkaApis.scala:1146)
>   at kafka.server.KafkaApis.handle(KafkaApis.scala:170)
>   at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:74)
>   at java.base/java.lang.Thread.run(Unknown Source)
> [2021-04-21 05:53:01,464] ERROR [Kafka Request Handler 1 on Broker 1], 
> Exception when handling request (kafka.server.KafkaRequestHandler)
> java.lang.NullPointerException
>   at 
> org.apache.kafka.common.message.MetadataResponseData$MetadataResponseTopic.addSize(MetadataResponseData.java:1247)
>   at 
> org.apache.kafka.common.message.MetadataResponseData.addSize(MetadataResponseData.java:417)
>   at 
> org.apache.kafka.common.protocol.SendBuilder.buildSend(SendBuilder.java:218)
>   at 
> org.apache.kafka.common.protocol.SendBuilder.buildResponseSend(SendBuilder.java:200)
>   at 
> org.apache.kafka.common.requests.AbstractResponse.toSend(AbstractResponse.java:43)
>   at 
> org.apache.kafka.common.requests.RequestContext.buildResponseSend(RequestContext.java:111)
>   at 
> kafka.network.RequestChannel$Request.buildResponseSend(RequestChannel.scala:132)
>   at 
> kafka.server.RequestHandlerHelper.sendResponse(RequestHandlerHelper.scala:185)
>   at 
> kafka.server.RequestHandlerHelper.sendErrorOrCloseConnection(RequestHandlerHelper.scala:155)
>   at 
> kafka.server.RequestHandlerHelper.sendErrorResponseMaybeThrottle(RequestHandlerHelper.scala:109)
>   at 
> 

[jira] [Created] (KAFKA-12701) NPE in MetadataRequest when using topic IDs

2021-04-21 Thread Travis Bischel (Jira)
Travis Bischel created KAFKA-12701:
--

 Summary: NPE in MetadataRequest when using topic IDs
 Key: KAFKA-12701
 URL: https://issues.apache.org/jira/browse/KAFKA-12701
 Project: Kafka
  Issue Type: Bug
Affects Versions: 2.8.0
Reporter: Travis Bischel


Authorized result checking relies on topic name to not be null, which, when 
using topic IDs, it is.

Unlike the logic in handleDeleteTopicsRequest, hanelMetadataRequest does not 
check zk for the names corresponding to topic IDs if topic IDs are present.

{noformat}
[2021-04-21 05:53:01,463] ERROR [KafkaApi-1] Error when handling request: 
clientId=kgo, correlationId=1, api=METADATA, version=11, 
body=MetadataRequestData(topics=[MetadataRequestTopic(topicId=LmqOoFOASnqQp_4-oJgeKA,
 name=null)], allowAutoTopicCreation=false, 
includeClusterAuthorizedOperations=false, 
includeTopicAuthorizedOperations=false) (kafka.server.RequestHandlerHelper)
java.lang.NullPointerException: name
at java.base/java.util.Objects.requireNonNull(Unknown Source)
at 
org.apache.kafka.common.resource.ResourcePattern.(ResourcePattern.java:50)
at 
kafka.server.AuthHelper.$anonfun$filterByAuthorized$3(AuthHelper.scala:121)
at scala.collection.Iterator$$anon$9.next(Iterator.scala:575)
at scala.collection.mutable.Growable.addAll(Growable.scala:62)
at scala.collection.mutable.Growable.addAll$(Growable.scala:57)
at scala.collection.mutable.ArrayBuffer.addAll(ArrayBuffer.scala:142)
at scala.collection.mutable.ArrayBuffer.addAll(ArrayBuffer.scala:42)
at scala.collection.mutable.ArrayBuffer$.from(ArrayBuffer.scala:258)
at scala.collection.mutable.ArrayBuffer$.from(ArrayBuffer.scala:247)
at scala.collection.SeqFactory$Delegate.from(Factory.scala:306)
at scala.collection.IterableOnceOps.toBuffer(IterableOnce.scala:1270)
at scala.collection.IterableOnceOps.toBuffer$(IterableOnce.scala:1270)
at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1288)
at kafka.server.AuthHelper.filterByAuthorized(AuthHelper.scala:120)
at 
kafka.server.KafkaApis.handleTopicMetadataRequest(KafkaApis.scala:1146)
at kafka.server.KafkaApis.handle(KafkaApis.scala:170)
at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:74)
at java.base/java.lang.Thread.run(Unknown Source)
[2021-04-21 05:53:01,464] ERROR [Kafka Request Handler 1 on Broker 1], 
Exception when handling request (kafka.server.KafkaRequestHandler)
java.lang.NullPointerException
at 
org.apache.kafka.common.message.MetadataResponseData$MetadataResponseTopic.addSize(MetadataResponseData.java:1247)
at 
org.apache.kafka.common.message.MetadataResponseData.addSize(MetadataResponseData.java:417)
at 
org.apache.kafka.common.protocol.SendBuilder.buildSend(SendBuilder.java:218)
at 
org.apache.kafka.common.protocol.SendBuilder.buildResponseSend(SendBuilder.java:200)
at 
org.apache.kafka.common.requests.AbstractResponse.toSend(AbstractResponse.java:43)
at 
org.apache.kafka.common.requests.RequestContext.buildResponseSend(RequestContext.java:111)
at 
kafka.network.RequestChannel$Request.buildResponseSend(RequestChannel.scala:132)
at 
kafka.server.RequestHandlerHelper.sendResponse(RequestHandlerHelper.scala:185)
at 
kafka.server.RequestHandlerHelper.sendErrorOrCloseConnection(RequestHandlerHelper.scala:155)
at 
kafka.server.RequestHandlerHelper.sendErrorResponseMaybeThrottle(RequestHandlerHelper.scala:109)
at 
kafka.server.RequestHandlerHelper.handleError(RequestHandlerHelper.scala:79)
at kafka.server.KafkaApis.handle(KafkaApis.scala:229)
at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:74)
at java.base/java.lang.Thread.run(Unknown Source)
{noformat}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Updated] (KAFKA-12671) Out of order processing with a transactional producer can lead to a stuck LastStableOffset

2021-04-20 Thread Travis Bischel (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-12671?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Travis Bischel updated KAFKA-12671:
---
Priority: Critical  (was: Major)

> Out of order processing with a transactional producer can lead to a stuck 
> LastStableOffset
> --
>
> Key: KAFKA-12671
> URL: https://issues.apache.org/jira/browse/KAFKA-12671
> Project: Kafka
>  Issue Type: Bug
>Reporter: Travis Bischel
>Priority: Critical
>
> If there is pathological processing of incoming produce requests and EndTxn 
> requests, then the LastStableOffset can get stuck, which will block consuming 
> in READ_COMMITTED mode.
> To transactionally produce, the standard flow is to InitProducerId, and then 
> loop AddPartitionsToTxn -> Produce+ -> EndTxn. The AddPartitionsToTxn is 
> responsible for fencing and adding partitions to a transaction, and the end 
> transaction is responsible for finishing the transaction. Producing itself is 
> mostly uninvolved with the proper fencing / ending flow, but produce requests 
> are required to be after AddPartitionsToTxn and before EndTxn.
> When a ProduceRequest is handled, Kafka uses an internal ProducerStateManager 
> to mildly manage transactions. The ProducerStateManager is completely 
> independent of the TxnCoordinator, and its guarantees are relatively weak. 
> The ProducerStateManager handles two types of "batches" being added: a data 
> batch and a transaction marker. When a data batch is added, a "transaction" 
> is begun and tied to the producer ID that is producing the batch. When a 
> transaction marker is handled, the ProducerStateManager removes the 
> transaction for the producer ID (roughly).
> EndTxn is what triggers transaction markers to be sent to the 
> ProducerStateManager. In essence, EndTxn is the one part of the transactional 
> producer flow that talks across both the TxnCoordinator and the 
> ProducerStateManager.
> If a ProduceRequest is issued before EndTxn, but handled internally in Kafka 
> after EndTxn, then the ProduceRequest will begin a new transaction in the 
> ProducerStateManager. If the client was disconnecting, and the EndTxn was the 
> final request issued, the new transaction created in ProducerStateManager is 
> orphaned and nothing can clean it up. The LastStableOffset then hangs based 
> off of this hung transaction.
> This same problem can be triggered by a produce request that is issued with a 
> transactional ID outside of the context of a transaction at all (no 
> AddPartitionsToTxn). This problem cannot be triggered by producing for so 
> long that the transaction expires; the difference here is that the 
> transaction coordinator bumps the epoch for the producer ID, thus producing 
> again with the old epoch does not work.
> Theoretically, we are supposed have unlimited retries on produce requests, 
> but in the context of wanting to abort everything and shut down, this is not 
> always feasible. As it currently stands, I'm not sure there's a truly safe 
> way to shut down _without_ flushing and receiving responses for every record 
> produced, even if I want to abort everything and quit. The safest approach I 
> can think of is to actually avoid issuing an EndTxn so that instead we rely 
> on Kafka internally to time things out after a period of time.
> —
> For some context, here's my request logs from the client. Note that I write 
> two ProduceRequests, read one, and then issue EndTxn (because I know I want 
> to quit). The second ProduceRequest is read successfully before shutdown, but 
> I ignore the results because I am shutting down. I've taken out logs related 
> to consuming, but the order of the logs is unchanged:
> {noformat}
> [INFO] done waiting for unknown topic, metadata was successful; topic: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765
> [INFO] initializing producer id
> [DEBUG] wrote FindCoordinator v3; err: 
> [DEBUG] read FindCoordinator v3; err: 
> [DEBUG] wrote InitProducerID v4; err: 
> [DEBUG] read InitProducerID v4; err: 
> [INFO] producer id initialization success; id: 1463, epoch: 0
> [DEBUG] wrote AddPartitionsToTxn v2; err: 
> [DEBUG] read AddPartitionsToTxn v2; err: 
> [DEBUG] wrote Produce v8; err: 
> [DEBUG] read Produce v8; err: 
> [DEBUG] produced; to: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765[1{15589=>19686}]
> [DEBUG] wrote Produce v8; err: 
> [DEBUG] wrote EndTxn v2; err: 
> [DEBUG] read EndTxn v2; err: 
> [DEBUG] read from broker errored, killing connection; addr: localhost:9092, 
> id: 1, successful_reads: 1, err: context canceled
> [DEBUG] read Produce v8; err: 
> [DEBUG] produced; to: 
> 2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765[1{skipped}]
> {noformat}
> And then from the broker's point 

[jira] [Comment Edited] (KAFKA-12675) Improve sticky general assignor scalability and performance

2021-04-19 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12675?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17324358#comment-17324358
 ] 

Travis Bischel edited comment on KAFKA-12675 at 4/19/21, 11:58 PM:
---

Great insight on getting rid of partition2AllPotentialConsumers, as well as 
keeping some more things sorted! I was able to translate that into my own code 
and dropped the large imbalance from 9.5s to 0.5s, as well as from 8.5G memory 
util to 0.5G :)

I'll take a look at the code more in depth soon.

Edit: after further improvements I was able to get the large imbalance in my 
client down to 220ms and 150MB.


was (Author: twmb):
Great insight on getting rid of partition2AllPotentialConsumers, as well as 
keeping some more things sorted! I was able to translate that into my own code 
and dropped the large imbalance from 9.5s to 0.5s, as well as from 8.5G memory 
util to 0.5G :)

I'll take a look at the code more in depth soon.

Edit: after further improvements I was able to get the large imbalance down to 
220ms and 171MB,

> Improve sticky general assignor scalability and performance
> ---
>
> Key: KAFKA-12675
> URL: https://issues.apache.org/jira/browse/KAFKA-12675
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Luke Chen
>Assignee: Luke Chen
>Priority: Major
>
> Currently, we have "general assignor" for non-equal subscription case and 
> "constrained assignor" for all equal subscription case. There's a performance 
> test for constrained assignor with:
> topicCount = {color:#ff}500{color};
>  partitionCount = {color:#ff}2000{color}; 
>  consumerCount = {color:#ff}2000{color};
> in _testLargeAssignmentAndGroupWithUniformSubscription,_ total 1 million 
> partitions and we can complete the assignment within 2 second in my machine.
> However, if we let 1 of the consumer subscribe to only 1 topic, it'll use 
> "general assignor", and the result with the same setting as above is: 
> *OutOfMemory,* 
>  Even we down the count to:
> topicCount = {color:#ff}50{color};
>  partitionCount = 1{color:#ff}000{color}; 
>  consumerCount = 1{color:#ff}000{color};
> We still got *OutOfMemory*.
> With this setting:
> topicCount = {color:#ff}50{color};
>  partitionCount = 8{color:#ff}00{color}; 
>  consumerCount = 8{color:#ff}00{color};
> We can complete in 10 seconds in my machine, which is still slow.
>  
> Since we are going to set default assignment strategy to 
> "CooperativeStickyAssignor" soon,  we should improve the scalability and 
> performance for sticky general assignor.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Comment Edited] (KAFKA-12675) Improve sticky general assignor scalability and performance

2021-04-18 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12675?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17324358#comment-17324358
 ] 

Travis Bischel edited comment on KAFKA-12675 at 4/19/21, 3:33 AM:
--

Great insight on getting rid of partition2AllPotentialConsumers, as well as 
keeping some more things sorted! I was able to translate that into my own code 
and dropped the large imbalance from 9.5s to 0.5s, as well as from 8.5G memory 
util to 0.5G :)

I'll take a look at the code more in depth soon.

Edit: after further improvements I was able to get the large imbalance down to 
220ms and 171MB,


was (Author: twmb):
Great insight on getting rid of partition2AllPotentialConsumers, as well as 
keeping some more things sorted! I was able to translate that into my own code 
and dropped the large imbalance from 9.5s to 0.5s, as well as from 8.5G memory 
util to 0.5G :)

I'll take a look at the code more in depth soon.

> Improve sticky general assignor scalability and performance
> ---
>
> Key: KAFKA-12675
> URL: https://issues.apache.org/jira/browse/KAFKA-12675
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Luke Chen
>Assignee: Luke Chen
>Priority: Major
>
> Currently, we have "general assignor" for non-equal subscription case and 
> "constrained assignor" for all equal subscription case. There's a performance 
> test for constrained assignor with:
> topicCount = {color:#ff}500{color};
>  partitionCount = {color:#ff}2000{color}; 
>  consumerCount = {color:#ff}2000{color};
> in _testLargeAssignmentAndGroupWithUniformSubscription,_ total 1 million 
> partitions and we can complete the assignment within 2 second in my machine.
> However, if we let 1 of the consumer subscribe to only 1 topic, it'll use 
> "general assignor", and the result with the same setting as above is: 
> *OutOfMemory,* 
>  Even we down the count to:
> topicCount = {color:#ff}50{color};
>  partitionCount = 1{color:#ff}000{color}; 
>  consumerCount = 1{color:#ff}000{color};
> We still got *OutOfMemory*.
> With this setting:
> topicCount = {color:#ff}50{color};
>  partitionCount = 8{color:#ff}00{color}; 
>  consumerCount = 8{color:#ff}00{color};
> We can complete in 10 seconds in my machine, which is still slow.
>  
> Since we are going to set default assignment strategy to 
> "CooperativeStickyAssignor" soon,  we should improve the scalability and 
> performance for sticky general assignor.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-12675) Improve sticky general assignor scalability and performance

2021-04-17 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12675?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17324358#comment-17324358
 ] 

Travis Bischel commented on KAFKA-12675:


Great insight on getting rid of partition2AllPotentialConsumers, as well as 
keeping some more things sorted! I was able to translate that into my own code 
and dropped the large imbalance from 9.5s to 0.5s, as well as from 8.5G memory 
util to 0.5G :)

I'll take a look at the code more in depth soon.

> Improve sticky general assignor scalability and performance
> ---
>
> Key: KAFKA-12675
> URL: https://issues.apache.org/jira/browse/KAFKA-12675
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Luke Chen
>Assignee: Luke Chen
>Priority: Major
>
> Currently, we have "general assignor" for non-equal subscription case and 
> "constrained assignor" for all equal subscription case. There's a performance 
> test for constrained assignor with:
> topicCount = {color:#ff}500{color};
>  partitionCount = {color:#ff}2000{color}; 
>  consumerCount = {color:#ff}2000{color};
> in _testLargeAssignmentAndGroupWithUniformSubscription,_ total 1 million 
> partitions and we can complete the assignment within 2 second in my machine.
> However, if we let 1 of the consumer subscribe to only 1 topic, it'll use 
> "general assignor", and the result with the same setting as above is: 
> *OutOfMemory,* 
>  Even we down the count to:
> topicCount = {color:#ff}50{color};
>  partitionCount = 1{color:#ff}000{color}; 
>  consumerCount = 1{color:#ff}000{color};
> We still got *OutOfMemory*.
> With this setting:
> topicCount = {color:#ff}50{color};
>  partitionCount = 8{color:#ff}00{color}; 
>  consumerCount = 8{color:#ff}00{color};
> We can complete in 10 seconds in my machine, which is still slow.
>  
> Since we are going to set default assignment strategy to 
> "CooperativeStickyAssignor" soon,  we should improve the scalability and 
> performance for sticky general assignor.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-12675) Improve sticky general assignor scalability and performance

2021-04-16 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12675?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17324153#comment-17324153
 ] 

Travis Bischel commented on KAFKA-12675:


Yep, that's as I understood it :). For me, running the _ 
testLargeAssignmentAndGroupWithUniformSubscription_ with a single extra 
consumer that consumes from one topic (causing an imbalance) results in my 
balancing algorithm to average 9.5s per balance.

> Improve sticky general assignor scalability and performance
> ---
>
> Key: KAFKA-12675
> URL: https://issues.apache.org/jira/browse/KAFKA-12675
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Luke Chen
>Assignee: Luke Chen
>Priority: Major
>
> Currently, we have "general assignor" for non-equal subscription case and 
> "constrained assignor" for all equal subscription case. There's a performance 
> test for constrained assignor with:
> topicCount = {color:#ff}500{color};
>  partitionCount = {color:#ff}2000{color}; 
>  consumerCount = {color:#ff}2000{color};
> in _testLargeAssignmentAndGroupWithUniformSubscription,_ total 1 million 
> partitions and we can complete the assignment within 2 second in my machine.
> However, if we let 1 of the consumer subscribe to only 1 topic, it'll use 
> "general assignor", and the result with the same setting as above is: 
> *OutOfMemory,* 
>  Even we down the count to:
> topicCount = {color:#ff}50{color};
>  partitionCount = 1{color:#ff}000{color}; 
>  consumerCount = 1{color:#ff}000{color};
> We still got *OutOfMemory*.
> With this setting:
> topicCount = {color:#ff}50{color};
>  partitionCount = 8{color:#ff}00{color}; 
>  consumerCount = 8{color:#ff}00{color};
> We can complete in 10 seconds in my machine, which is still slow.
>  
> Since we are going to set default assignment strategy to 
> "CooperativeStickyAssignor" soon,  we should improve the scalability and 
> performance for sticky general assignor.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-12675) Improve sticky general assignor scalability and performance

2021-04-16 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12675?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17324139#comment-17324139
 ] 

Travis Bischel commented on KAFKA-12675:


Interesting, I'm looking forward to seeing the changes, since 5s with the large 
imbalance beats my 9.5s!

> Improve sticky general assignor scalability and performance
> ---
>
> Key: KAFKA-12675
> URL: https://issues.apache.org/jira/browse/KAFKA-12675
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Luke Chen
>Assignee: Luke Chen
>Priority: Major
>
> Currently, we have "general assignor" for non-equal subscription case and 
> "constrained assignor" for all equal subscription case. There's a performance 
> test for constrained assignor with:
> topicCount = {color:#ff}500{color};
>  partitionCount = {color:#ff}2000{color}; 
>  consumerCount = {color:#ff}2000{color};
> in _testLargeAssignmentAndGroupWithUniformSubscription,_ total 1 million 
> partitions and we can complete the assignment within 2 second in my machine.
> However, if we let 1 of the consumer subscribe to only 1 topic, it'll use 
> "general assignor", and the result with the same setting as above is: 
> *OutOfMemory,* 
>  Even we down the count to:
> topicCount = {color:#ff}50{color};
>  partitionCount = 1{color:#ff}000{color}; 
>  consumerCount = 1{color:#ff}000{color};
> We still got *OutOfMemory*.
> With this setting:
> topicCount = {color:#ff}50{color};
>  partitionCount = 8{color:#ff}00{color}; 
>  consumerCount = 8{color:#ff}00{color};
> We can complete in 10 seconds in my machine, which is still slow.
>  
> Since we are going to set default assignment strategy to 
> "CooperativeStickyAssignor" soon,  we should improve the scalability and 
> performance for sticky general assignor.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-12671) Out of order processing with a transactional producer can lead to a stuck LastStableOffset

2021-04-16 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12671?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17324135#comment-17324135
 ] 

Travis Bischel commented on KAFKA-12671:


I've mostly worked around this issue in my client with this commit: 
https://github.com/twmb/franz-go/commit/10b743ed5cefa0ef321ce7c2bd12bb19ff529f28

Quoting the commit message,
"""
What we do is, when aborting, we wait for all sinks to be completely
done with their produce requests. This solves most of the problem to
begin with: before, we did not wait, so if we issued a produce request
we could immediately proceed to issuing EndTxn, and the EndTxn may be
handled first. By waiting, we _ensure_ that Kafka has handled our
produce requests.

However, if we wait and the request is cut, it may be that our
connection died right after writing the produce request. In this case,
we now see that the last produce request had an issuing error, and we
wait 1s before sending EndTxn. The hope is that this 1s is enough time
for the ProduceRequest to be processed by Kafka itself.

We have effectively changed this issue from a slim change to an
extremely slim chance under very bad conditions.
"""

I do think that this issue is worth solving within the broker itself.

> Out of order processing with a transactional producer can lead to a stuck 
> LastStableOffset
> --
>
> Key: KAFKA-12671
> URL: https://issues.apache.org/jira/browse/KAFKA-12671
> Project: Kafka
>  Issue Type: Bug
>Reporter: Travis Bischel
>Priority: Major
>
> If there is pathological processing of incoming produce requests and EndTxn 
> requests, then the LastStableOffset can get stuck, which will block consuming 
> in READ_COMMITTED mode.
> To transactionally produce, the standard flow is to InitProducerId, and then 
> loop AddPartitionsToTxn -> Produce+ -> EndTxn. The AddPartitionsToTxn is 
> responsible for fencing and adding partitions to a transaction, and the end 
> transaction is responsible for finishing the transaction. Producing itself is 
> mostly uninvolved with the proper fencing / ending flow, but produce requests 
> are required to be after AddPartitionsToTxn and before EndTxn.
> When a ProduceRequest is handled, Kafka uses an internal ProducerStateManager 
> to mildly manage transactions. The ProducerStateManager is completely 
> independent of the TxnCoordinator, and its guarantees are relatively weak. 
> The ProducerStateManager handles two types of "batches" being added: a data 
> batch and a transaction marker. When a data batch is added, a "transaction" 
> is begun and tied to the producer ID that is producing the batch. When a 
> transaction marker is handled, the ProducerStateManager removes the 
> transaction for the producer ID (roughly).
> EndTxn is what triggers transaction markers to be sent to the 
> ProducerStateManager. In essence, EndTxn is the one part of the transactional 
> producer flow that talks across both the TxnCoordinator and the 
> ProducerStateManager.
> If a ProduceRequest is issued before EndTxn, but handled internally in Kafka 
> after EndTxn, then the ProduceRequest will begin a new transaction in the 
> ProducerStateManager. If the client was disconnecting, and the EndTxn was the 
> final request issued, the new transaction created in ProducerStateManager is 
> orphaned and nothing can clean it up. The LastStableOffset then hangs based 
> off of this hung transaction.
> This same problem can be triggered by a produce request that is issued with a 
> transactional ID outside of the context of a transaction at all (no 
> AddPartitionsToTxn). This problem cannot be triggered by producing for so 
> long that the transaction expires; the difference here is that the 
> transaction coordinator bumps the epoch for the producer ID, thus producing 
> again with the old epoch does not work.
> Theoretically, we are supposed have unlimited retries on produce requests, 
> but in the context of wanting to abort everything and shut down, this is not 
> always feasible. As it currently stands, I'm not sure there's a truly safe 
> way to shut down _without_ flushing and receiving responses for every record 
> produced, even if I want to abort everything and quit. The safest approach I 
> can think of is to actually avoid issuing an EndTxn so that instead we rely 
> on Kafka internally to time things out after a period of time.
> —
> For some context, here's my request logs from the client. Note that I write 
> two ProduceRequests, read one, and then issue EndTxn (because I know I want 
> to quit). The second ProduceRequest is read successfully before shutdown, but 
> I ignore the results because I am shutting down. I've taken out logs related 
> to consuming, but the order of the logs is unchanged:
> {noformat}
> [INFO] done 

[jira] [Comment Edited] (KAFKA-12675) Improve sticky general assignor scalability and performance

2021-04-15 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12675?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17322603#comment-17322603
 ] 

Travis Bischel edited comment on KAFKA-12675 at 4/16/21, 4:25 AM:
--

What I mean to say is that the logic powering the existing sticky algorithm is 
heuristic and not truly balanced, and that the logic itself can be changed to 
be more exact to the sticky goals while being much more efficient. That is, 
changes can be made for the imbalanced case similar to how [~ableegoldman] made 
changes to the balanced case (for KAFKA-9987), and these changes will more 
exactly fulfill the goal of sticky balancing while being more efficient. This 
does not change how things are balanced / it does not change the actual sticky 
aspect. Basically, improving the underlying algorithm for the imbalanced case 
directly fulfills the goals of this ticket to improve the scalability and 
performance.

For some numbers:
{noformat}
$ go test -run nothing -bench Java -benchmem -v -benchtime 60s
goos: darwin
goarch: amd64
pkg: github.com/twmb/franz-go/pkg/kgo/internal/sticky
cpu: Intel(R) Core(TM) i7-1068NG7 CPU @ 2.30GHz
BenchmarkJava
BenchmarkJava/large
sticky_test.go:1419: avg 2.696451608s per 1 balances of 2000 members and 
100 total partitions
sticky_test.go:1419: avg 2.573368814s per 26 balances of 2000 members and 
100 total partitions
BenchmarkJava/large-8 262573370133 ns/op
531478500 B/op   1038983 allocs/op
BenchmarkJava/large_imbalance
sticky_test.go:1419: avg 13.798672936s per 1 balances of 2001 members and 
100 total partitions
sticky_test.go:1419: avg 9.581320518s per 4 balances of 2001 members and 
100 total partitions
sticky_test.go:1419: avg 9.626729812s per 7 balances of 2001 members and 
100 total partitions
BenchmarkJava/large_imbalance-879626739151 
ns/op8535692965 B/op  1039100 allocs/op
BenchmarkJava/medium
sticky_test.go:1419: avg 77.798053ms per 1 balances of 1000 members and 
5 total partitions
sticky_test.go:1419: avg 72.271454ms per 100 balances of 1000 members and 
5 total partitions
sticky_test.go:1419: avg 72.044377ms per 996 balances of 1000 members and 
5 total partitions
BenchmarkJava/medium-8   996  72044411 
ns/op22502623 B/op  56085 allocs/op
BenchmarkJava/medium_imbalance
sticky_test.go:1419: avg 216.340842ms per 1 balances of 1001 members and 
5 total partitions
sticky_test.go:1419: avg 217.385765ms per 100 balances of 1001 members and 
5 total partitions
sticky_test.go:1419: avg 218.218478ms per 331 balances of 1001 members and 
5 total partitions
BenchmarkJava/medium_imbalance-8 331 218218666 
ns/op222795358 B/op 56097 allocs/op
BenchmarkJava/small
sticky_test.go:1419: avg 52.22238ms per 1 balances of 800 members and 4 
total partitions
sticky_test.go:1419: avg 50.190192ms per 100 balances of 800 members and 
4 total partitions
sticky_test.go:1419: avg 50.252975ms per 1434 balances of 800 members and 
4 total partitions
BenchmarkJava/small-8   1434  50253022 
ns/op18823337 B/op  44906 allocs/op
BenchmarkJava/small_imbalance
sticky_test.go:1419: avg 149.416236ms per 1 balances of 801 members and 
4 total partitions
sticky_test.go:1419: avg 149.050743ms per 100 balances of 801 members and 
4 total partitions
sticky_test.go:1419: avg 149.224721ms per 482 balances of 801 members and 
4 total partitions
BenchmarkJava/small_imbalance-8  482 149224854 
ns/op147060761 B/op 44914 allocs/op
{noformat}

I've just pushed the code for this benchmark in [this 
commit|https://github.com/twmb/franz-go/commit/e0c960e094e8f100924411f6c5fb514b79fc761a].

This is still clearly using a decent amount of memory (up to 8G in the 
imbalanced case), but I spent a good amount of time already optimizing how much 
memory this can consume. I'm sure I can lose some speed to drop some memory 
usage. But, as it stands, 1mil partitions and 2000 members just has to take up 
some memory.

What I mean by heuristic is that the general assignor does not really have much 
reasoning for what it is doing, it just somewhat tries a bunch of things and 
then says "good enough" at a certain point. It also tries a bunch of things, 
checks to see if what it did got a better balance score, and if not, discards 
what it tried. This can be seen on lines 642 and 643 of 
[AbstractStickyAssignor.java|https://github.com/apache/kafka/blob/637c44c976c115b7e770a6fd9e62e8822051b45b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java#L642-L643]:
{noformat}
   // if we are not preserving existing assignments 

[jira] [Comment Edited] (KAFKA-12675) Improve sticky general assignor scalability and performance

2021-04-15 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12675?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17322603#comment-17322603
 ] 

Travis Bischel edited comment on KAFKA-12675 at 4/16/21, 4:24 AM:
--

What I mean to say is that the logic powering the existing cooperative-sticky 
algorithm is heuristic and not truly balanced, and that the logic itself can be 
changed to be more exact to the cooperative-sticky goals while being much more 
efficient. That is, changes can be made for the imbalanced case similar to how 
[~ableegoldman] made changes to the balanced case (for KAFKA-9987), and these 
changes will more exactly fulfill the goal of cooperative sticky while being 
more efficient. This does not change how things are balanced / it does not 
change the actual sticky aspect. Basically, improving the underlying algorithm 
for the imbalanced case directly fulfills the goals of this ticket to improve 
the scalability and performance.

For some numbers:
{noformat}
$ go test -run nothing -bench Java -benchmem -v -benchtime 60s
goos: darwin
goarch: amd64
pkg: github.com/twmb/franz-go/pkg/kgo/internal/sticky
cpu: Intel(R) Core(TM) i7-1068NG7 CPU @ 2.30GHz
BenchmarkJava
BenchmarkJava/large
sticky_test.go:1419: avg 2.696451608s per 1 balances of 2000 members and 
100 total partitions
sticky_test.go:1419: avg 2.573368814s per 26 balances of 2000 members and 
100 total partitions
BenchmarkJava/large-8 262573370133 ns/op
531478500 B/op   1038983 allocs/op
BenchmarkJava/large_imbalance
sticky_test.go:1419: avg 13.798672936s per 1 balances of 2001 members and 
100 total partitions
sticky_test.go:1419: avg 9.581320518s per 4 balances of 2001 members and 
100 total partitions
sticky_test.go:1419: avg 9.626729812s per 7 balances of 2001 members and 
100 total partitions
BenchmarkJava/large_imbalance-879626739151 
ns/op8535692965 B/op  1039100 allocs/op
BenchmarkJava/medium
sticky_test.go:1419: avg 77.798053ms per 1 balances of 1000 members and 
5 total partitions
sticky_test.go:1419: avg 72.271454ms per 100 balances of 1000 members and 
5 total partitions
sticky_test.go:1419: avg 72.044377ms per 996 balances of 1000 members and 
5 total partitions
BenchmarkJava/medium-8   996  72044411 
ns/op22502623 B/op  56085 allocs/op
BenchmarkJava/medium_imbalance
sticky_test.go:1419: avg 216.340842ms per 1 balances of 1001 members and 
5 total partitions
sticky_test.go:1419: avg 217.385765ms per 100 balances of 1001 members and 
5 total partitions
sticky_test.go:1419: avg 218.218478ms per 331 balances of 1001 members and 
5 total partitions
BenchmarkJava/medium_imbalance-8 331 218218666 
ns/op222795358 B/op 56097 allocs/op
BenchmarkJava/small
sticky_test.go:1419: avg 52.22238ms per 1 balances of 800 members and 4 
total partitions
sticky_test.go:1419: avg 50.190192ms per 100 balances of 800 members and 
4 total partitions
sticky_test.go:1419: avg 50.252975ms per 1434 balances of 800 members and 
4 total partitions
BenchmarkJava/small-8   1434  50253022 
ns/op18823337 B/op  44906 allocs/op
BenchmarkJava/small_imbalance
sticky_test.go:1419: avg 149.416236ms per 1 balances of 801 members and 
4 total partitions
sticky_test.go:1419: avg 149.050743ms per 100 balances of 801 members and 
4 total partitions
sticky_test.go:1419: avg 149.224721ms per 482 balances of 801 members and 
4 total partitions
BenchmarkJava/small_imbalance-8  482 149224854 
ns/op147060761 B/op 44914 allocs/op
{noformat}

I've just pushed the code for this benchmark in [this 
commit|https://github.com/twmb/franz-go/commit/e0c960e094e8f100924411f6c5fb514b79fc761a].

This is still clearly using a decent amount of memory (up to 8G in the 
imbalanced case), but I spent a good amount of time already optimizing how much 
memory this can consume. I'm sure I can lose some speed to drop some memory 
usage. But, as it stands, 1mil partitions and 2000 members just has to take up 
some memory.

What I mean by heuristic is that the general assignor does not really have much 
reasoning for what it is doing, it just somewhat tries a bunch of things and 
then says "good enough" at a certain point. It also tries a bunch of things, 
checks to see if what it did got a better balance score, and if not, discards 
what it tried. This can be seen on lines 642 and 643 of 
[AbstractStickyAssignor.java|https://github.com/apache/kafka/blob/637c44c976c115b7e770a6fd9e62e8822051b45b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java#L642-L643]:
{noformat}
   // if we are not 

[jira] [Comment Edited] (KAFKA-12675) Improve sticky general assignor scalability and performance

2021-04-15 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12675?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17322603#comment-17322603
 ] 

Travis Bischel edited comment on KAFKA-12675 at 4/16/21, 4:10 AM:
--

What I mean to say is that the logic powering the existing cooperative-sticky 
algorithm is heuristic and not truly balanced, and that the logic itself can be 
changed to be more exact to the cooperative-sticky goals while being much more 
efficient. That is, changes can be made for the imbalanced case similar to how 
[~ableegoldman] made changes to the balanced case (for KAFKA-9987), and these 
changes will more exactly fulfill the goal of cooperative sticky while being 
more efficient. This does not change how things are balanced / it does not 
change the actual sticky aspect. Basically, improving the underlying algorithm 
for the imbalanced case directly fulfills the goals of this ticket to improve 
the scalability and performance. I'll edit this comment shortly with some 
benchmarking numbers.


was (Author: twmb):
What I mean to say is that the logic powering the existing cooperative-sticky 
algorithm is heuristic and not truly balanced, and that the logic itself can be 
changed to be more exact to the cooperative-sticky goals while being much more 
efficient. That is, changes can be made for the imbalanced case similar to how 
[~ableegoldman] made changes to the balanced case, and these changes will more 
exactly fulfill the goal of cooperative sticky while being more efficient. This 
does not change how things are balanced / it does not change the actual sticky 
aspect. Basically, improving the underlying algorithm for the imbalanced case 
directly fulfills the goals of this ticket to improve the scalability and 
performance. I'll edit this comment shortly with some benchmarking numbers.

> Improve sticky general assignor scalability and performance
> ---
>
> Key: KAFKA-12675
> URL: https://issues.apache.org/jira/browse/KAFKA-12675
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Luke Chen
>Assignee: Luke Chen
>Priority: Major
>
> Currently, we have "general assignor" for non-equal subscription case and 
> "constrained assignor" for all equal subscription case. There's a performance 
> test for constrained assignor with:
> topicCount = {color:#ff}500{color};
>  partitionCount = {color:#ff}2000{color}; 
>  consumerCount = {color:#ff}2000{color};
> in _testLargeAssignmentAndGroupWithUniformSubscription,_ total 1 million 
> partitions and we can complete the assignment within 2 second in my machine.
> However, if we let 1 of the consumer subscribe to only 1 topic, it'll use 
> "general assignor", and the result with the same setting as above is: 
> *OutOfMemory,* 
>  Even we down the count to:
> topicCount = {color:#ff}50{color};
>  partitionCount = 1{color:#ff}000{color}; 
>  consumerCount = 1{color:#ff}000{color};
> We still got *OutOfMemory*.
> With this setting:
> topicCount = {color:#ff}50{color};
>  partitionCount = 8{color:#ff}00{color}; 
>  consumerCount = 8{color:#ff}00{color};
> We can complete in 10 seconds in my machine, which is still slow.
>  
> Since we are going to set default assignment strategy to 
> "CooperativeStickyAssignor" soon,  we should improve the scalability and 
> performance for sticky general assignor.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-12675) Improve sticky general assignor scalability and performance

2021-04-15 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12675?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17322603#comment-17322603
 ] 

Travis Bischel commented on KAFKA-12675:


What I mean to say is that the logic powering the existing cooperative-sticky 
algorithm is heuristic and not truly balanced, and that the logic itself can be 
changed to be more exact to the cooperative-sticky goals while being much more 
efficient. That is, changes can be made for the imbalanced case similar to how 
[~ableegoldman] made changes to the balanced case, and these changes will more 
exactly fulfill the goal of cooperative sticky while being more efficient. This 
does not change how things are balanced / it does not change the actual sticky 
aspect. Basically, improving the underlying algorithm for the imbalanced case 
directly fulfills the goals of this ticket to improve the scalability and 
performance. I'll edit this comment shortly with some benchmarking numbers.

> Improve sticky general assignor scalability and performance
> ---
>
> Key: KAFKA-12675
> URL: https://issues.apache.org/jira/browse/KAFKA-12675
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Luke Chen
>Assignee: Luke Chen
>Priority: Major
>
> Currently, we have "general assignor" for non-equal subscription case and 
> "constrained assignor" for all equal subscription case. There's a performance 
> test for constrained assignor with:
> topicCount = {color:#ff}500{color};
>  partitionCount = {color:#ff}2000{color}; 
>  consumerCount = {color:#ff}2000{color};
> in _testLargeAssignmentAndGroupWithUniformSubscription,_ total 1 million 
> partitions and we can complete the assignment within 2 second in my machine.
> However, if we let 1 of the consumer subscribe to only 1 topic, it'll use 
> "general assignor", and the result with the same setting as above is: 
> *OutOfMemory,* 
>  Even we down the count to:
> topicCount = {color:#ff}50{color};
>  partitionCount = 1{color:#ff}000{color}; 
>  consumerCount = 1{color:#ff}000{color};
> We still got *OutOfMemory*.
> With this setting:
> topicCount = {color:#ff}50{color};
>  partitionCount = 8{color:#ff}00{color}; 
>  consumerCount = 8{color:#ff}00{color};
> We can complete in 10 seconds in my machine, which is still slow.
>  
> Since we are going to set default assignment strategy to 
> "CooperativeStickyAssignor" soon,  we should improve the scalability and 
> performance for sticky general assignor.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-12675) Improve sticky general assignor scalability and performance

2021-04-15 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-12675?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17322593#comment-17322593
 ] 

Travis Bischel commented on KAFKA-12675:


An option to evaluate is the algorithm I devised in my franz-go client, which 
translates the balancing into a graph and uses A* search to perform an exact 
balance much more efficiently. I noticed that the existing Java algorithm is 
heuristic based, and I have a few tests in my repo showing edge cases that the 
existing heuristic algorithm cannot really handle.

The algorithm is here: 
https://github.com/twmb/franz-go/blob/master/pkg/kgo/internal/sticky/graph.go
with the option to switch into that algorithm in the sticky.go file.

> Improve sticky general assignor scalability and performance
> ---
>
> Key: KAFKA-12675
> URL: https://issues.apache.org/jira/browse/KAFKA-12675
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Luke Chen
>Assignee: Luke Chen
>Priority: Major
>
> Currently, we have "general assignor" for non-equal subscription case and 
> "constrained assignor" for all equal subscription case. There's a performance 
> test for constrained assignor with:
> topicCount = {color:#ff}500{color};
>  partitionCount = {color:#ff}2000{color}; 
>  consumerCount = {color:#ff}2000{color};
> in _testLargeAssignmentAndGroupWithUniformSubscription,_ total 1 million 
> partitions and we can complete the assignment within 2 second in my machine.
> However, if we let 1 of the consumer subscribe to only 1 topic, it'll use 
> "general assignor", and the result with the same setting as above is: 
> *OutOfMemory,* 
>  Even we down the count to:
> topicCount = {color:#ff}50{color};
>  partitionCount = 1{color:#ff}000{color}; 
>  consumerCount = 1{color:#ff}000{color};
> We still got *OutOfMemory*.
> With this setting:
> topicCount = {color:#ff}50{color};
>  partitionCount = 8{color:#ff}00{color}; 
>  consumerCount = 8{color:#ff}00{color};
> We can complete in 10 seconds in my machine, which is still slow.
>  
> Since we are going to set default assignment strategy to 
> "CooperativeStickyAssignor" soon,  we should improve the scalability and 
> performance for sticky general assignor.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Updated] (KAFKA-12671) Out of order processing with a transactional producer can lead to a stuck LastStableOffset

2021-04-15 Thread Travis Bischel (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-12671?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Travis Bischel updated KAFKA-12671:
---
Description: 
If there is pathological processing of incoming produce requests and EndTxn 
requests, then the LastStableOffset can get stuck, which will block consuming 
in READ_COMMITTED mode.

To transactionally produce, the standard flow is to InitProducerId, and then 
loop AddPartitionsToTxn -> Produce+ -> EndTxn. The AddPartitionsToTxn is 
responsible for fencing and adding partitions to a transaction, and the end 
transaction is responsible for finishing the transaction. Producing itself is 
mostly uninvolved with the proper fencing / ending flow, but produce requests 
are required to be after AddPartitionsToTxn and before EndTxn.

When a ProduceRequest is handled, Kafka uses an internal ProducerStateManager 
to mildly manage transactions. The ProducerStateManager is completely 
independent of the TxnCoordinator, and its guarantees are relatively weak. The 
ProducerStateManager handles two types of "batches" being added: a data batch 
and a transaction marker. When a data batch is added, a "transaction" is begun 
and tied to the producer ID that is producing the batch. When a transaction 
marker is handled, the ProducerStateManager removes the transaction for the 
producer ID (roughly).

EndTxn is what triggers transaction markers to be sent to the 
ProducerStateManager. In essence, EndTxn is the one part of the transactional 
producer flow that talks across both the TxnCoordinator and the 
ProducerStateManager.

If a ProduceRequest is issued before EndTxn, but handled internally in Kafka 
after EndTxn, then the ProduceRequest will begin a new transaction in the 
ProducerStateManager. If the client was disconnecting, and the EndTxn was the 
final request issued, the new transaction created in ProducerStateManager is 
orphaned and nothing can clean it up. The LastStableOffset then hangs based off 
of this hung transaction.

This same problem can be triggered by a produce request that is issued with a 
transactional ID outside of the context of a transaction at all (no 
AddPartitionsToTxn). This problem cannot be triggered by producing for so long 
that the transaction expires; the difference here is that the transaction 
coordinator bumps the epoch for the producer ID, thus producing again with the 
old epoch does not work.

Theoretically, we are supposed have unlimited retries on produce requests, but 
in the context of wanting to abort everything and shut down, this is not always 
feasible. As it currently stands, I'm not sure there's a truly safe way to shut 
down _without_ flushing and receiving responses for every record produced, even 
if I want to abort everything and quit. The safest approach I can think of is 
to actually avoid issuing an EndTxn so that instead we rely on Kafka internally 
to time things out after a period of time.

—

For some context, here's my request logs from the client. Note that I write two 
ProduceRequests, read one, and then issue EndTxn (because I know I want to 
quit). The second ProduceRequest is read successfully before shutdown, but I 
ignore the results because I am shutting down. I've taken out logs related to 
consuming, but the order of the logs is unchanged:
{noformat}
[INFO] done waiting for unknown topic, metadata was successful; topic: 
2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765
[INFO] initializing producer id
[DEBUG] wrote FindCoordinator v3; err: 
[DEBUG] read FindCoordinator v3; err: 
[DEBUG] wrote InitProducerID v4; err: 
[DEBUG] read InitProducerID v4; err: 
[INFO] producer id initialization success; id: 1463, epoch: 0
[DEBUG] wrote AddPartitionsToTxn v2; err: 
[DEBUG] read AddPartitionsToTxn v2; err: 
[DEBUG] wrote Produce v8; err: 
[DEBUG] read Produce v8; err: 
[DEBUG] produced; to: 
2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765[1{15589=>19686}]
[DEBUG] wrote Produce v8; err: 
[DEBUG] wrote EndTxn v2; err: 
[DEBUG] read EndTxn v2; err: 
[DEBUG] read from broker errored, killing connection; addr: localhost:9092, id: 
1, successful_reads: 1, err: context canceled
[DEBUG] read Produce v8; err: 
[DEBUG] produced; to: 
2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765[1{skipped}]
{noformat}
And then from the broker's point of view. Across two brokers, the second 
ProduceRequest is completed after EndTxn is handled (and after the 
WriteTxnMarkers request is handled, which is the important one that hooks into 
the ProducerStateManager):
{noformat}
/// Broker 3: init producer ID
[2021-04-15 00:56:40,030] DEBUG Completed 
request:RequestHeader(apiKey=INIT_PRODUCER_ID, apiVersion=4, clientId=kgo, 
correlationId=3) -- 

[jira] [Updated] (KAFKA-12671) Out of order processing with a transactional producer can lead to a stuck LastStableOffset

2021-04-15 Thread Travis Bischel (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-12671?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Travis Bischel updated KAFKA-12671:
---
Description: 
If there is pathological processing of incoming produce requests and EndTxn 
requests, then the LastStableOffset can get stuck, which will block consuming 
in READ_COMMITTED mode.

To transactionally produce, the standard flow is to InitProducerId, and then 
loop AddPartitionsToTxn -> Produce+ -> EndTxn. The AddPartitionsToTxn is 
responsible for fencing and adding partitions to a transaction, and the end 
transaction is responsible for finishing the transaction. Producing itself is 
mostly uninvolved with the proper fencing / ending flow, but produce requests 
are required to be after AddPartitionsToTxn and before EndTxn.

When a ProduceRequest is handled, Kafka uses an internal ProducerStateManager 
to mildly manage transactions. The ProducerStateManager is completely 
independent of the TxnCoordinator, and its guarantees are relatively weak. The 
ProducerStateManager handles two types of "batches" being added: a data batch 
and a transaction marker. When a data batch is added, a "transaction" is begun 
and tied to the producer ID that is producing the batch. When a transaction 
marker is handled, the ProducerStateManager removes the transaction for the 
producer ID (roughly).

EndTxn is what triggers transaction markers to be sent to the 
ProducerStateManager. In essence, EndTxn is the one part of the transactional 
producer flow that talks across both the TxnCoordinator and the 
ProducerStateManager.

If a ProduceRequest is issued before EndTxn, but handled internally in Kafka 
after EndTxn, then the ProduceRequest will begin a new transaction in the 
ProducerStateManager. If the client was disconnecting, and the EndTxn was the 
final request issued, the new transaction created in ProducerStateManager is 
orphaned and nothing can clean it up. The LastStableOffset then hangs based off 
of this hung transaction.

This same problem can be triggered by a produce request that is issued with a 
transactional ID outside of the context of a transaction at all (no 
AddPartitionsToTxn). This problem cannot be triggered by producing for so long 
that the transaction expires; the difference here is that the transaction 
coordinator bumps the epoch for the producer ID, thus producing again with the 
old epoch does not work.

Theoretically, we are supposed have unlimited retries on produce requests, but 
in the context of wanting to abort everything and shut down, this is not always 
feasible. As it currently stands, I'm not sure there's a truly safe way to shut 
down _without_ flushing and receiving responses for every record produced, even 
if I want to abort everything and quit. The safest approach I can think of is 
to actually avoid issuing an EndTxn so that instead we rely on Kafka internally 
to time things out after a period of time.

—

For some context, here's my request logs from the client. Note that I write two 
ProduceRequests, read one, and then issue EndTxn (because I know I want to 
quit). The second ProduceRequest is read successfully before shutdown, but I 
ignore the results because I am shutting down. I've taken out logs related to 
consuming, but the order of the logs is unchanged:
{noformat}
[INFO] done waiting for unknown topic, metadata was successful; topic: 
2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765
[INFO] initializing producer id
[DEBUG] wrote FindCoordinator v3; err: 
[DEBUG] read FindCoordinator v3; err: 
[DEBUG] wrote InitProducerID v4; err: 
[DEBUG] read InitProducerID v4; err: 
[INFO] producer id initialization success; id: 1463, epoch: 0

[DEBUG] wrote AddPartitionsToTxn v2; err: 
[DEBUG] read AddPartitionsToTxn v2; err: 

[DEBUG] read Produce v8; err: 
[DEBUG] produced; to: 
2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765[1{15589=>19686}]
[DEBUG] wrote Produce v8; err: 

[DEBUG] wrote EndTxn v2; err: 
[DEBUG] read EndTxn v2; err: 

[DEBUG] read from broker errored, killing connection; addr: localhost:9092, id: 
1, successful_reads: 1, err: context canceled
[DEBUG] read Produce v8; err: 
[DEBUG] produced; to: 
2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765[1{skipped}]
{noformat}
And then from the broker's point of view. Across two brokers, the second 
ProduceRequest is completed after EndTxn is handled (and after the 
WriteTxnMarkers request is handled, which is the important one that hooks into 
the ProducerStateManager):
{noformat}
/// Broker 3: init producer ID
[2021-04-15 00:56:40,030] DEBUG Completed 
request:RequestHeader(apiKey=INIT_PRODUCER_ID, apiVersion=4, clientId=kgo, 
correlationId=3) -- 

[jira] [Created] (KAFKA-12671) Out of order processing with a transactional producer can lead to a stuck LastStableOffset

2021-04-15 Thread Travis Bischel (Jira)
Travis Bischel created KAFKA-12671:
--

 Summary: Out of order processing with a transactional producer can 
lead to a stuck LastStableOffset
 Key: KAFKA-12671
 URL: https://issues.apache.org/jira/browse/KAFKA-12671
 Project: Kafka
  Issue Type: Bug
Reporter: Travis Bischel


If there is pathological processing of incoming produce requests and EndTxn 
requests, then the LastStableOffset can get stuck, which will block consuming 
in READ_COMMITTED mode.

To transactionally produce, the standard flow is to InitProducerId, and then 
loop AddPartitionsToTxn -> Produce+ -> EndTxn. The AddPartitionsToTxn is 
responsible for fencing and adding partitions to a transaction, and the end 
transaction is responsible for finishing the transaction. Producing itself is 
mostly uninvolved with the proper fencing / ending flow, but produce requests 
are required to be after AddPartitionsToTxn and before EndTxn.

When a ProduceRequest is handled, Kafka uses an internal ProducerStateManager 
to mildly manage transactions. The ProducerStateManager is completely 
independent of the TxnCoordinator, and its guarantees are relatively weak. The 
ProducerStateManager handles two types of "batches" being added: a data batch 
and a transaction marker. When a data batch is added, a "transaction" is begun 
and tied to the producer ID that is producing the batch. When a transaction 
marker is handled, the ProducerStateManager removes the transaction for the 
producer ID (roughly).

EndTxn is what triggers transaction markers to be sent to the 
ProducerStateManager. In essence, EndTxn is the one part of the transactional 
producer flow that talks across both the TxnCoordinator and the 
ProducerStateManager.

If a ProduceRequest is issued before EndTxn, but handled internally in Kafka 
after EndTxn, then the ProduceRequest will begin a new transaction in the 
ProducerStateManager. If the client was disconnecting, and the EndTxn was the 
final request issued, the new transaction created in ProducerStateManager is 
orphaned and nothing can clean it up. The LastStableOffset then hangs based off 
of this hung transaction.

This same problem can be triggered by a produce request that is issued with a 
transactional ID outside of the context of a transaction at all (no 
InitProducerId). This problem cannot be triggered by producing for so long that 
the transaction expires; the difference here is that the transaction 
coordinator bumps the epoch for the producer ID, thus producing again with the 
old epoch does not work.

Theoretically, we are supposed have unlimited retries on produce requests, but 
in the context of wanting to abort everything and shut down, this is not always 
feasible. As it currently stands, I'm not sure there's a truly safe way to shut 
down _without_ flushing and receiving responses for every record produced, even 
if I want to abort everything and quit. The safest approach I can think of is 
to actually avoid issuing an EndTxn so that instead we rely on Kafka internally 
to time things out after a period of time.

---

For some context, here's my request logs from the client. Note that I write two 
ProduceRequests, read one, and then issue EndTxn (because I know I want to 
quit). The second ProduceRequest is read successfully before shutdown, but I 
ignore the results because I am shutting down. I've taken out logs related to 
consuming, but the order of the logs is unchanged:

{noformat}
[INFO] done waiting for unknown topic, metadata was successful; topic: 
2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765
[INFO] initializing producer id
[DEBUG] wrote FindCoordinator v3; err: 
[DEBUG] read FindCoordinator v3; err: 
[DEBUG] wrote InitProducerID v4; err: 
[DEBUG] read InitProducerID v4; err: 
[INFO] producer id initialization success; id: 1463, epoch: 0

[DEBUG] wrote AddPartitionsToTxn v2; err: 
[DEBUG] read AddPartitionsToTxn v2; err: 

[DEBUG] read Produce v8; err: 
[DEBUG] produced; to: 
2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765[1{15589=>19686}]
[DEBUG] wrote Produce v8; err: 

[DEBUG] wrote EndTxn v2; err: 
[DEBUG] read EndTxn v2; err: 

[DEBUG] read from broker errored, killing connection; addr: localhost:9092, id: 
1, successful_reads: 1, err: context canceled
[DEBUG] read Produce v8; err: 
[DEBUG] produced; to: 
2135e281e688406306830c19b0aca37476dd5b0b4b50689672a9244203721765[1{skipped}]
{noformat}

And then from the broker's point of view. Across two brokers, the second 
ProduceRequest is completed after EndTxn is handled (and after the 
WriteTxnMarkers request is handled, which is the important one that hooks into 
the ProducerStateManager):

{noformat}
/// Broker 3: init producer ID
[2021-04-15 00:56:40,030] DEBUG Completed 
request:RequestHeader(apiKey=INIT_PRODUCER_ID, apiVersion=4, clientId=kgo, 
correlationId=3) -- 

[jira] [Commented] (KAFKA-7740) Kafka Admin Client should be able to manage user/client configurations for users and clients

2020-08-24 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-7740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17183512#comment-17183512
 ] 

Travis Bischel commented on KAFKA-7740:
---

Oh cool, that's great. I thought that aspect was completely removed, but I 
didn't know there was a different blocker. Sorry for the ping!

> Kafka Admin Client should be able to manage user/client configurations for 
> users and clients
> 
>
> Key: KAFKA-7740
> URL: https://issues.apache.org/jira/browse/KAFKA-7740
> Project: Kafka
>  Issue Type: Improvement
>  Components: clients
>Affects Versions: 1.1.0, 1.1.1, 2.0.0, 2.1.0
> Environment: linux
>Reporter: Yaodong Yang
>Assignee: Brian Byrne
>Priority: Major
>  Labels: features
> Fix For: 2.6.0
>
>
> Right now, Kafka Admin Client only allow users to change the configuration of 
> brokers and topics. There are some use cases that users want to setOrUpdate 
> quota configurations for users and clients through Kafka Admin Client. 
> Without this new capability, users have to manually talk to zookeeper for 
> this, which will pose other challenges for customers.
> Considering we have already have the framework for the much complex brokers 
> and topic configuration changes, it seems straightforward to add the support 
> for the alterConfig and describeConfig for users and clients as well.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-7740) Kafka Admin Client should be able to manage user/client configurations for users and clients

2020-08-23 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-7740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17182928#comment-17182928
 ] 

Travis Bischel commented on KAFKA-7740:
---

Hi,

The KIP still refers to the whole "resolve" request. Can the KIP be updated to 
reflect the final implementation?'

Thanks!

> Kafka Admin Client should be able to manage user/client configurations for 
> users and clients
> 
>
> Key: KAFKA-7740
> URL: https://issues.apache.org/jira/browse/KAFKA-7740
> Project: Kafka
>  Issue Type: Improvement
>  Components: clients
>Affects Versions: 1.1.0, 1.1.1, 2.0.0, 2.1.0
> Environment: linux
>Reporter: Yaodong Yang
>Assignee: Brian Byrne
>Priority: Major
>  Labels: features
> Fix For: 2.6.0
>
>
> Right now, Kafka Admin Client only allow users to change the configuration of 
> brokers and topics. There are some use cases that users want to setOrUpdate 
> quota configurations for users and clients through Kafka Admin Client. 
> Without this new capability, users have to manually talk to zookeeper for 
> this, which will pose other challenges for customers.
> Considering we have already have the framework for the much complex brokers 
> and topic configuration changes, it seems straightforward to add the support 
> for the alterConfig and describeConfig for users and clients as well.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9494) Include data type of the config in ConfigEntry

2020-06-01 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9494?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17121211#comment-17121211
 ] 

Travis Bischel commented on KAFKA-9494:
---

Thanks!

> Include data type of the config in ConfigEntry
> --
>
> Key: KAFKA-9494
> URL: https://issues.apache.org/jira/browse/KAFKA-9494
> Project: Kafka
>  Issue Type: Improvement
>  Components: clients, core
>Reporter: Shailesh Panwar
>Priority: Minor
> Fix For: 2.6.0
>
>
> Why this request?
> To provide better validation. Including the data type can significantly 
> improve the validation on client side (be it web or cli or any other client). 
> In the absence of `type` the only way to know if the user specified value is 
> correct is to make an `alter` call and check if there is no error.
>  
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Comment Edited] (KAFKA-9987) Improve sticky partition assignor algorithm

2020-05-31 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9987?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17108705#comment-17108705
 ] 

Travis Bischel edited comment on KAFKA-9987 at 5/31/20, 11:19 PM:
--

For context, here's my current benchmarks (WithExisting mirrors an existing 
cluster rejoining, Imbalanced means unequal subscriptions):

{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 24104 total partitions; 100 total 
members
BenchmarkLarge: sticky_test.go:1272: 24104 total partitions; 100 total 
members
BenchmarkLarge-12100  11918236 
ns/op 7121221 B/op   9563 allocs/op
BenchmarkLargeWithExisting
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting-12 74  16180851 
ns/op 9605267 B/op  34015 allocs/op
BenchmarkLargeImbalanced
BenchmarkLargeImbalanced: sticky_test.go:1272: 24104 total partitions; 101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 24104 total partitions; 101 
total members
BenchmarkLargeImbalanced-12   68  17798614 
ns/op17025139 B/op   9995 allocs/op
BenchmarkLargeWithExistingImbalanced
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 24104 total 
partitions; 101 total members
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 24104 total 
partitions; 101 total members
BenchmarkLargeWithExistingImbalanced-12   74  15852596 
ns/op 9602434 B/op  33806 allocs/op
{noformat}

Switching up some numbers to better mirror this issue's problem statement:
{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 4274 total partitions; 2100 total 
members
BenchmarkLarge: sticky_test.go:1272: 4274 total partitions; 2100 total 
members
BenchmarkLarge: sticky_test.go:1272: 4274 total partitions; 2100 total 
members
BenchmarkLarge-12  3 447516434 
ns/op13942640 B/op  10619 allocs/op
BenchmarkLargeWithExisting
BenchmarkLargeWithExisting: sticky_test.go:1272: 4274 total partitions; 
2100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 4274 total partitions; 
2100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 4274 total partitions; 
2100 total members
BenchmarkLargeWithExisting-12  3 460263266 
ns/op14482474 B/op  27700 allocs/op
BenchmarkLargeImbalanced
BenchmarkLargeImbalanced: sticky_test.go:1272: 4274 total partitions; 2101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 4274 total partitions; 2101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 4274 total partitions; 2101 
total members
BenchmarkLargeImbalanced-123 487361276 
ns/op50107610 B/op  10636 allocs/op
BenchmarkLargeWithExistingImbalanced
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 4274 total 
partitions; 2101 total members
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 4274 total 
partitions; 2101 total members
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 4274 total 
partitions; 2101 total members
BenchmarkLargeWithExistingImbalanced-123 459259448 
ns/op14482096 B/op  27695 allocs/op
{noformat}

More extreme:

{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 1276057 total partitions; 1000 total 
members
BenchmarkLarge-12  11889004419 
ns/op430359568 B/op829830 allocs/op
BenchmarkLargeWithExisting
BenchmarkLargeWithExisting: sticky_test.go:1272: 1276057 total partitions; 
1000 total members
BenchmarkLargeWithExisting-12  13086791088 
ns/op617969240 B/op   2516550 allocs/op
BenchmarkLargeImbalanced
BenchmarkLargeImbalanced: sticky_test.go:1272: 1276057 total partitions; 
1001 total members
tBenchmarkLargeImbalanced-12   132948262382 
ns/op   5543028064 B/op   830336 allocs/op
BenchmarkLargeWithExistingImbalanced
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 1276057 total 
partitions; 1001 total members
BenchmarkLargeWithExistingImbalanced-1215206902130 
ns/op617954512 B/op   2515084 allocs/op
{noformat}

Note that the prior case uses quite a bit of RAM (~5-6G), but it also is 
balancing quite a lot of partitions among quite a lot of members; the actual 
planning itself only took ~0.5G, setup was the expensive part.

1 

[jira] [Commented] (KAFKA-9494) Include data type of the config in ConfigEntry

2020-05-31 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9494?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17120634#comment-17120634
 ] 

Travis Bischel commented on KAFKA-9494:
---

It may be beneficial to update the KIP to avoid mentioning IncludeType in the 
request, since that is no longer a thing.

> Include data type of the config in ConfigEntry
> --
>
> Key: KAFKA-9494
> URL: https://issues.apache.org/jira/browse/KAFKA-9494
> Project: Kafka
>  Issue Type: Improvement
>  Components: clients, core
>Reporter: Shailesh Panwar
>Priority: Minor
> Fix For: 2.6.0
>
>
> Why this request?
> To provide better validation. Including the data type can significantly 
> improve the validation on client side (be it web or cli or any other client). 
> In the absence of `type` the only way to know if the user specified value is 
> correct is to make an `alter` call and check if there is no error.
>  
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Comment Edited] (KAFKA-9987) Improve sticky partition assignor algorithm

2020-05-15 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9987?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17108741#comment-17108741
 ] 

Travis Bischel edited comment on KAFKA-9987 at 5/16/20, 1:22 AM:
-

Benchmark setup / code here: 
https://github.com/twmb/kafka-go/blob/3182a1fbce567ef8fbd95220027ce999a866557d/pkg/kgo/internal/sticky/sticky_test.go#L1237-L1291

Looks like (had to stare at this a bit) I setup a ton of topics, one per 
member, then I give each topic a random amount of partitions. For 
"WithExisting" benchmarks (prior plan), I encode the result of planning the 
initial setup, and then over each member, I give them what was just planned. So 
basically, all members are joining with the prior plan, and the new plan should 
be exactly the same. It's a bit weird, but all the work still needs to be done. 
You could enforce a difference by removing one member though, if you want.

The setup could maybe be a bit better w.r.t. number of topics, but from the 
code perspective, the number of topics doesn't matter, only the number of 
partitions.

Although, thinking about it, the setup itself doesn't really stress the simple 
algorithm. Manually changing some of the code (removing some user data to 
simluate a new member giving that member's prior partitions to other members, 
or adding completely new members, or removing members to simulate leaving) show 
that these standard operations (one leaving or one joining at a time) do not 
meaningfully impact the benchmarks in any noteworthy way. I'd be interested to 
see what you come up with in benchmarks so that I can port them to my own as 
well.


was (Author: twmb):
Benchmark setup / code here: 
https://github.com/twmb/kafka-go/blob/3182a1fbce567ef8fbd95220027ce999a866557d/pkg/kgo/internal/sticky/sticky_test.go#L1237-L1291

Looks like (had to stare at this a bit) I setup a ton of topics, one per 
member, then I give each topic a random amount of partitions. For 
"WithExisting" benchmarks (prior plan), I encode the result of planning the 
initial setup, and then over each member, I give them what was just planned. So 
basically, all members are joining with the prior plan, and the new plan should 
be exactly the same. It's a bit weird, but all the work still needs to be done. 
You could enforce a difference by removing one member though, if you want.

The setup could maybe be a bit better w.r.t. number of topics, but from the 
code perspective, the number of topics doesn't matter, only the number of 
partitions.

> Improve sticky partition assignor algorithm
> ---
>
> Key: KAFKA-9987
> URL: https://issues.apache.org/jira/browse/KAFKA-9987
> Project: Kafka
>  Issue Type: Improvement
>  Components: clients
>Reporter: Sophie Blee-Goldman
>Assignee: Sophie Blee-Goldman
>Priority: Major
>
> In 
> [KIP-429|https://cwiki.apache.org/confluence/display/KAFKA/KIP-429%3A+Kafka+Consumer+Incremental+Rebalance+Protocol]
>  we added the new CooperativeStickyAssignor which leverages on the underlying 
> sticky assignment algorithm of the existing StickyAssignor (moved to 
> AbstractStickyAssignor). The algorithm is fairly complex as it tries to 
> optimize stickiness while satisfying perfect balance _in the case individual 
> consumers may be subscribed to different subsets of the topics._ While it 
> does a pretty good job at what it promises to do, it doesn't scale well with 
> large numbers of consumers and partitions.
> To give a concrete example, users have reported that it takes 2.5 minutes for 
> the assignment to complete with just 2100 consumers reading from 2100 
> partitions. Since partitions revoked during the first of two cooperative 
> rebalances will remain unassigned until the end of the second rebalance, it's 
> important for the rebalance to be as fast as possible. And since one of the 
> primary improvements of the cooperative rebalancing protocol is better 
> scaling experience, the only OOTB cooperative assignor should not itself 
> scale poorly
> If we can constrain the problem a bit, we can simplify the algorithm greatly. 
> In many cases the individual consumers won't be subscribed to some random 
> subset of the total subscription, they will all be subscribed to the same set 
> of topics and rely on the assignor to balance the partition workload.
> We can detect this case by checking the group's individual subscriptions and 
> call on a more efficient assignment algorithm. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9987) Improve sticky partition assignor algorithm

2020-05-15 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9987?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17108741#comment-17108741
 ] 

Travis Bischel commented on KAFKA-9987:
---

Benchmark setup / code here: 
https://github.com/twmb/kafka-go/blob/3182a1fbce567ef8fbd95220027ce999a866557d/pkg/kgo/internal/sticky/sticky_test.go#L1237-L1291

Looks like (had to stare at this a bit) I setup a ton of topics, one per 
member, then I give each topic a random amount of partitions. For 
"WithExisting" benchmarks (prior plan), I encode the result of planning the 
initial setup, and then over each member, I give them what was just planned. So 
basically, all members are joining with the prior plan, and the new plan should 
be exactly the same. It's a bit weird, but all the work still needs to be done. 
You could enforce a difference by removing one member though, if you want.

The setup could maybe be a bit better w.r.t. number of topics, but from the 
code perspective, the number of topics doesn't matter, only the number of 
partitions.

> Improve sticky partition assignor algorithm
> ---
>
> Key: KAFKA-9987
> URL: https://issues.apache.org/jira/browse/KAFKA-9987
> Project: Kafka
>  Issue Type: Improvement
>  Components: clients
>Reporter: Sophie Blee-Goldman
>Assignee: Sophie Blee-Goldman
>Priority: Major
>
> In 
> [KIP-429|https://cwiki.apache.org/confluence/display/KAFKA/KIP-429%3A+Kafka+Consumer+Incremental+Rebalance+Protocol]
>  we added the new CooperativeStickyAssignor which leverages on the underlying 
> sticky assignment algorithm of the existing StickyAssignor (moved to 
> AbstractStickyAssignor). The algorithm is fairly complex as it tries to 
> optimize stickiness while satisfying perfect balance _in the case individual 
> consumers may be subscribed to different subsets of the topics._ While it 
> does a pretty good job at what it promises to do, it doesn't scale well with 
> large numbers of consumers and partitions.
> To give a concrete example, users have reported that it takes 2.5 minutes for 
> the assignment to complete with just 2100 consumers reading from 2100 
> partitions. Since partitions revoked during the first of two cooperative 
> rebalances will remain unassigned until the end of the second rebalance, it's 
> important for the rebalance to be as fast as possible. And since one of the 
> primary improvements of the cooperative rebalancing protocol is better 
> scaling experience, the only OOTB cooperative assignor should not itself 
> scale poorly
> If we can constrain the problem a bit, we can simplify the algorithm greatly. 
> In many cases the individual consumers won't be subscribed to some random 
> subset of the total subscription, they will all be subscribed to the same set 
> of topics and rely on the assignor to balance the partition workload.
> We can detect this case by checking the group's individual subscriptions and 
> call on a more efficient assignment algorithm. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Comment Edited] (KAFKA-9987) Improve sticky partition assignor algorithm

2020-05-15 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9987?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17108720#comment-17108720
 ] 

Travis Bischel edited comment on KAFKA-9987 at 5/15/20, 11:06 PM:
--

Yes, definitely seems the common case. For context, here's my specific loop 
that rebalances in that case: 
https://github.com/twmb/kafka-go/blob/3182a1fbce567ef8fbd95220027ce999a866557d/pkg/kgo/internal/sticky/sticky.go#L555-L600

In terms of memory, I only mean that the extremely large case uses a good bit 
of RAM, but this truly is the extremely large case. For the 2100 member / 4274 
partitions case, my code uses 13MiB. I'm not sure what more memory savings 
exist; I spent a fair bit of time optimizing memory and speed, but this was 
last August/September.


was (Author: twmb):
Yes, definitely seems the common case. For context, here's my specific loop 
that rebalances in that case: 
https://github.com/twmb/kafka-go/blob/3182a1fbce567ef8fbd95220027ce999a866557d/pkg/kgo/internal/sticky/sticky.go#L555-L600

In terms of memory, I only mean that the extremely large case uses a good bit 
of RAM, but this truly is the extremely large case. For the 2100 member / 4274 
partitions case, my code uses 893.4m. I'm not sure what more memory savings 
exist; I spent a fair bit of time optimizing memory and speed, but this was 
last August/September.

> Improve sticky partition assignor algorithm
> ---
>
> Key: KAFKA-9987
> URL: https://issues.apache.org/jira/browse/KAFKA-9987
> Project: Kafka
>  Issue Type: Improvement
>  Components: clients
>Reporter: Sophie Blee-Goldman
>Assignee: Sophie Blee-Goldman
>Priority: Major
>
> In 
> [KIP-429|https://cwiki.apache.org/confluence/display/KAFKA/KIP-429%3A+Kafka+Consumer+Incremental+Rebalance+Protocol]
>  we added the new CooperativeStickyAssignor which leverages on the underlying 
> sticky assignment algorithm of the existing StickyAssignor (moved to 
> AbstractStickyAssignor). The algorithm is fairly complex as it tries to 
> optimize stickiness while satisfying perfect balance _in the case individual 
> consumers may be subscribed to different subsets of the topics._ While it 
> does a pretty good job at what it promises to do, it doesn't scale well with 
> large numbers of consumers and partitions.
> To give a concrete example, users have reported that it takes 2.5 minutes for 
> the assignment to complete with just 2100 consumers reading from 2100 
> partitions. Since partitions revoked during the first of two cooperative 
> rebalances will remain unassigned until the end of the second rebalance, it's 
> important for the rebalance to be as fast as possible. And since one of the 
> primary improvements of the cooperative rebalancing protocol is better 
> scaling experience, the only OOTB cooperative assignor should not itself 
> scale poorly
> If we can constrain the problem a bit, we can simplify the algorithm greatly. 
> In many cases the individual consumers won't be subscribed to some random 
> subset of the total subscription, they will all be subscribed to the same set 
> of topics and rely on the assignor to balance the partition workload.
> We can detect this case by checking the group's individual subscriptions and 
> call on a more efficient assignment algorithm. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Comment Edited] (KAFKA-9987) Improve sticky partition assignor algorithm

2020-05-15 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9987?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17108705#comment-17108705
 ] 

Travis Bischel edited comment on KAFKA-9987 at 5/15/20, 11:06 PM:
--

For context, here's my current benchmarks (WithExisting mirrors an existing 
cluster rejoining, Imbalanced means unequal subscriptions):

{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 24104 total partitions; 100 total 
members
BenchmarkLarge: sticky_test.go:1272: 24104 total partitions; 100 total 
members
BenchmarkLarge-12100  11918236 
ns/op 7121221 B/op   9563 allocs/op
BenchmarkLargeWithExisting
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting-12 74  16180851 
ns/op 9605267 B/op  34015 allocs/op
BenchmarkLargeImbalanced
BenchmarkLargeImbalanced: sticky_test.go:1272: 24104 total partitions; 101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 24104 total partitions; 101 
total members
BenchmarkLargeImbalanced-12   68  17798614 
ns/op17025139 B/op   9995 allocs/op
BenchmarkLargeWithExistingImbalanced
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 24104 total 
partitions; 101 total members
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 24104 total 
partitions; 101 total members
BenchmarkLargeWithExistingImbalanced-12   74  15852596 
ns/op 9602434 B/op  33806 allocs/op
{noformat}

Switching up some numbers to better mirror this issue's problem statement:
{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 4274 total partitions; 2100 total 
members
BenchmarkLarge: sticky_test.go:1272: 4274 total partitions; 2100 total 
members
BenchmarkLarge: sticky_test.go:1272: 4274 total partitions; 2100 total 
members
BenchmarkLarge-12  3 447516434 
ns/op13942640 B/op  10619 allocs/op
BenchmarkLargeWithExisting
BenchmarkLargeWithExisting: sticky_test.go:1272: 4274 total partitions; 
2100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 4274 total partitions; 
2100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 4274 total partitions; 
2100 total members
BenchmarkLargeWithExisting-12  3 460263266 
ns/op14482474 B/op  27700 allocs/op
BenchmarkLargeImbalanced
BenchmarkLargeImbalanced: sticky_test.go:1272: 4274 total partitions; 2101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 4274 total partitions; 2101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 4274 total partitions; 2101 
total members
BenchmarkLargeImbalanced-123 487361276 
ns/op50107610 B/op  10636 allocs/op
BenchmarkLargeWithExistingImbalanced
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 4274 total 
partitions; 2101 total members
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 4274 total 
partitions; 2101 total members
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 4274 total 
partitions; 2101 total members
BenchmarkLargeWithExistingImbalanced-123 459259448 
ns/op14482096 B/op  27695 allocs/op
{noformat}

More extreme:

{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 1276057 total partitions; 1000 total 
members
BenchmarkLarge-12  11889004419 
ns/op430359568 B/op829830 allocs/op
BenchmarkLargeWithExisting
BenchmarkLargeWithExisting: sticky_test.go:1272: 1276057 total partitions; 
1000 total members
BenchmarkLargeWithExisting-12  13086791088 
ns/op617969240 B/op   2516550 allocs/op
BenchmarkLargeImbalanced
BenchmarkLargeImbalanced: sticky_test.go:1272: 1276057 total partitions; 
1001 total members
tBenchmarkLargeImbalanced-12   132948262382 
ns/op   5543028064 B/op   830336 allocs/op
BenchmarkLargeWithExistingImbalanced
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 1276057 total 
partitions; 1001 total members
BenchmarkLargeWithExistingImbalanced-1215206902130 
ns/op617954512 B/op   2515084 allocs/op
{noformat}

Note that the prior case uses quite a bit of RAM (~5-6G), but it also is 
balancing quite a lot of partitions among quite a lot of members.

1 topic, 2100 partitions, 2100 members
{noformat}
BenchmarkLargeWithExisting-12 

[jira] [Comment Edited] (KAFKA-9987) Improve sticky partition assignor algorithm

2020-05-15 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9987?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17108705#comment-17108705
 ] 

Travis Bischel edited comment on KAFKA-9987 at 5/15/20, 10:50 PM:
--

For context, here's my current benchmarks (WithExisting mirrors an existing 
cluster rejoining, Imbalanced means unequal subscriptions):

{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 24104 total partitions; 100 total 
members
BenchmarkLarge: sticky_test.go:1272: 24104 total partitions; 100 total 
members
BenchmarkLarge-12100  11918236 
ns/op 7121221 B/op   9563 allocs/op
BenchmarkLargeWithExisting
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting-12 74  16180851 
ns/op 9605267 B/op  34015 allocs/op
BenchmarkLargeImbalanced
BenchmarkLargeImbalanced: sticky_test.go:1272: 24104 total partitions; 101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 24104 total partitions; 101 
total members
BenchmarkLargeImbalanced-12   68  17798614 
ns/op17025139 B/op   9995 allocs/op
BenchmarkLargeWithExistingImbalanced
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 24104 total 
partitions; 101 total members
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 24104 total 
partitions; 101 total members
BenchmarkLargeWithExistingImbalanced-12   74  15852596 
ns/op 9602434 B/op  33806 allocs/op
{noformat}

Switching up some numbers to better mirror this issue's problem statement:
{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 4274 total partitions; 2100 total 
members
BenchmarkLarge: sticky_test.go:1272: 4274 total partitions; 2100 total 
members
BenchmarkLarge: sticky_test.go:1272: 4274 total partitions; 2100 total 
members
BenchmarkLarge-12  3 447516434 
ns/op13942640 B/op  10619 allocs/op
BenchmarkLargeWithExisting
BenchmarkLargeWithExisting: sticky_test.go:1272: 4274 total partitions; 
2100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 4274 total partitions; 
2100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 4274 total partitions; 
2100 total members
BenchmarkLargeWithExisting-12  3 460263266 
ns/op14482474 B/op  27700 allocs/op
BenchmarkLargeImbalanced
BenchmarkLargeImbalanced: sticky_test.go:1272: 4274 total partitions; 2101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 4274 total partitions; 2101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 4274 total partitions; 2101 
total members
BenchmarkLargeImbalanced-123 487361276 
ns/op50107610 B/op  10636 allocs/op
BenchmarkLargeWithExistingImbalanced
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 4274 total 
partitions; 2101 total members
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 4274 total 
partitions; 2101 total members
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 4274 total 
partitions; 2101 total members
BenchmarkLargeWithExistingImbalanced-123 459259448 
ns/op14482096 B/op  27695 allocs/op
{noformat}

More extreme:

{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 1276057 total partitions; 1000 total 
members
BenchmarkLarge-12  11889004419 
ns/op430359568 B/op829830 allocs/op
BenchmarkLargeWithExisting
BenchmarkLargeWithExisting: sticky_test.go:1272: 1276057 total partitions; 
1000 total members
BenchmarkLargeWithExisting-12  13086791088 
ns/op617969240 B/op   2516550 allocs/op
BenchmarkLargeImbalanced
BenchmarkLargeImbalanced: sticky_test.go:1272: 1276057 total partitions; 
1001 total members
tBenchmarkLargeImbalanced-12   132948262382 
ns/op   5543028064 B/op   830336 allocs/op
BenchmarkLargeWithExistingImbalanced
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 1276057 total 
partitions; 1001 total members
BenchmarkLargeWithExistingImbalanced-1215206902130 
ns/op617954512 B/op   2515084 allocs/op
{noformat}

Note that the prior case uses quite a bit of RAM (~5-6G), but it also is 
balancing quite a lot of partitions among quite a lot of members.

1 topic, 2100 partitions, 2100 members (this uses 691.6m of memory)
{noformat}

[jira] [Comment Edited] (KAFKA-9987) Improve sticky partition assignor algorithm

2020-05-15 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9987?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17108720#comment-17108720
 ] 

Travis Bischel edited comment on KAFKA-9987 at 5/15/20, 10:42 PM:
--

Yes, definitely seems the common case. For context, here's my specific loop 
that rebalances in that case: 
https://github.com/twmb/kafka-go/blob/3182a1fbce567ef8fbd95220027ce999a866557d/pkg/kgo/internal/sticky/sticky.go#L555-L600

In terms of memory, I only mean that the extremely large case uses a good bit 
of RAM, but this truly is the extremely large case. For the 2100 member / 4274 
partitions case, my code uses 893.4m. I'm not sure what more memory savings 
exist; I spent a fair bit of time optimizing memory and speed, but this was 
last August/September.


was (Author: twmb):
Yes, definitely seems the common case. For context, here's my specific loop 
that rebalances in that case: 
https://github.com/twmb/kafka-go/blob/3182a1fbce567ef8fbd95220027ce999a866557d/pkg/kgo/internal/sticky/sticky.go#L555-L600

In terms of memory, I only mean that the extremely large case uses a good bit 
of RAM, but this truly is the extremely large case. For the 2100 member / 4274 
partitions case, my code uses 893.4m. I'm not sure what more memory savings 
exist; I spent a fair bit of time optimizing memory and speed.

> Improve sticky partition assignor algorithm
> ---
>
> Key: KAFKA-9987
> URL: https://issues.apache.org/jira/browse/KAFKA-9987
> Project: Kafka
>  Issue Type: Improvement
>  Components: clients
>Reporter: Sophie Blee-Goldman
>Assignee: Sophie Blee-Goldman
>Priority: Major
>
> In 
> [KIP-429|https://cwiki.apache.org/confluence/display/KAFKA/KIP-429%3A+Kafka+Consumer+Incremental+Rebalance+Protocol]
>  we added the new CooperativeStickyAssignor which leverages on the underlying 
> sticky assignment algorithm of the existing StickyAssignor (moved to 
> AbstractStickyAssignor). The algorithm is fairly complex as it tries to 
> optimize stickiness while satisfying perfect balance _in the case individual 
> consumers may be subscribed to different subsets of the topics._ While it 
> does a pretty good job at what it promises to do, it doesn't scale well with 
> large numbers of consumers and partitions.
> To give a concrete example, users have reported that it takes 2.5 minutes for 
> the assignment to complete with just 2100 consumers reading from 2100 
> partitions. Since partitions revoked during the first of two cooperative 
> rebalances will remain unassigned until the end of the second rebalance, it's 
> important for the rebalance to be as fast as possible. And since one of the 
> primary improvements of the cooperative rebalancing protocol is better 
> scaling experience, the only OOTB cooperative assignor should not itself 
> scale poorly
> If we can constrain the problem a bit, we can simplify the algorithm greatly. 
> In many cases the individual consumers won't be subscribed to some random 
> subset of the total subscription, they will all be subscribed to the same set 
> of topics and rely on the assignor to balance the partition workload.
> We can detect this case by checking the group's individual subscriptions and 
> call on a more efficient assignment algorithm. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9987) Improve sticky partition assignor algorithm

2020-05-15 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9987?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17108720#comment-17108720
 ] 

Travis Bischel commented on KAFKA-9987:
---

Yes, definitely seems the common case. For context, here's my specific loop 
that rebalances in that case: 
https://github.com/twmb/kafka-go/blob/3182a1fbce567ef8fbd95220027ce999a866557d/pkg/kgo/internal/sticky/sticky.go#L555-L600

In terms of memory, I only mean that the extremely large case uses a good bit 
of RAM, but this truly is the extremely large case. For the 2100 member / 4274 
partitions case, my code uses 893.4m. I'm not sure what more memory savings 
exist; I spent a fair bit of time optimizing memory and speed.

> Improve sticky partition assignor algorithm
> ---
>
> Key: KAFKA-9987
> URL: https://issues.apache.org/jira/browse/KAFKA-9987
> Project: Kafka
>  Issue Type: Improvement
>  Components: clients
>Reporter: Sophie Blee-Goldman
>Assignee: Sophie Blee-Goldman
>Priority: Major
>
> In 
> [KIP-429|https://cwiki.apache.org/confluence/display/KAFKA/KIP-429%3A+Kafka+Consumer+Incremental+Rebalance+Protocol]
>  we added the new CooperativeStickyAssignor which leverages on the underlying 
> sticky assignment algorithm of the existing StickyAssignor (moved to 
> AbstractStickyAssignor). The algorithm is fairly complex as it tries to 
> optimize stickiness while satisfying perfect balance _in the case individual 
> consumers may be subscribed to different subsets of the topics._ While it 
> does a pretty good job at what it promises to do, it doesn't scale well with 
> large numbers of consumers and partitions.
> To give a concrete example, users have reported that it takes 2.5 minutes for 
> the assignment to complete with just 2100 consumers reading from 2100 
> partitions. Since partitions revoked during the first of two cooperative 
> rebalances will remain unassigned until the end of the second rebalance, it's 
> important for the rebalance to be as fast as possible. And since one of the 
> primary improvements of the cooperative rebalancing protocol is better 
> scaling experience, the only OOTB cooperative assignor should not itself 
> scale poorly
> If we can constrain the problem a bit, we can simplify the algorithm greatly. 
> In many cases the individual consumers won't be subscribed to some random 
> subset of the total subscription, they will all be subscribed to the same set 
> of topics and rely on the assignor to balance the partition workload.
> We can detect this case by checking the group's individual subscriptions and 
> call on a more efficient assignment algorithm. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Comment Edited] (KAFKA-9987) Improve sticky partition assignor algorithm

2020-05-15 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9987?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17108705#comment-17108705
 ] 

Travis Bischel edited comment on KAFKA-9987 at 5/15/20, 10:27 PM:
--

For context, here's my current benchmarks (WithExisting mirrors an existing 
cluster rejoining, Imbalanced means unequal subscriptions):

{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 24104 total partitions; 100 total 
members
BenchmarkLarge: sticky_test.go:1272: 24104 total partitions; 100 total 
members
BenchmarkLarge-12100  11918236 
ns/op 7121221 B/op   9563 allocs/op
BenchmarkLargeWithExisting
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting-12 74  16180851 
ns/op 9605267 B/op  34015 allocs/op
BenchmarkLargeImbalanced
BenchmarkLargeImbalanced: sticky_test.go:1272: 24104 total partitions; 101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 24104 total partitions; 101 
total members
BenchmarkLargeImbalanced-12   68  17798614 
ns/op17025139 B/op   9995 allocs/op
BenchmarkLargeWithExistingImbalanced
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 24104 total 
partitions; 101 total members
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 24104 total 
partitions; 101 total members
BenchmarkLargeWithExistingImbalanced-12   74  15852596 
ns/op 9602434 B/op  33806 allocs/op
{noformat}

Switching up some numbers to better mirror this issue's problem statement:
{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 4274 total partitions; 2100 total 
members
BenchmarkLarge: sticky_test.go:1272: 4274 total partitions; 2100 total 
members
BenchmarkLarge: sticky_test.go:1272: 4274 total partitions; 2100 total 
members
BenchmarkLarge-12  3 447516434 
ns/op13942640 B/op  10619 allocs/op
BenchmarkLargeWithExisting
BenchmarkLargeWithExisting: sticky_test.go:1272: 4274 total partitions; 
2100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 4274 total partitions; 
2100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 4274 total partitions; 
2100 total members
BenchmarkLargeWithExisting-12  3 460263266 
ns/op14482474 B/op  27700 allocs/op
BenchmarkLargeImbalanced
BenchmarkLargeImbalanced: sticky_test.go:1272: 4274 total partitions; 2101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 4274 total partitions; 2101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 4274 total partitions; 2101 
total members
BenchmarkLargeImbalanced-123 487361276 
ns/op50107610 B/op  10636 allocs/op
BenchmarkLargeWithExistingImbalanced
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 4274 total 
partitions; 2101 total members
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 4274 total 
partitions; 2101 total members
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 4274 total 
partitions; 2101 total members
BenchmarkLargeWithExistingImbalanced-123 459259448 
ns/op14482096 B/op  27695 allocs/op
{noformat}

More extreme:

{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 1276057 total partitions; 1000 total 
members
BenchmarkLarge-12  11889004419 
ns/op430359568 B/op829830 allocs/op
BenchmarkLargeWithExisting
BenchmarkLargeWithExisting: sticky_test.go:1272: 1276057 total partitions; 
1000 total members
BenchmarkLargeWithExisting-12  13086791088 
ns/op617969240 B/op   2516550 allocs/op
BenchmarkLargeImbalanced
BenchmarkLargeImbalanced: sticky_test.go:1272: 1276057 total partitions; 
1001 total members
tBenchmarkLargeImbalanced-12   132948262382 
ns/op   5543028064 B/op   830336 allocs/op
BenchmarkLargeWithExistingImbalanced
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 1276057 total 
partitions; 1001 total members
BenchmarkLargeWithExistingImbalanced-1215206902130 
ns/op617954512 B/op   2515084 allocs/op
{noformat}

Note that this last case uses quite a bit of RAM (~5-6G), but it also is 
balancing quite a lot of partitions among quite a lot of members.


was (Author: twmb):
For context, here's my current benchmarks:
{noformat}

[jira] [Comment Edited] (KAFKA-9987) Improve sticky partition assignor algorithm

2020-05-15 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9987?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17108705#comment-17108705
 ] 

Travis Bischel edited comment on KAFKA-9987 at 5/15/20, 10:25 PM:
--

For context, here's my current benchmarks:
{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 24104 total partitions; 100 total 
members
BenchmarkLarge: sticky_test.go:1272: 24104 total partitions; 100 total 
members
BenchmarkLarge-12100  11918236 
ns/op 7121221 B/op   9563 allocs/op
BenchmarkLargeWithExisting
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting-12 74  16180851 
ns/op 9605267 B/op  34015 allocs/op
BenchmarkLargeImbalanced
BenchmarkLargeImbalanced: sticky_test.go:1272: 24104 total partitions; 101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 24104 total partitions; 101 
total members
BenchmarkLargeImbalanced-12   68  17798614 
ns/op17025139 B/op   9995 allocs/op
BenchmarkLargeWithExistingImbalanced
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 24104 total 
partitions; 101 total members
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 24104 total 
partitions; 101 total members
BenchmarkLargeWithExistingImbalanced-12   74  15852596 
ns/op 9602434 B/op  33806 allocs/op
{noformat}

Switching up some numbers to better mirror this issue's problem statement:
{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 4274 total partitions; 2100 total 
members
BenchmarkLarge: sticky_test.go:1272: 4274 total partitions; 2100 total 
members
BenchmarkLarge: sticky_test.go:1272: 4274 total partitions; 2100 total 
members
BenchmarkLarge-12  3 447516434 
ns/op13942640 B/op  10619 allocs/op
BenchmarkLargeWithExisting
BenchmarkLargeWithExisting: sticky_test.go:1272: 4274 total partitions; 
2100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 4274 total partitions; 
2100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 4274 total partitions; 
2100 total members
BenchmarkLargeWithExisting-12  3 460263266 
ns/op14482474 B/op  27700 allocs/op
BenchmarkLargeImbalanced
BenchmarkLargeImbalanced: sticky_test.go:1272: 4274 total partitions; 2101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 4274 total partitions; 2101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 4274 total partitions; 2101 
total members
BenchmarkLargeImbalanced-123 487361276 
ns/op50107610 B/op  10636 allocs/op
BenchmarkLargeWithExistingImbalanced
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 4274 total 
partitions; 2101 total members
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 4274 total 
partitions; 2101 total members
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 4274 total 
partitions; 2101 total members
BenchmarkLargeWithExistingImbalanced-123 459259448 
ns/op14482096 B/op  27695 allocs/op
{noformat}

{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 1276057 total partitions; 1000 total 
members
BenchmarkLarge-12  11889004419 
ns/op430359568 B/op829830 allocs/op
BenchmarkLargeWithExisting
BenchmarkLargeWithExisting: sticky_test.go:1272: 1276057 total partitions; 
1000 total members
BenchmarkLargeWithExisting-12  13086791088 
ns/op617969240 B/op   2516550 allocs/op
BenchmarkLargeImbalanced
BenchmarkLargeImbalanced: sticky_test.go:1272: 1276057 total partitions; 
1001 total members
tBenchmarkLargeImbalanced-12   132948262382 
ns/op   5543028064 B/op   830336 allocs/op
BenchmarkLargeWithExistingImbalanced
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 1276057 total 
partitions; 1001 total members
BenchmarkLargeWithExistingImbalanced-1215206902130 
ns/op617954512 B/op   2515084 allocs/op
{noformat}

Note that this last case uses quite a bit of RAM (~5-6G), but it also is 
balancing quite a lot of partitions among quite a lot of members.


was (Author: twmb):
For context, here's my current benchmarks:
{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 24104 total partitions; 100 total 
members

[jira] [Commented] (KAFKA-9987) Improve sticky partition assignor algorithm

2020-05-15 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9987?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17108712#comment-17108712
 ] 

Travis Bischel commented on KAFKA-9987:
---

Unless I'm mistaken, the algorithm in the first comment should be relatively 
optimal in cases where the subscriptions of all members are equal, but _I 
think_ has some edge cases with unequal subscriptions. This is (I think) 
implied in the ticket description.

My code solves both cases by switching to a more complex algorithm (A*) to find 
steal paths on unequal subscriptions, but sticks with a very simple algorithm 
on equal subscription cases. The simple algorithm just moves from partitions 
from the most loaded member to the least loaded until the partition delta 
between the most and least is at most one.

> Improve sticky partition assignor algorithm
> ---
>
> Key: KAFKA-9987
> URL: https://issues.apache.org/jira/browse/KAFKA-9987
> Project: Kafka
>  Issue Type: Improvement
>  Components: clients
>Reporter: Sophie Blee-Goldman
>Assignee: Sophie Blee-Goldman
>Priority: Major
>
> In 
> [KIP-429|https://cwiki.apache.org/confluence/display/KAFKA/KIP-429%3A+Kafka+Consumer+Incremental+Rebalance+Protocol]
>  we added the new CooperativeStickyAssignor which leverages on the underlying 
> sticky assignment algorithm of the existing StickyAssignor (moved to 
> AbstractStickyAssignor). The algorithm is fairly complex as it tries to 
> optimize stickiness while satisfying perfect balance _in the case individual 
> consumers may be subscribed to different subsets of the topics._ While it 
> does a pretty good job at what it promises to do, it doesn't scale well with 
> large numbers of consumers and partitions.
> To give a concrete example, users have reported that it takes 2.5 minutes for 
> the assignment to complete with just 2100 consumers reading from 2100 
> partitions. Since partitions revoked during the first of two cooperative 
> rebalances will remain unassigned until the end of the second rebalance, it's 
> important for the rebalance to be as fast as possible. And since one of the 
> primary improvements of the cooperative rebalancing protocol is better 
> scaling experience, the only OOTB cooperative assignor should not itself 
> scale poorly
> If we can constrain the problem a bit, we can simplify the algorithm greatly. 
> In many cases the individual consumers won't be subscribed to some random 
> subset of the total subscription, they will all be subscribed to the same set 
> of topics and rely on the assignor to balance the partition workload.
> We can detect this case by checking the group's individual subscriptions and 
> call on a more efficient assignment algorithm. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Comment Edited] (KAFKA-9987) Improve sticky partition assignor algorithm

2020-05-15 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9987?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17108705#comment-17108705
 ] 

Travis Bischel edited comment on KAFKA-9987 at 5/15/20, 10:21 PM:
--

For context, here's my current benchmarks:
{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 24104 total partitions; 100 total 
members
BenchmarkLarge: sticky_test.go:1272: 24104 total partitions; 100 total 
members
BenchmarkLarge-12100  11918236 
ns/op 7121221 B/op   9563 allocs/op
BenchmarkLargeWithExisting
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting-12 74  16180851 
ns/op 9605267 B/op  34015 allocs/op
BenchmarkLargeImbalanced
BenchmarkLargeImbalanced: sticky_test.go:1272: 24104 total partitions; 101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 24104 total partitions; 101 
total members
BenchmarkLargeImbalanced-12   68  17798614 
ns/op17025139 B/op   9995 allocs/op
BenchmarkLargeWithExistingImbalanced
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 24104 total 
partitions; 101 total members
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 24104 total 
partitions; 101 total members
BenchmarkLargeWithExistingImbalanced-12   74  15852596 
ns/op 9602434 B/op  33806 allocs/op
{noformat}

Switching up some numbers to better mirror this issue's problem statement:
{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 4274 total partitions; 2100 total 
members
BenchmarkLarge: sticky_test.go:1272: 4274 total partitions; 2100 total 
members
BenchmarkLarge: sticky_test.go:1272: 4274 total partitions; 2100 total 
members
BenchmarkLarge-12  3 447516434 
ns/op13942640 B/op  10619 allocs/op
BenchmarkLargeWithExisting
BenchmarkLargeWithExisting: sticky_test.go:1272: 4274 total partitions; 
2100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 4274 total partitions; 
2100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 4274 total partitions; 
2100 total members
BenchmarkLargeWithExisting-12  3 460263266 
ns/op14482474 B/op  27700 allocs/op
BenchmarkLargeImbalanced
BenchmarkLargeImbalanced: sticky_test.go:1272: 4274 total partitions; 2101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 4274 total partitions; 2101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 4274 total partitions; 2101 
total members
BenchmarkLargeImbalanced-123 487361276 
ns/op50107610 B/op  10636 allocs/op
BenchmarkLargeWithExistingImbalanced
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 4274 total 
partitions; 2101 total members
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 4274 total 
partitions; 2101 total members
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 4274 total 
partitions; 2101 total members
BenchmarkLargeWithExistingImbalanced-123 459259448 
ns/op14482096 B/op  27695 allocs/op
{noformat}


was (Author: twmb):
For context, here's my current benchmarks:
{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 24104 total partitions; 100 total 
members
BenchmarkLarge: sticky_test.go:1272: 24104 total partitions; 100 total 
members
BenchmarkLarge-12100  11918236 
ns/op 7121221 B/op   9563 allocs/op
BenchmarkLargeWithExisting
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting-12 74  16180851 
ns/op 9605267 B/op  34015 allocs/op
BenchmarkLargeImbalanced
BenchmarkLargeImbalanced: sticky_test.go:1272: 24104 total partitions; 101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 24104 total partitions; 101 
total members
BenchmarkLargeImbalanced-12   68  17798614 
ns/op17025139 B/op   9995 allocs/op
BenchmarkLargeWithExistingImbalanced
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 24104 total 
partitions; 101 total members
BenchmarkLargeWithExistingImbalanced: 

[jira] [Commented] (KAFKA-9987) Improve sticky partition assignor algorithm

2020-05-15 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9987?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17108705#comment-17108705
 ] 

Travis Bischel commented on KAFKA-9987:
---

For context, here's my current benchmarks:
{noformat}
BenchmarkLarge
BenchmarkLarge: sticky_test.go:1272: 24104 total partitions; 100 total 
members
BenchmarkLarge: sticky_test.go:1272: 24104 total partitions; 100 total 
members
BenchmarkLarge-12100  11918236 
ns/op 7121221 B/op   9563 allocs/op
BenchmarkLargeWithExisting
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting: sticky_test.go:1272: 24104 total partitions; 
100 total members
BenchmarkLargeWithExisting-12 74  16180851 
ns/op 9605267 B/op  34015 allocs/op
BenchmarkLargeImbalanced
BenchmarkLargeImbalanced: sticky_test.go:1272: 24104 total partitions; 101 
total members
BenchmarkLargeImbalanced: sticky_test.go:1272: 24104 total partitions; 101 
total members
BenchmarkLargeImbalanced-12   68  17798614 
ns/op17025139 B/op   9995 allocs/op
BenchmarkLargeWithExistingImbalanced
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 24104 total 
partitions; 101 total members
BenchmarkLargeWithExistingImbalanced: sticky_test.go:1272: 24104 total 
partitions; 101 total members
BenchmarkLargeWithExistingImbalanced-12   74  15852596 
ns/op 9602434 B/op  33806 allocs/op
{noformat}

> Improve sticky partition assignor algorithm
> ---
>
> Key: KAFKA-9987
> URL: https://issues.apache.org/jira/browse/KAFKA-9987
> Project: Kafka
>  Issue Type: Improvement
>  Components: clients
>Reporter: Sophie Blee-Goldman
>Assignee: Sophie Blee-Goldman
>Priority: Major
>
> In 
> [KIP-429|https://cwiki.apache.org/confluence/display/KAFKA/KIP-429%3A+Kafka+Consumer+Incremental+Rebalance+Protocol]
>  we added the new CooperativeStickyAssignor which leverages on the underlying 
> sticky assignment algorithm of the existing StickyAssignor (moved to 
> AbstractStickyAssignor). The algorithm is fairly complex as it tries to 
> optimize stickiness while satisfying perfect balance _in the case individual 
> consumers may be subscribed to different subsets of the topics._ While it 
> does a pretty good job at what it promises to do, it doesn't scale well with 
> large numbers of consumers and partitions.
> To give a concrete example, users have reported that it takes 2.5 minutes for 
> the assignment to complete with just 2100 consumers reading from 2100 
> partitions. Since partitions revoked during the first of two cooperative 
> rebalances will remain unassigned until the end of the second rebalance, it's 
> important for the rebalance to be as fast as possible. And since one of the 
> primary improvements of the cooperative rebalancing protocol is better 
> scaling experience, the only OOTB cooperative assignor should not itself 
> scale poorly
> If we can constrain the problem a bit, we can simplify the algorithm greatly. 
> In many cases the individual consumers won't be subscribed to some random 
> subset of the total subscription, they will all be subscribed to the same set 
> of topics and rely on the assignor to balance the partition workload.
> We can detect this case by checking the group's individual subscriptions and 
> call on a more efficient assignment algorithm. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9987) Improve sticky partition assignor algorithm

2020-05-15 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9987?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17108704#comment-17108704
 ] 

Travis Bischel commented on KAFKA-9987:
---

Hi, I implemented a much faster sticky algorithm that (I believe) also optimal 
a year ago (whereas the current is slow [as pointed out here] and non-optimal).

Maybe it is worth taking a look at 
https://github.com/twmb/kafka-go/blob/master/pkg/kgo/internal/sticky/sticky.go 
and 
https://github.com/twmb/kafka-go/blob/master/pkg/kgo/internal/sticky/graph.go ?

Sorry I didn't see this sooner, and that I didn't take my client to completion 
sooner. Writing a client is a lot more work than I expected, so I stalled in 
the last few percent, but I'm getting back to it now during the quarantine.

> Improve sticky partition assignor algorithm
> ---
>
> Key: KAFKA-9987
> URL: https://issues.apache.org/jira/browse/KAFKA-9987
> Project: Kafka
>  Issue Type: Improvement
>  Components: clients
>Reporter: Sophie Blee-Goldman
>Assignee: Sophie Blee-Goldman
>Priority: Major
>
> In 
> [KIP-429|https://cwiki.apache.org/confluence/display/KAFKA/KIP-429%3A+Kafka+Consumer+Incremental+Rebalance+Protocol]
>  we added the new CooperativeStickyAssignor which leverages on the underlying 
> sticky assignment algorithm of the existing StickyAssignor (moved to 
> AbstractStickyAssignor). The algorithm is fairly complex as it tries to 
> optimize stickiness while satisfying perfect balance _in the case individual 
> consumers may be subscribed to different subsets of the topics._ While it 
> does a pretty good job at what it promises to do, it doesn't scale well with 
> large numbers of consumers and partitions.
> To give a concrete example, users have reported that it takes 2.5 minutes for 
> the assignment to complete with just 2100 consumers reading from 2100 
> partitions. Since partitions revoked during the first of two cooperative 
> rebalances will remain unassigned until the end of the second rebalance, it's 
> important for the rebalance to be as fast as possible. And since one of the 
> primary improvements of the cooperative rebalancing protocol is better 
> scaling experience, the only OOTB cooperative assignor should not itself 
> scale poorly
> If we can constrain the problem a bit, we can simplify the algorithm greatly. 
> In many cases the individual consumers won't be subscribed to some random 
> subset of the total subscription, they will all be subscribed to the same set 
> of topics and rely on the assignor to balance the partition workload.
> We can detect this case by checking the group's individual subscriptions and 
> call on a more efficient assignment algorithm. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9365) Add consumer group information to txn commit

2020-01-18 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9365?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17018729#comment-17018729
 ] 

Travis Bischel commented on KAFKA-9365:
---

Part of KIP-447, in 2.5.0.

> Add consumer group information to txn commit 
> -
>
> Key: KAFKA-9365
> URL: https://issues.apache.org/jira/browse/KAFKA-9365
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Boyang Chen
>Assignee: Boyang Chen
>Priority: Major
>
> This effort adds consumer group information to the txn commit protocol on the 
> broker side.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9346) Consumer fetch offset back-off with pending transactions

2020-01-18 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9346?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17018730#comment-17018730
 ] 

Travis Bischel commented on KAFKA-9346:
---

Part of KIP-447, in 2.5.0.

> Consumer fetch offset back-off with pending transactions
> 
>
> Key: KAFKA-9346
> URL: https://issues.apache.org/jira/browse/KAFKA-9346
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Boyang Chen
>Assignee: Boyang Chen
>Priority: Major
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-8432) Add static membership to Sticky assignor

2019-10-26 Thread Travis Bischel (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-8432?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16960468#comment-16960468
 ] 

Travis Bischel commented on KAFKA-8432:
---

I don't think that a static ID necessarily makes sense to use within the sticky 
balance strategy itself, after an initial "sort by id" setup (which I'll go 
into below).

First, as a note, I don't believe the Java sticky balancer is implemented as 
best it could be. I've designed a different method of sticky rebalancing that 
(I believe) is easier to understand than the current Java strategy, is faster, 
and I believe it is optimal. It covers a bunch of edge cases that the Java one 
misses, and I'm going to write the rest of this response assuming the sticky 
strategy follows my implementation (the Java one works in some confusing ways 
with respect to stickiness). My implementation can be seen 
[here|https://github.com/twmb/kafka-go/blob/c3faf878b88ec27463dc006ca011b77d46603b01/pkg/kgo/internal/sticky/sticky.go|http://example.com]
 (I may try writing a paper about it).

In the range and roundrobin strategies, the instance ID was beneficial because 
it provided both strategies some consistent IDs across restarts, which would 
preserve the original sort order of either range or roundrobin. The sticky 
strategy already has this, though. Each rejoining member says what they are 
working on and, in the best case, they keep everything. A single restarted 
member will have no history, but it will get back everything it was working on: 
everything it was working on is now in a free list, and that entire free list 
will be given back to the restarted member.

This leaves the case where two sticky members restart simultaneously. Both will 
rejoin, and both will begin draining the free list. Let's say that there were 3 
members originally in the initial join (instance IDs A, B, C), and they divvied 
up 8 partitions (1, 2, 3, 4, 5, 6).

{noformat}
A: 1 4
B: 2 5
C: 3 6
{noformat}

C stays alive, A, B restart, meaning 1, 2, 4, 5 go onto the free. If the 
strategy always assigns the lowest partition to the lowest member 
(alphanumerically) that has the fewest partitions (and can take it), then the 
rebalance will perserve ordering perfectly across restarts.

This is a benefit over the current case where, had both A and B restarted, they 
may get new IDs E and D (note alphabetic reversal), causing the partitions they 
owned prior to flip flop.

{noformat}
E: 2 5 (was A)
D: 1 4 (was B)
C: 3 6
{noformat}

**Point is**, if the strategy always works from smallest partition number goes 
to smallest alphabetic member, and if all members have instance IDs, and if the 
strategy sorts by instance ID first, then no logic change is necessary inside 
the actual algorithm itself. In fact, if the strategy just works off of the 
input ordering, then the ordering can be done outside the strategy and the 
sticky balancer does not even need to be aware of the new instance ID at all. 
This is the method I've taken: on sticky balance, I assign numbers to all input 
members in order, and then always work off the numbers in order. That an 
instance ID field was added is of no consequence.

The only flaw is that intermingling instance IDs with non-instance IDs is not 
really compatible. If a few non-instance ID member restarts, their ordering is 
up in the air.

Hopefully this makes some sense. I can try to clarify as necessary. I believe 
that if the Java code adopted the sticky strategy I devised, it'd be clearer, 
faster, cover more edge cases, and, with this new instance ID, require no work 
to have even better balancing.


> Add static membership to Sticky assignor
> 
>
> Key: KAFKA-8432
> URL: https://issues.apache.org/jira/browse/KAFKA-8432
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Boyang Chen
>Priority: Major
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)