[jira] [Resolved] (KAFKA-10453) Backport of PR-7781

2020-08-31 Thread Niketh Sabbineni (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10453?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Niketh Sabbineni resolved KAFKA-10453. -- Resolution: Workaround > Backport of PR-7781 > --- > >

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-08-31 Thread Jun Rao
Hi, Satish, Thanks for the updated KIP. Made another pass. A few more comments below. 600. The topic deletion logic needs more details. 600.1 The KIP mentions "The controller considers the topic partition is deleted only when it determines that there are no log segments for that topic partition b

Re: [DISCUSS] KIP-667: Remove deprecated methods from ReadOnlyWindowStore

2020-08-31 Thread Sophie Blee-Goldman
Thanks for this KIP as well! Seems like the methods were deprecated in 2.1. What's our rule for how something has to stay deprecated before we can go ahead and remove it? Assuming 3.0 comes after 2.8, it certainly seems like enough time/releases have passed for us to do so in 3.0. But I'm pretty s

Re: [DISCUSS] KIP-666: Add Instant-based methods to ReadOnlySessionStore

2020-08-31 Thread Sophie Blee-Goldman
Thanks for bringing the IQ API into alignment -- the proposal looks good, although one nit: you missed updating the startTime long to Instant in both appearances of the fetchSession(key, startTime, sessionEndTime) method. Also, I think by "startTime" you actually meant "earliestSessionEndTime". On

Build failed in Jenkins: Kafka » kafka-trunk-jdk8 #36

2020-08-31 Thread Apache Jenkins Server
See Changes: [github] KAFKA-5636: SlidingWindows (KIP-450) (#9039) -- [...truncated 6.47 MB...] org.apache.kafka.streams.internals.WindowStoreFacadeTest > sho

Build failed in Jenkins: Kafka » kafka-trunk-jdk15 #37

2020-08-31 Thread Apache Jenkins Server
See Changes: [github] KAFKA-5636: SlidingWindows (KIP-450) (#9039) -- [...truncated 3.25 MB...] org.apache.kafka.streams.test.OutputVerifierTest > shouldFai

Build failed in Jenkins: Kafka » kafka-trunk-jdk11 #38

2020-08-31 Thread Apache Jenkins Server
See Changes: [github] KAFKA-5636: SlidingWindows (KIP-450) (#9039) -- [...truncated 3.25 MB...] org.apache.kafka.streams.TopologyTestDriverTest > shouldUseS

Can't find any sendfile system call trace from Kafka process?

2020-08-31 Thread Ming Liu
Hi Kafka dev community, As we know, one major reason that Kafka is fast is because it is using sendfile() for zero copy, as what it described at https://kafka.apache.org/documentation/#producerconfigs, *This combination of pagecache and sendfile means that on a Kafka cluster where the consu

[DISCUSS] KIP-665 Kafka Connect Hash SMT

2020-08-31 Thread Brandon Brown
Hey everybody, I’ve created the following and would love some feedback. One place where this could be of use would be to say hashing the key used as an identifier for inserting into elasticsearch (which has a size limit) or obfuscating sensitive values like say passwords or ssn. https://cwiki.

Re: [DISCUSS] KIP-664: Provide tooling to detect and abort hanging transactions

2020-08-31 Thread Guozhang Wang
Thanks Jason, I do not have more comments on the KIP then. On Mon, Aug 31, 2020 at 3:19 PM Jason Gustafson wrote: > > Hmm, but the "TxnStartOffset" is not included in the DescribeProducers > response either? > > Oh, I accidentally called it `CurrentTxnStartTimestamp` in the schema. > Fixed now!

Re: [DISCUSS] KIP-664: Provide tooling to detect and abort hanging transactions

2020-08-31 Thread Jason Gustafson
> Hmm, but the "TxnStartOffset" is not included in the DescribeProducers response either? Oh, I accidentally called it `CurrentTxnStartTimestamp` in the schema. Fixed now! -Jason On Mon, Aug 31, 2020 at 3:04 PM Guozhang Wang wrote: > On Mon, Aug 31, 2020 at 12:28 PM Jason Gustafson > wrote: >

Re: [DISCUSS] KIP-664: Provide tooling to detect and abort hanging transactions

2020-08-31 Thread Guozhang Wang
On Mon, Aug 31, 2020 at 12:28 PM Jason Gustafson wrote: > Hey Guozhang, > > Thanks for the detailed comments. Responses inline: > > > 1. I'd like to clarify how we can make "--abort" work with old brokers, > since without the additional field "Partitions" the tool needs to set the > coordinator e

[jira] [Created] (KAFKA-10453) Backport of PR-7781

2020-08-31 Thread Niketh Sabbineni (Jira)
Niketh Sabbineni created KAFKA-10453: Summary: Backport of PR-7781 Key: KAFKA-10453 URL: https://issues.apache.org/jira/browse/KAFKA-10453 Project: Kafka Issue Type: Wish Compon

Re: [DISCUSS] KIP-659: Improve TimeWindowedDeserializer and TimeWindowedSerde to handle window size

2020-08-31 Thread Leah Thomas
Hey Sophie, Thanks for the catch! It makes sense that the consumer would accept a deserializer somewhere, so we can definitely skip the additional configs. I updated the KIP to reflect that. John seems to know Scala better than I do as well, but I think we need to keep the current implicit that a

[jira] [Created] (KAFKA-10452) Only expire preferred read replica if a leader is alive for the topic

2020-08-31 Thread Jeff Kim (Jira)
Jeff Kim created KAFKA-10452: Summary: Only expire preferred read replica if a leader is alive for the topic Key: KAFKA-10452 URL: https://issues.apache.org/jira/browse/KAFKA-10452 Project: Kafka

Re: [DISCUSS] KIP-664: Provide tooling to detect and abort hanging transactions

2020-08-31 Thread Jason Gustafson
Hi Bob, Thanks for the comment. > I'm not sure how much value the MaxActiveTransactionDuration metric adds, given that we have the --find-hanging option in the tool. As you mention, instances of these transactions are expected to be rare, and a partition-level metric, which can generate a lot of

[GitHub] [kafka-site] ewencp merged pull request #300: [MINOR] adding Itau Unibanco and OTICS to the powered-by page

2020-08-31 Thread GitBox
ewencp merged pull request #300: URL: https://github.com/apache/kafka-site/pull/300 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

Re: [DISCUSS] KIP-664: Provide tooling to detect and abort hanging transactions

2020-08-31 Thread Jason Gustafson
Hey Guozhang, Thanks for the detailed comments. Responses inline: > 1. I'd like to clarify how we can make "--abort" work with old brokers, since without the additional field "Partitions" the tool needs to set the coordinator epoch correctly instead of "-1"? Arguably that's still doable but would

[jira] [Resolved] (KAFKA-10384) Separate converters from generated messages

2020-08-31 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10384?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-10384. -- Fix Version/s: 2.7.0 Resolution: Fixed > Separate converters from generated messages >

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-08-31 Thread Satish Duggana
KIP is updated with - Remote log segment metadata topic message format/schema. - Added remote log segment metadata state transitions and explained how the deletion of segments is handled, including the case of partition deletions. - Added a few more limitations in the "Non goals" section. Thanks,

Re: Request to subscribe to kafka mailing list

2020-08-31 Thread Guozhang Wang
Hi SaiTejia, You can add yourself to the mailing list, it's self service: https://kafka.apache.org/contact Guozhang On Sat, Aug 29, 2020 at 8:35 AM SaiTeja Ramisetty wrote: > Regards, > SaiTeja - Data Engineer > -- -- Guozhang

[jira] [Created] (KAFKA-10451) system tests send large command over ssh instead of using remote file for security config

2020-08-31 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-10451: - Summary: system tests send large command over ssh instead of using remote file for security config Key: KAFKA-10451 URL: https://issues.apache.org/jira/browse/KAFKA-10451

Re: There is an error on the main page

2020-08-31 Thread Sophie Blee-Goldman
Thanks for the heads up. Would you be interested in submitting a PR to fix this? The typo seems to appear in two files, quickstart-docker.html and quickstart-zookeeper.html, of the kafka-site repo . On Mon, Aug 31, 2020 at 8:43 AM Liu Lycos wrote: > Hello,

[jira] [Created] (KAFKA-10450) console-producer throws Uncaught error in kafka producer I/O thread:  (org.apache.kafka.clients.producer.internals.Sender) java.lang.IllegalStateException: There are no

2020-08-31 Thread Jigar Naik (Jira)
Jigar Naik created KAFKA-10450: -- Summary: console-producer throws Uncaught error in kafka producer I/O thread:  (org.apache.kafka.clients.producer.internals.Sender) java.lang.IllegalStateException: There are no in-flight requests for node -1

Re: Permission to create KIP

2020-08-31 Thread Jun Rao
Hi, Sorin, Thanks for your interest. Just gave you the wiki permission. Jun On Sat, Aug 29, 2020 at 8:35 AM sorin wrote: > Hi guys I just wanted to propose an addition to the Consumer API to add > a new poll method which would also accept a collection of paused > partitions to automatically do

There is an error on the main page

2020-08-31 Thread Liu Lycos
Hello, I'm a developer. When I wanted to learn Kafka, I found the official website. I found that there was an error code, the target location --> http://kafka.apache.org/documentation/#quickstart --> 1. GETTING STARTED --> 1.3 Quick Start --> STEP 7: PROCESS YOUR EVENTS WITH KAFKA STREAMS The

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-08-31 Thread Unmesh Joshi
>>The reason for including LeaseStartTimeMs in the request is to ensure that the time required to communicate with the controller gets included in >>the lease time. Since requests can potentially be delayed in the network for a long time, this is important. The network time will be added anyway, b

Re: [VOTE] KIP-635: GetOffsetShell: support for multiple topics and consumer configuration override

2020-08-31 Thread Dániel Urbán
Hello everyone, I'd like to ask you to consider voting for this KIP, it only needs 2 more binding votes. This KIP focuses on the GetOffsetShell tool. The tool is useful for monitoring and investigations as well. Unfortunately, the tool misses one key, and some quality-of-life features: - Because

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-08-31 Thread Colin McCabe
On Sat, Aug 29, 2020, at 01:12, Unmesh Joshi wrote: > >>>Can you repeat your questions about broker leases? > > The LeaseStartTimeMs is expected to be the broker's > 'System.currentTimeMillis()' at the point of the request. The active > controller will add its lease period to this in order >>>

Re: [VOTE] KIP-662: Throw Exception when Source Topics of a Streams App are Deleted

2020-08-31 Thread Bruno Cadonna
Thank you for voting! The KIP passes with 5 votes: - 4 binding votes (Guozhang, Bill, John, and Matthias) - 1 non-binding vote (Walker) Best, Bruno On 28.08.20 02:46, Matthias J. Sax wrote: +1 (binding) On 8/27/20 1:10 PM, John Roesler wrote: Thanks, Bruno! I'm +1 (binding) -John On Thu,