[jira] [Commented] (KAFKA-8803) Stream will not start due to TimeoutException: Timeout expired after 60000milliseconds while awaiting InitProducerId

2019-12-01 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on KAFKA-8803:
---

abbccdda commented on pull request #7766: KAFKA-8803: [Streams fix] retry for 
initTxn
URL: https://github.com/apache/kafka/pull/7766
 
 
   Currently the initTransactions() API on StreamTask will throw uncaught 
timeout exception. It is not a desirable outcome as we normally would expect to 
retry upon timeout exception as this could indicate some temporary server side 
unavailability. Failing immediately is not ideal.
   
   The fix is that if we hit non-fatal exception,  stream task keeps retrying 
`initTransactions` until succeed.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Stream will not start due to TimeoutException: Timeout expired after 
> 6milliseconds while awaiting InitProducerId
> 
>
> Key: KAFKA-8803
> URL: https://issues.apache.org/jira/browse/KAFKA-8803
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Reporter: Raman Gupta
>Assignee: Boyang Chen
>Priority: Major
> Attachments: logs.txt.gz, screenshot-1.png
>
>
> One streams app is consistently failing at startup with the following 
> exception:
> {code}
> 2019-08-14 17:02:29,568 ERROR --- [2ce1b-StreamThread-2] 
> org.apa.kaf.str.pro.int.StreamTask: task [0_36] Timeout 
> exception caught when initializing transactions for task 0_36. This might 
> happen if the broker is slow to respond, if the network connection to the 
> broker was interrupted, or if similar circumstances arise. You can increase 
> producer parameter `max.block.ms` to increase this timeout.
> org.apache.kafka.common.errors.TimeoutException: Timeout expired after 
> 6milliseconds while awaiting InitProducerId
> {code}
> These same brokers are used by many other streams without any issue, 
> including some in the very same processes for the stream which consistently 
> throws this exception.
> *UPDATE 08/16:*
> The very first instance of this error is August 13th 2019, 17:03:36.754 and 
> it happened for 4 different streams. For 3 of these streams, the error only 
> happened once, and then the stream recovered. For the 4th stream, the error 
> has continued to happen, and continues to happen now.
> I looked up the broker logs for this time, and see that at August 13th 2019, 
> 16:47:43, two of four brokers started reporting messages like this, for 
> multiple partitions:
> [2019-08-13 20:47:43,658] INFO [ReplicaFetcher replicaId=3, leaderId=1, 
> fetcherId=0] Retrying leaderEpoch request for partition xxx-1 as the leader 
> reported an error: UNKNOWN_LEADER_EPOCH (kafka.server.ReplicaFetcherThread)
> The UNKNOWN_LEADER_EPOCH messages continued for some time, and then stopped, 
> here is a view of the count of these messages over time:
>  !screenshot-1.png! 
> However, as noted, the stream task timeout error continues to happen.
> I use the static consumer group protocol with Kafka 2.3.0 clients and 2.3.0 
> broker. The broker has a patch for KAFKA-8773.



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


[jira] [Resolved] (KAFKA-9213) BufferOverflowException on rolling new segment after upgrading Kafka from 1.1.0 to 2.3.1

2019-12-01 Thread Ismael Juma (Jira)


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

Ismael Juma resolved KAFKA-9213.

Resolution: Duplicate

Duplicate of KAFKA-9156.

> BufferOverflowException on rolling new segment after upgrading Kafka from 
> 1.1.0 to 2.3.1
> 
>
> Key: KAFKA-9213
> URL: https://issues.apache.org/jira/browse/KAFKA-9213
> Project: Kafka
>  Issue Type: Bug
>  Components: log
>Affects Versions: 2.3.1
> Environment: Ubuntu 16.04, AWS instance d2.8xlarge.
> JAVA Options:
> -Xms16G 
> -Xmx16G 
> -XX:G1HeapRegionSize=16M 
> -XX:MetaspaceSize=96m 
> -XX:MinMetaspaceFreeRatio=50 
>Reporter: Daniyar
>Priority: Blocker
>
> We updated our Kafka cluster from 1.1.0 version to 2.3.1. We followed up to 
> step 2 of the [update 
> instruction|[https://kafka.apache.org/documentation/#upgrade]].
> Message format and inter-broker protocol versions were left the same:
> inter.broker.protocol.version=1.1
> log.message.format.version=1.1
>  
> After upgrading, we started to get some occasional exceptions:
> {code:java}
> 2019/11/19 05:30:53 INFO [ProducerStateManager
> partition=matchmaker_retry_clicks_15m-2] Writing producer snapshot at
> offset 788532 (kafka.log.ProducerStateManager)
> 2019/11/19 05:30:53 INFO [Log partition=matchmaker_retry_clicks_15m-2,
> dir=/mnt/kafka] Rolled new log segment at offset 788532 in 1 ms.
> (kafka.log.Log)
> 2019/11/19 05:31:01 ERROR [ReplicaManager broker=0] Error processing append
> operation on partition matchmaker_retry_clicks_15m-2
> (kafka.server.ReplicaManager)
> 2019/11/19 05:31:01 java.nio.BufferOverflowException
> 2019/11/19 05:31:01 at java.nio.Buffer.nextPutIndex(Buffer.java:527)
> 2019/11/19 05:31:01 at
> java.nio.DirectByteBuffer.putLong(DirectByteBuffer.java:797)
> 2019/11/19 05:31:01 at
> kafka.log.TimeIndex.$anonfun$maybeAppend$1(TimeIndex.scala:134)
> 2019/11/19 05:31:01 at
> scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
> 2019/11/19 05:31:01 at
> kafka.utils.CoreUtils$.inLock(CoreUtils.scala:253)
> 2019/11/19 05:31:01 at
> kafka.log.TimeIndex.maybeAppend(TimeIndex.scala:114)
> 2019/11/19 05:31:01 at
> kafka.log.LogSegment.onBecomeInactiveSegment(LogSegment.scala:520)
> 2019/11/19 05:31:01 at kafka.log.Log.$anonfun$roll$8(Log.scala:1690)
> 2019/11/19 05:31:01 at
> kafka.log.Log.$anonfun$roll$8$adapted(Log.scala:1690)
> 2019/11/19 05:31:01 at scala.Option.foreach(Option.scala:407)
> 2019/11/19 05:31:01 at kafka.log.Log.$anonfun$roll$2(Log.scala:1690)
> 2019/11/19 05:31:01 at
> kafka.log.Log.maybeHandleIOException(Log.scala:2085)
> 2019/11/19 05:31:01 at kafka.log.Log.roll(Log.scala:1654)
> 2019/11/19 05:31:01 at kafka.log.Log.maybeRoll(Log.scala:1639)
> 2019/11/19 05:31:01 at kafka.log.Log.$anonfun$append$2(Log.scala:966)
> 2019/11/19 05:31:01 at
> kafka.log.Log.maybeHandleIOException(Log.scala:2085)
> 2019/11/19 05:31:01 at kafka.log.Log.append(Log.scala:850)
> 2019/11/19 05:31:01 at kafka.log.Log.appendAsLeader(Log.scala:819)
> 2019/11/19 05:31:01 at
> kafka.cluster.Partition.$anonfun$appendRecordsToLeader$1(Partition.scala:772)
> 2019/11/19 05:31:01 at
> kafka.utils.CoreUtils$.inLock(CoreUtils.scala:253)
> 2019/11/19 05:31:01 at
> kafka.utils.CoreUtils$.inReadLock(CoreUtils.scala:259)
> 2019/11/19 05:31:01 at
> kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:759)
> 2019/11/19 05:31:01 at
> kafka.server.ReplicaManager.$anonfun$appendToLocalLog$2(ReplicaManager.scala:763)
> 2019/11/19 05:31:01 at
> scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
> 2019/11/19 05:31:01 at
> scala.collection.mutable.HashMap.$anonfun$foreach$1(HashMap.scala:149)
> 2019/11/19 05:31:01 at
> scala.collection.mutable.HashTable.foreachEntry(HashTable.scala:237)
> 2019/11/19 05:31:01 at
> scala.collection.mutable.HashTable.foreachEntry$(HashTable.scala:230)
> 2019/11/19 05:31:01 at
> scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:44)
> 2019/11/19 05:31:01 at
> scala.collection.mutable.HashMap.foreach(HashMap.scala:149)
> 2019/11/19 05:31:01 at
> scala.collection.TraversableLike.map(TraversableLike.scala:238)
> 2019/11/19 05:31:01 at
> scala.collection.TraversableLike.map$(TraversableLike.scala:231)
> 2019/11/19 05:31:01 at
> scala.collection.AbstractTraversable.map(Traversable.scala:108)
> 2019/11/19 05:31:01 at
> kafka.server.ReplicaManager.appendToLocalLog(ReplicaManager.scala:751)
> 2019/11/19 05:31:01 at
> kafka.server.ReplicaManager.appendRecords(ReplicaManager.scala:492)
> 2019/11/19 05:31:01 at
> kafka.server.KafkaApis.handleProduceRequest(KafkaApis.scala:544)
> 2019/11/19 05:31:01 at
> 

[jira] [Updated] (KAFKA-9156) LazyTimeIndex & LazyOffsetIndex may cause niobufferoverflow in concurrent state

2019-12-01 Thread Ismael Juma (Jira)


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

Ismael Juma updated KAFKA-9156:
---
Labels: regression  (was: )

> LazyTimeIndex & LazyOffsetIndex may cause niobufferoverflow in concurrent 
> state
> ---
>
> Key: KAFKA-9156
> URL: https://issues.apache.org/jira/browse/KAFKA-9156
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 2.3.0, 2.3.1
>Reporter: shilin Lu
>Priority: Blocker
>  Labels: regression
> Fix For: 2.4.0, 2.3.2
>
> Attachments: image-2019-11-07-17-42-13-852.png, 
> image-2019-11-07-17-44-05-357.png, image-2019-11-07-17-46-53-650.png
>
>
> !image-2019-11-07-17-42-13-852.png!
> this timeindex get function is not thread safe ,may cause create some 
> timeindex.
> !image-2019-11-07-17-44-05-357.png!
> When create timeindex not exactly one ,may cause mappedbytebuffer position to 
> end. Then write index entry to this mmap file will cause 
> java.nio.BufferOverflowException.
>  
> !image-2019-11-07-17-46-53-650.png!
>  
>  
>  



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


[jira] [Updated] (KAFKA-9213) BufferOverflowException on rolling new segment after upgrading Kafka from 1.1.0 to 2.3.1

2019-12-01 Thread Ismael Juma (Jira)


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

Ismael Juma updated KAFKA-9213:
---
Affects Version/s: 2.3.0

> BufferOverflowException on rolling new segment after upgrading Kafka from 
> 1.1.0 to 2.3.1
> 
>
> Key: KAFKA-9213
> URL: https://issues.apache.org/jira/browse/KAFKA-9213
> Project: Kafka
>  Issue Type: Bug
>  Components: log
>Affects Versions: 2.3.0, 2.3.1
> Environment: Ubuntu 16.04, AWS instance d2.8xlarge.
> JAVA Options:
> -Xms16G 
> -Xmx16G 
> -XX:G1HeapRegionSize=16M 
> -XX:MetaspaceSize=96m 
> -XX:MinMetaspaceFreeRatio=50 
>Reporter: Daniyar
>Priority: Blocker
>
> We updated our Kafka cluster from 1.1.0 version to 2.3.1. We followed up to 
> step 2 of the [update 
> instruction|[https://kafka.apache.org/documentation/#upgrade]].
> Message format and inter-broker protocol versions were left the same:
> inter.broker.protocol.version=1.1
> log.message.format.version=1.1
>  
> After upgrading, we started to get some occasional exceptions:
> {code:java}
> 2019/11/19 05:30:53 INFO [ProducerStateManager
> partition=matchmaker_retry_clicks_15m-2] Writing producer snapshot at
> offset 788532 (kafka.log.ProducerStateManager)
> 2019/11/19 05:30:53 INFO [Log partition=matchmaker_retry_clicks_15m-2,
> dir=/mnt/kafka] Rolled new log segment at offset 788532 in 1 ms.
> (kafka.log.Log)
> 2019/11/19 05:31:01 ERROR [ReplicaManager broker=0] Error processing append
> operation on partition matchmaker_retry_clicks_15m-2
> (kafka.server.ReplicaManager)
> 2019/11/19 05:31:01 java.nio.BufferOverflowException
> 2019/11/19 05:31:01 at java.nio.Buffer.nextPutIndex(Buffer.java:527)
> 2019/11/19 05:31:01 at
> java.nio.DirectByteBuffer.putLong(DirectByteBuffer.java:797)
> 2019/11/19 05:31:01 at
> kafka.log.TimeIndex.$anonfun$maybeAppend$1(TimeIndex.scala:134)
> 2019/11/19 05:31:01 at
> scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
> 2019/11/19 05:31:01 at
> kafka.utils.CoreUtils$.inLock(CoreUtils.scala:253)
> 2019/11/19 05:31:01 at
> kafka.log.TimeIndex.maybeAppend(TimeIndex.scala:114)
> 2019/11/19 05:31:01 at
> kafka.log.LogSegment.onBecomeInactiveSegment(LogSegment.scala:520)
> 2019/11/19 05:31:01 at kafka.log.Log.$anonfun$roll$8(Log.scala:1690)
> 2019/11/19 05:31:01 at
> kafka.log.Log.$anonfun$roll$8$adapted(Log.scala:1690)
> 2019/11/19 05:31:01 at scala.Option.foreach(Option.scala:407)
> 2019/11/19 05:31:01 at kafka.log.Log.$anonfun$roll$2(Log.scala:1690)
> 2019/11/19 05:31:01 at
> kafka.log.Log.maybeHandleIOException(Log.scala:2085)
> 2019/11/19 05:31:01 at kafka.log.Log.roll(Log.scala:1654)
> 2019/11/19 05:31:01 at kafka.log.Log.maybeRoll(Log.scala:1639)
> 2019/11/19 05:31:01 at kafka.log.Log.$anonfun$append$2(Log.scala:966)
> 2019/11/19 05:31:01 at
> kafka.log.Log.maybeHandleIOException(Log.scala:2085)
> 2019/11/19 05:31:01 at kafka.log.Log.append(Log.scala:850)
> 2019/11/19 05:31:01 at kafka.log.Log.appendAsLeader(Log.scala:819)
> 2019/11/19 05:31:01 at
> kafka.cluster.Partition.$anonfun$appendRecordsToLeader$1(Partition.scala:772)
> 2019/11/19 05:31:01 at
> kafka.utils.CoreUtils$.inLock(CoreUtils.scala:253)
> 2019/11/19 05:31:01 at
> kafka.utils.CoreUtils$.inReadLock(CoreUtils.scala:259)
> 2019/11/19 05:31:01 at
> kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:759)
> 2019/11/19 05:31:01 at
> kafka.server.ReplicaManager.$anonfun$appendToLocalLog$2(ReplicaManager.scala:763)
> 2019/11/19 05:31:01 at
> scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
> 2019/11/19 05:31:01 at
> scala.collection.mutable.HashMap.$anonfun$foreach$1(HashMap.scala:149)
> 2019/11/19 05:31:01 at
> scala.collection.mutable.HashTable.foreachEntry(HashTable.scala:237)
> 2019/11/19 05:31:01 at
> scala.collection.mutable.HashTable.foreachEntry$(HashTable.scala:230)
> 2019/11/19 05:31:01 at
> scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:44)
> 2019/11/19 05:31:01 at
> scala.collection.mutable.HashMap.foreach(HashMap.scala:149)
> 2019/11/19 05:31:01 at
> scala.collection.TraversableLike.map(TraversableLike.scala:238)
> 2019/11/19 05:31:01 at
> scala.collection.TraversableLike.map$(TraversableLike.scala:231)
> 2019/11/19 05:31:01 at
> scala.collection.AbstractTraversable.map(Traversable.scala:108)
> 2019/11/19 05:31:01 at
> kafka.server.ReplicaManager.appendToLocalLog(ReplicaManager.scala:751)
> 2019/11/19 05:31:01 at
> kafka.server.ReplicaManager.appendRecords(ReplicaManager.scala:492)
> 2019/11/19 05:31:01 at
> kafka.server.KafkaApis.handleProduceRequest(KafkaApis.scala:544)
> 2019/11/19 05:31:01 at
> 

[jira] [Updated] (KAFKA-9156) LazyTimeIndex & LazyOffsetIndex may cause niobufferoverflow in concurrent state

2019-12-01 Thread Ismael Juma (Jira)


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

Ismael Juma updated KAFKA-9156:
---
Affects Version/s: 2.3.1

> LazyTimeIndex & LazyOffsetIndex may cause niobufferoverflow in concurrent 
> state
> ---
>
> Key: KAFKA-9156
> URL: https://issues.apache.org/jira/browse/KAFKA-9156
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 2.3.0, 2.3.1
>Reporter: shilin Lu
>Priority: Blocker
> Fix For: 2.4.0, 2.3.2
>
> Attachments: image-2019-11-07-17-42-13-852.png, 
> image-2019-11-07-17-44-05-357.png, image-2019-11-07-17-46-53-650.png
>
>
> !image-2019-11-07-17-42-13-852.png!
> this timeindex get function is not thread safe ,may cause create some 
> timeindex.
> !image-2019-11-07-17-44-05-357.png!
> When create timeindex not exactly one ,may cause mappedbytebuffer position to 
> end. Then write index entry to this mmap file will cause 
> java.nio.BufferOverflowException.
>  
> !image-2019-11-07-17-46-53-650.png!
>  
>  
>  



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


[jira] [Updated] (KAFKA-9156) LazyTimeIndex & LazyOffsetIndex may cause niobufferoverflow in concurrent state

2019-12-01 Thread Ismael Juma (Jira)


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

Ismael Juma updated KAFKA-9156:
---
Fix Version/s: 2.3.2
   2.4.0

> LazyTimeIndex & LazyOffsetIndex may cause niobufferoverflow in concurrent 
> state
> ---
>
> Key: KAFKA-9156
> URL: https://issues.apache.org/jira/browse/KAFKA-9156
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 2.3.0
>Reporter: shilin Lu
>Priority: Critical
> Fix For: 2.4.0, 2.3.2
>
> Attachments: image-2019-11-07-17-42-13-852.png, 
> image-2019-11-07-17-44-05-357.png, image-2019-11-07-17-46-53-650.png
>
>
> !image-2019-11-07-17-42-13-852.png!
> this timeindex get function is not thread safe ,may cause create some 
> timeindex.
> !image-2019-11-07-17-44-05-357.png!
> When create timeindex not exactly one ,may cause mappedbytebuffer position to 
> end. Then write index entry to this mmap file will cause 
> java.nio.BufferOverflowException.
>  
> !image-2019-11-07-17-46-53-650.png!
>  
>  
>  



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


[jira] [Updated] (KAFKA-9156) LazyTimeIndex & LazyOffsetIndex may cause niobufferoverflow in concurrent state

2019-12-01 Thread Ismael Juma (Jira)


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

Ismael Juma updated KAFKA-9156:
---
Priority: Blocker  (was: Critical)

> LazyTimeIndex & LazyOffsetIndex may cause niobufferoverflow in concurrent 
> state
> ---
>
> Key: KAFKA-9156
> URL: https://issues.apache.org/jira/browse/KAFKA-9156
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 2.3.0
>Reporter: shilin Lu
>Priority: Blocker
> Fix For: 2.4.0, 2.3.2
>
> Attachments: image-2019-11-07-17-42-13-852.png, 
> image-2019-11-07-17-44-05-357.png, image-2019-11-07-17-46-53-650.png
>
>
> !image-2019-11-07-17-42-13-852.png!
> this timeindex get function is not thread safe ,may cause create some 
> timeindex.
> !image-2019-11-07-17-44-05-357.png!
> When create timeindex not exactly one ,may cause mappedbytebuffer position to 
> end. Then write index entry to this mmap file will cause 
> java.nio.BufferOverflowException.
>  
> !image-2019-11-07-17-46-53-650.png!
>  
>  
>  



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


[jira] [Commented] (KAFKA-9255) MessageSet v1 protocol wrong specification

2019-12-01 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on KAFKA-9255:
---

ffosilva commented on pull request #7764: KAFKA-9255: MessageSet v1 protocol 
wrong specification
URL: https://github.com/apache/kafka/pull/7764
 
 
   *More detailed description of your change,
   if necessary. The PR title and PR message become
   the squashed commit message, so use a separate
   comment to ping reviewers.*
   
   *Summary of testing strategy (including rationale)
   for the feature or bug fix. Unit and/or integration
   tests are expected for any behaviour change and
   system tests should be considered for larger changes.*
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> MessageSet v1 protocol wrong specification
> --
>
> Key: KAFKA-9255
> URL: https://issues.apache.org/jira/browse/KAFKA-9255
> Project: Kafka
>  Issue Type: Bug
>  Components: documentation
>Reporter: Fábio Silva
>Priority: Major
>
> The documentation contains a BNF specification missing the timestamp field on 
> 'message' field entry.



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


[jira] [Updated] (KAFKA-9255) MessageSet v1 protocol wrong specification

2019-12-01 Thread Jira


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

Fábio Silva updated KAFKA-9255:
---
Reviewer: Fábio Silva

> MessageSet v1 protocol wrong specification
> --
>
> Key: KAFKA-9255
> URL: https://issues.apache.org/jira/browse/KAFKA-9255
> Project: Kafka
>  Issue Type: Bug
>  Components: documentation
>Reporter: Fábio Silva
>Priority: Major
>
> The documentation contains a BNF specification missing the timestamp field on 
> 'message' field entry.



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


[jira] [Updated] (KAFKA-9255) MessageSet v1 protocol wrong specification

2019-12-01 Thread Jira


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

Fábio Silva updated KAFKA-9255:
---
Reviewer:   (was: Fábio Silva)

> MessageSet v1 protocol wrong specification
> --
>
> Key: KAFKA-9255
> URL: https://issues.apache.org/jira/browse/KAFKA-9255
> Project: Kafka
>  Issue Type: Bug
>  Components: documentation
>Reporter: Fábio Silva
>Priority: Major
>
> The documentation contains a BNF specification missing the timestamp field on 
> 'message' field entry.



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


[jira] [Created] (KAFKA-9255) MessageSet v1 protocol wrong specification

2019-12-01 Thread Jira
Fábio Silva created KAFKA-9255:
--

 Summary: MessageSet v1 protocol wrong specification
 Key: KAFKA-9255
 URL: https://issues.apache.org/jira/browse/KAFKA-9255
 Project: Kafka
  Issue Type: Bug
  Components: documentation
Reporter: Fábio Silva


The documentation contains a BNF specification missing the timestamp field on 
'message' field entry.



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


[jira] [Commented] (KAFKA-7658) Add KStream#toTable to the Streams DSL

2019-12-01 Thread Matthias J. Sax (Jira)


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

Matthias J. Sax commented on KAFKA-7658:


[~ash26389] – the KIP was approved. Are you working on a PR for this already?

> Add KStream#toTable to the Streams DSL
> --
>
> Key: KAFKA-7658
> URL: https://issues.apache.org/jira/browse/KAFKA-7658
> Project: Kafka
>  Issue Type: Improvement
>  Components: streams
>Reporter: Guozhang Wang
>Assignee: Aishwarya Pradeep Kumar
>Priority: Major
>  Labels: kip, newbie
>
> KIP-523: 
> [https://cwiki.apache.org/confluence/display/KAFKA/KIP-523%3A+Add+KStream%23toTable+to+the+Streams+DSL]
>  
> We'd like to add a new API to the KStream object of the Streams DSL:
> {code:java}
> KTable KStream.toTable()
> KTable KStream.toTable(Materialized)
> {code}
> The function re-interpret the event stream {{KStream}} as a changelog stream 
> {{KTable}}. Note that this should NOT be treated as a syntax-sugar as a dummy 
> {{KStream.reduce()}} function which always take the new value, as it has the 
> following difference:
> 1) an aggregation operator of {{KStream}} is for aggregating a event stream 
> into an evolving table, which will drop null-values from the input event 
> stream; whereas a {{toTable}} function will completely change the semantics 
> of the input stream from event stream to changelog stream, and null-values 
> will still be serialized, and if the resulted bytes are also null they will 
> be interpreted as "deletes" to the materialized KTable (i.e. tombstones in 
> the changelog stream).
> 2) the aggregation result {{KTable}} will always be materialized, whereas 
> {{toTable}} resulted KTable may only be materialized if the overloaded 
> function with Materialized is used (and if optimization is turned on it may 
> still be only logically materialized if the queryable name is not set).
> Therefore, for users who want to take a event stream into a changelog stream 
> (no matter why they cannot read from the source topic as a changelog stream 
> {{KTable}} at the beginning), they should be using this new API instead of 
> the dummy reduction function.



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


[jira] [Updated] (KAFKA-7658) Add KStream#toTable to the Streams DSL

2019-12-01 Thread Matthias J. Sax (Jira)


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

Matthias J. Sax updated KAFKA-7658:
---
Description: 
KIP-523: 
[https://cwiki.apache.org/confluence/display/KAFKA/KIP-523%3A+Add+KStream%23toTable+to+the+Streams+DSL]

 

We'd like to add a new API to the KStream object of the Streams DSL:
{code:java}
KTable KStream.toTable()

KTable KStream.toTable(Materialized)
{code}
The function re-interpret the event stream {{KStream}} as a changelog stream 
{{KTable}}. Note that this should NOT be treated as a syntax-sugar as a dummy 
{{KStream.reduce()}} function which always take the new value, as it has the 
following difference:

1) an aggregation operator of {{KStream}} is for aggregating a event stream 
into an evolving table, which will drop null-values from the input event 
stream; whereas a {{toTable}} function will completely change the semantics of 
the input stream from event stream to changelog stream, and null-values will 
still be serialized, and if the resulted bytes are also null they will be 
interpreted as "deletes" to the materialized KTable (i.e. tombstones in the 
changelog stream).

2) the aggregation result {{KTable}} will always be materialized, whereas 
{{toTable}} resulted KTable may only be materialized if the overloaded function 
with Materialized is used (and if optimization is turned on it may still be 
only logically materialized if the queryable name is not set).

Therefore, for users who want to take a event stream into a changelog stream 
(no matter why they cannot read from the source topic as a changelog stream 
{{KTable}} at the beginning), they should be using this new API instead of the 
dummy reduction function.

  was:
We'd like to add a new API to the KStream object of the Streams DSL:

{code}
KTable KStream.toTable()

KTable KStream.toTable(Materialized)
{code}

The function re-interpret the event stream {{KStream}} as a changelog stream 
{{KTable}}. Note that this should NOT be treated as a syntax-sugar as a dummy 
{{KStream.reduce()}} function which always take the new value, as it has the 
following difference: 

1) an aggregation operator of {{KStream}} is for aggregating a event stream 
into an evolving table, which will drop null-values from the input event 
stream; whereas a {{toTable}} function will completely change the semantics of 
the input stream from event stream to changelog stream, and null-values will 
still be serialized, and if the resulted bytes are also null they will be 
interpreted as "deletes" to the materialized KTable (i.e. tombstones in the 
changelog stream).

2) the aggregation result {{KTable}} will always be materialized, whereas 
{{toTable}} resulted KTable may only be materialized if the overloaded function 
with Materialized is used (and if optimization is turned on it may still be 
only logically materialized if the queryable name is not set).

Therefore, for users who want to take a event stream into a changelog stream 
(no matter why they cannot read from the source topic as a changelog stream 
{{KTable}} at the beginning), they should be using this new API instead of the 
dummy reduction function.


> Add KStream#toTable to the Streams DSL
> --
>
> Key: KAFKA-7658
> URL: https://issues.apache.org/jira/browse/KAFKA-7658
> Project: Kafka
>  Issue Type: Improvement
>  Components: streams
>Reporter: Guozhang Wang
>Assignee: Aishwarya Pradeep Kumar
>Priority: Major
>  Labels: kip, newbie
>
> KIP-523: 
> [https://cwiki.apache.org/confluence/display/KAFKA/KIP-523%3A+Add+KStream%23toTable+to+the+Streams+DSL]
>  
> We'd like to add a new API to the KStream object of the Streams DSL:
> {code:java}
> KTable KStream.toTable()
> KTable KStream.toTable(Materialized)
> {code}
> The function re-interpret the event stream {{KStream}} as a changelog stream 
> {{KTable}}. Note that this should NOT be treated as a syntax-sugar as a dummy 
> {{KStream.reduce()}} function which always take the new value, as it has the 
> following difference:
> 1) an aggregation operator of {{KStream}} is for aggregating a event stream 
> into an evolving table, which will drop null-values from the input event 
> stream; whereas a {{toTable}} function will completely change the semantics 
> of the input stream from event stream to changelog stream, and null-values 
> will still be serialized, and if the resulted bytes are also null they will 
> be interpreted as "deletes" to the materialized KTable (i.e. tombstones in 
> the changelog stream).
> 2) the aggregation result {{KTable}} will always be materialized, whereas 
> {{toTable}} resulted KTable may only be materialized if the overloaded 
> function with Materialized is used (and if optimization is turned on it may 
> still be only logically materialized if the 

[jira] [Updated] (KAFKA-7658) Add KStream#toTable to the Streams DSL

2019-12-01 Thread Matthias J. Sax (Jira)


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

Matthias J. Sax updated KAFKA-7658:
---
Labels: kip newbie  (was: needs-kip newbie)

> Add KStream#toTable to the Streams DSL
> --
>
> Key: KAFKA-7658
> URL: https://issues.apache.org/jira/browse/KAFKA-7658
> Project: Kafka
>  Issue Type: Improvement
>  Components: streams
>Reporter: Guozhang Wang
>Assignee: Aishwarya Pradeep Kumar
>Priority: Major
>  Labels: kip, newbie
>
> We'd like to add a new API to the KStream object of the Streams DSL:
> {code}
> KTable KStream.toTable()
> KTable KStream.toTable(Materialized)
> {code}
> The function re-interpret the event stream {{KStream}} as a changelog stream 
> {{KTable}}. Note that this should NOT be treated as a syntax-sugar as a dummy 
> {{KStream.reduce()}} function which always take the new value, as it has the 
> following difference: 
> 1) an aggregation operator of {{KStream}} is for aggregating a event stream 
> into an evolving table, which will drop null-values from the input event 
> stream; whereas a {{toTable}} function will completely change the semantics 
> of the input stream from event stream to changelog stream, and null-values 
> will still be serialized, and if the resulted bytes are also null they will 
> be interpreted as "deletes" to the materialized KTable (i.e. tombstones in 
> the changelog stream).
> 2) the aggregation result {{KTable}} will always be materialized, whereas 
> {{toTable}} resulted KTable may only be materialized if the overloaded 
> function with Materialized is used (and if optimization is turned on it may 
> still be only logically materialized if the queryable name is not set).
> Therefore, for users who want to take a event stream into a changelog stream 
> (no matter why they cannot read from the source topic as a changelog stream 
> {{KTable}} at the beginning), they should be using this new API instead of 
> the dummy reduction function.



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


[jira] [Updated] (KAFKA-1714) more better bootstrapping of the gradle-wrapper.jar

2019-12-01 Thread Randall Hauch (Jira)


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

Randall Hauch updated KAFKA-1714:
-
Fix Version/s: (was: 2.2.2)
   2.2.3

> more better bootstrapping of the gradle-wrapper.jar 
> 
>
> Key: KAFKA-1714
> URL: https://issues.apache.org/jira/browse/KAFKA-1714
> Project: Kafka
>  Issue Type: Bug
>  Components: build
>Affects Versions: 0.8.2.0
>Reporter: Joe Stein
>Assignee: Grant Henke
>Priority: Major
> Fix For: 1.0.3, 1.1.2, 2.0.2, 2.1.2, 2.4.0, 2.2.3, 2.3.2
>
>
> From https://issues.apache.org/jira/browse/KAFKA-1490 we moved out the 
> gradle-wrapper.jar for our source maintenance. This makes builds for folks 
> coming in the first step somewhat problematic.  A bootstrap step is required 
> if this could be somehow incorporated that would be great.



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


[jira] [Comment Edited] (KAFKA-9071) transactional.id.expiration.ms config value should be implemented as a Long

2019-12-01 Thread Matthias J. Sax (Jira)


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

Matthias J. Sax edited comment on KAFKA-9071 at 12/1/19 8:45 PM:
-

[~despondency], thanks for your interest. Just a quick clarification: you are 
not a _committer_ as this term has a special meaning in the Apache world 
([https://www.apache.org/foundation/how-it-works.html#committers]). If you are 
interested, you can find the list of Kafka committers and PMC members on the 
project web page: [https://kafka.apache.org/committers]

However, I added you to the list of _contributors_ 
([https://www.apache.org/foundation/how-it-works.html#developers]) on Jira, and 
you can now self-assign tickets. Please check out 
[https://kafka.apache.org/contributing] to get started.

Please open a PR against `trunk` and the committer who merges your PR will 
cherry-pick it to older branches if required. \cc [~hachikuji] who might be the 
best person to review and make a call here.

 


was (Author: mjsax):
[~despondency], thanks for your interest. Just a quick clarification: you are 
not a _committer_ as this term has a special meaning in the Apache world 
([https://www.apache.org/foundation/how-it-works.html#committers]). If you are 
interested, you can find the list of Kafka committers and PMC members on the 
project web page: [https://kafka.apache.org/committers]

However, I added you to the list of _contributors_ 
([https://www.apache.org/foundation/how-it-works.html#developers]) on Jira, and 
you can now self-assign tickets.

Please open a PR against `trunk` and the committer who merges your PR will 
cherry-pick it to older branches if required. \cc [~hachikuji] who might be the 
best person to review and make a call here.

 

> transactional.id.expiration.ms config value should be implemented as a Long
> ---
>
> Key: KAFKA-9071
> URL: https://issues.apache.org/jira/browse/KAFKA-9071
> Project: Kafka
>  Issue Type: Improvement
>  Components: config
>Affects Versions: 2.3.0
>Reporter: Joost van de Wijgerd
>Priority: Major
>
> Currently the value of this config parameter is limited to MAX_INT 
> effectively limiting the transactional id expiration to  ~ 25 days. This is 
> causing some issues for us on our Acceptance environment (which is not used 
> that often / heavily) where our transactional services will start failing 
> because if this issue.
> I believe best practice for millisecond values should be to implement them as 
> a Long and not as an Integer
> this is currently the max value: transactional.id.expiration.ms=2147483647
> while I would like to set it to: transactional.id.expiration.ms=3154000 
> (i.e. 1 year)



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


[jira] [Commented] (KAFKA-9071) transactional.id.expiration.ms config value should be implemented as a Long

2019-12-01 Thread Matthias J. Sax (Jira)


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

Matthias J. Sax commented on KAFKA-9071:


[~despondency], thanks for your interest. Just a quick clarification: you are 
not a _committer_ as this term has a special meaning in the Apache world 
([https://www.apache.org/foundation/how-it-works.html#committers]). If you are 
interested, you can find the list of Kafka committers and PMC members on the 
project web page: [https://kafka.apache.org/committers]

However, I added you to the list of _contributors_ 
([https://www.apache.org/foundation/how-it-works.html#developers]) on Jira, and 
you can now self-assign tickets.

Please open a PR against `trunk` and the committer who merges your PR will 
cherry-pick it to older branches if required. \cc [~hachikuji] who might be the 
best person to review and make a call here.

 

> transactional.id.expiration.ms config value should be implemented as a Long
> ---
>
> Key: KAFKA-9071
> URL: https://issues.apache.org/jira/browse/KAFKA-9071
> Project: Kafka
>  Issue Type: Improvement
>  Components: config
>Affects Versions: 2.3.0
>Reporter: Joost van de Wijgerd
>Priority: Major
>
> Currently the value of this config parameter is limited to MAX_INT 
> effectively limiting the transactional id expiration to  ~ 25 days. This is 
> causing some issues for us on our Acceptance environment (which is not used 
> that often / heavily) where our transactional services will start failing 
> because if this issue.
> I believe best practice for millisecond values should be to implement them as 
> a Long and not as an Integer
> this is currently the max value: transactional.id.expiration.ms=2147483647
> while I would like to set it to: transactional.id.expiration.ms=3154000 
> (i.e. 1 year)



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


[jira] [Updated] (KAFKA-9231) Streams Threads may die from recoverable errors with EOS enabled

2019-12-01 Thread Manikumar (Jira)


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

Manikumar updated KAFKA-9231:
-
Fix Version/s: 2.4.0

> Streams Threads may die from recoverable errors with EOS enabled
> 
>
> Key: KAFKA-9231
> URL: https://issues.apache.org/jira/browse/KAFKA-9231
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.4.0
>Reporter: John Roesler
>Assignee: John Roesler
>Priority: Blocker
> Fix For: 2.4.0
>
>
> While testing Streams in EOS mode under frequent and heavy network 
> partitions, I've encountered the following error, leading to thread death:
> {noformat}
> [2019-11-26 04:54:02,650] ERROR 
> [stream-soak-test-4290196e-d805-4acd-9f78-b459cc7e99ee-StreamThread-2] 
> stream-thread 
> [stream-soak-test-4290196e-d805-4acd-9f78-b459cc7e99ee-StreamThread-2] 
> Encountered the following unexpected Kafka exception during processing, this 
> usually indicate Streams internal errors: 
> (org.apache.kafka.streams.processor.internals.StreamThread)
> org.apache.kafka.streams.errors.StreamsException: stream-thread 
> [stream-soak-test-4290196e-d805-4acd-9f78-b459cc7e99ee-StreamThread-2] Failed 
> to rebalance.
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.pollRequests(StreamThread.java:852)
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:739)
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:698)
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:671)
> Caused by: org.apache.kafka.streams.errors.StreamsException: stream-thread 
> [stream-soak-test-4290196e-d805-4acd-9f78-b459cc7e99ee-StreamThread-2] failed 
> to suspend stream tasks
>   at 
> org.apache.kafka.streams.processor.internals.TaskManager.suspendActiveTasksAndState(TaskManager.java:253)
>   at 
> org.apache.kafka.streams.processor.internals.StreamsRebalanceListener.onPartitionsRevoked(StreamsRebalanceListener.java:116)
>   at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.invokePartitionsRevoked(ConsumerCoordinator.java:291)
>   at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.onLeavePrepare(ConsumerCoordinator.java:707)
>   at 
> org.apache.kafka.clients.consumer.KafkaConsumer.unsubscribe(KafkaConsumer.java:1073)
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.enforceRebalance(StreamThread.java:716)
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:710)
>   ... 1 more
> Caused by: org.apache.kafka.streams.errors.ProcessorStateException: task 
> [1_1] Failed to flush state store KSTREAM-AGGREGATE-STATE-STORE-07
>   at 
> org.apache.kafka.streams.processor.internals.ProcessorStateManager.flush(ProcessorStateManager.java:279)
>   at 
> org.apache.kafka.streams.processor.internals.AbstractTask.flushState(AbstractTask.java:175)
>   at 
> org.apache.kafka.streams.processor.internals.StreamTask.flushState(StreamTask.java:581)
>   at 
> org.apache.kafka.streams.processor.internals.StreamTask.commit(StreamTask.java:535)
>   at 
> org.apache.kafka.streams.processor.internals.StreamTask.suspend(StreamTask.java:660)
>   at 
> org.apache.kafka.streams.processor.internals.StreamTask.suspend(StreamTask.java:628)
>   at 
> org.apache.kafka.streams.processor.internals.AssignedStreamsTasks.suspendRunningTasks(AssignedStreamsTasks.java:145)
>   at 
> org.apache.kafka.streams.processor.internals.AssignedStreamsTasks.suspendOrCloseTasks(AssignedStreamsTasks.java:128)
>   at 
> org.apache.kafka.streams.processor.internals.TaskManager.suspendActiveTasksAndState(TaskManager.java:246)
>   ... 7 more
> Caused by: org.apache.kafka.common.errors.ProducerFencedException: Producer 
> attempted an operation with an old epoch. Either there is a newer producer 
> with the same transactionalId, or the producer's transaction has been expired 
> by the broker.
> [2019-11-26 04:54:02,650] INFO 
> [stream-soak-test-4290196e-d805-4acd-9f78-b459cc7e99ee-StreamThread-2] 
> stream-thread 
> [stream-soak-test-4290196e-d805-4acd-9f78-b459cc7e99ee-StreamThread-2] State 
> transition from PARTITIONS_REVOKED to PENDING_SHUTDOWN 
> (org.apache.kafka.streams.processor.internals.StreamThread)
> [2019-11-26 04:54:02,650] INFO 
> [stream-soak-test-4290196e-d805-4acd-9f78-b459cc7e99ee-StreamThread-2] 
> stream-thread 
> [stream-soak-test-4290196e-d805-4acd-9f78-b459cc7e99ee-StreamThread-2] 
> Shutting down (org.apache.kafka.streams.processor.internals.StreamThread)
> [2019-11-26 04:54:02,650] INFO 
> 

[jira] [Commented] (KAFKA-9071) transactional.id.expiration.ms config value should be implemented as a Long

2019-12-01 Thread Mario Georgiev (Jira)


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

Mario Georgiev commented on KAFKA-9071:
---

Hello I am a new committer, could I get that issue, since it appears pretty 
basic for a new starter. 

Also does that need to be backported to 2.3.0 as well as any newer version?

If it's OK to go forward with the issue, please do assign me.

> transactional.id.expiration.ms config value should be implemented as a Long
> ---
>
> Key: KAFKA-9071
> URL: https://issues.apache.org/jira/browse/KAFKA-9071
> Project: Kafka
>  Issue Type: Improvement
>  Components: config
>Affects Versions: 2.3.0
>Reporter: Joost van de Wijgerd
>Priority: Major
>
> Currently the value of this config parameter is limited to MAX_INT 
> effectively limiting the transactional id expiration to  ~ 25 days. This is 
> causing some issues for us on our Acceptance environment (which is not used 
> that often / heavily) where our transactional services will start failing 
> because if this issue.
> I believe best practice for millisecond values should be to implement them as 
> a Long and not as an Integer
> this is currently the max value: transactional.id.expiration.ms=2147483647
> while I would like to set it to: transactional.id.expiration.ms=3154000 
> (i.e. 1 year)



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


[jira] [Commented] (KAFKA-9236) Confused log after using CLI scripts to produce messages

2019-12-01 Thread huxihx (Jira)


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

huxihx commented on KAFKA-9236:
---

Sometimes, it makes sense to specify a timeout when closing the producer if you 
don't want to wait indefinitely to complete the sending of all incomplete 
requests. In such a case, logging with the timeout value sounds reasonable.

> Confused log after using CLI scripts to produce messages
> 
>
> Key: KAFKA-9236
> URL: https://issues.apache.org/jira/browse/KAFKA-9236
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Xiang Zhang
>Priority: Major
>




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


[jira] [Commented] (KAFKA-9254) Topic level configuration failed

2019-12-01 Thread huxihx (Jira)


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

huxihx commented on KAFKA-9254:
---

Seems I cannot reproduce it with 2.3.0. Could you retry with the newest version?

> Topic level configuration failed
> 
>
> Key: KAFKA-9254
> URL: https://issues.apache.org/jira/browse/KAFKA-9254
> Project: Kafka
>  Issue Type: Bug
>  Components: config, log, replication
>Affects Versions: 2.0.1
>Reporter: fenghong
>Priority: Critical
>
> We are engineers at Huobi and now encounter Kafka BUG 
> Modifying DynamicBrokerConfig more than 2 times will invalidate the topic 
> level unrelated configuration
> The bug reproduction method as follows:
>  # Set Kafka Broker config  server.properties min.insync.replicas=3
>  # Create topic test-1 and set topic‘s level config min.insync.replicas=2
>  # Dynamically modify the configuration twice as shown below
> {code:java}
> bin/kafka-configs.sh --bootstrap-server xxx:9092 --entity-type brokers 
> --entity-default --alter --add-config log.message.timestamp.type=LogAppendTime
> bin/kafka-configs.sh --bootstrap-server xxx:9092 --entity-type brokers 
> --entity-default --alter --add-config log.retention.ms=60480
> {code}
>  # stop a Kafka Server and found the Exception as shown below
>  org.apache.kafka.common.errors.NotEnoughReplicasException: Number of insync 
> replicas for partition test-1-0 is [2], below required minimum [3]
>  
>  



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