Re: [DISCUSS] KIP-622 Add currentSystemTimeMs and currentStreamTimeMs to ProcessorContext

2020-06-30 Thread Boyang Chen
Thanks Will for the KIP. A couple questions and suggestions: 1. I think for new APIs to make most sense, we should add a minimal example demonstrating how it could be useful to structure unit tests w/o the new APIs. 2. If this is a testing-only feature, could we only add it to

Re: [VOTE] KIP-623: Add "internal-topics" option to streams application reset tool

2020-06-30 Thread Boyang Chen
Hey Bruno, I agree adding a prompt would be a nice precaution, but it is not backward compatible as you suggested and could make the automation harder to achieve. If you want, we may consider starting a separate ticket to discuss whether adding a prompt to let users be aware of the topics that

Re: [DISCUSS] KIP-622 Add currentSystemTimeMs and currentStreamTimeMs to ProcessorContext

2020-06-30 Thread William Bottrell
Thanks, John! I made the change. How much longer should I let there be discussion before starting a VOTE? On Sat, Jun 27, 2020 at 6:50 AM John Roesler wrote: > Thanks, Will, > > That looks good to me. I would only add "cached" or something > to indicate that it wouldn't just transparently look

Re: 回复: [DISCUSS] KIP-308: GetOffsetShell: new KafkaConsumer API, support for multiple topics, minimize the number of requests to server

2020-06-30 Thread wang120445...@sina.com
maybe it just likes RBAC’s show tables; wang120445...@sina.com 发件人: Hu Xi 发送时间: 2020-06-30 23:04 收件人: dev@kafka.apache.org 主题: 回复: [DISCUSS] KIP-308: GetOffsetShell: new KafkaConsumer API, support for multiple topics, minimize the number of requests to server That's a great KIP for

Kafka Exactly-Once Semantics in .NET support

2020-06-30 Thread Saher Ahwal
Hi I am working on exactly-once semantics with Kafka and I have streaming scenario of read-process-write. I noticed the new exactly-once scalability design with correctness in case of partition reassignment here:

[jira] [Reopened] (KAFKA-10166) Excessive TaskCorruptedException seen in testing

2020-06-30 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10166?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman reopened KAFKA-10166: - Assignee: (was: Bruno Cadonna) Found two edge cases we missed earlier so

Permission to create a KIP

2020-06-30 Thread Mohamed Chebbi
Hi could somone grant permission to create a KIP to user mhmdchebbi? Best Reguards

Re: [DISCUSS] KIP-554: Add Broker-side SCRAM Config API

2020-06-30 Thread Colin McCabe
Hi Rajini, OK. Let's remove the encrypted credentials from ListScramUsersResponse and the associated API. I have updated the KIP-- take a look when you get a chance. best, Colin On Fri, May 15, 2020, at 06:54, Rajini Sivaram wrote: > Hi Colin, > > We have used different approaches for

Re: Running system tests on mac

2020-06-30 Thread Gokul Ramanan Subramanian
Thanks Colin. While at the subject of system tests, there are a few times I see tests timed out (even on a large machine such as m5.4xlarge EC2 with Linux). Are there any knobs that system tests provide to control timeouts / throughputs across all tests? Thanks. On Tue, Jun 30, 2020 at 6:32 PM

Permission to create a KIP

2020-06-30 Thread Jeremy Custenborder
Hello All, Could someone grant permissions to create a KIP to the user jcustenborder? Thanks!

[jira] [Resolved] (KAFKA-4996) Fix findbugs multithreaded correctness warnings for streams

2020-06-30 Thread Leah Thomas (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-4996?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Leah Thomas resolved KAFKA-4996. Resolution: Fixed > Fix findbugs multithreaded correctness warnings for streams >

Build failed in Jenkins: kafka-trunk-jdk11 #1608

2020-06-30 Thread Apache Jenkins Server
See Changes: [github] MINOR: Fix typo in ssl.client.auth config doc description (#8956) [github] MINOR: Update AlterConfigsOptions Javadoc (#8958) [github] KAFKA-10200: Fix testability of PAPI with windowed

Build failed in Jenkins: kafka-trunk-jdk8 #4681

2020-06-30 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10200: Fix testability of PAPI with windowed stores (#8927) [github] KAFKA-4996: Fix findbugs multithreaded correctness warnings for streams [github] MINOR: Do not swallow exception

Build failed in Jenkins: kafka-trunk-jdk14 #257

2020-06-30 Thread Apache Jenkins Server
See Changes: [github] MINOR: Update AlterConfigsOptions Javadoc (#8958) [github] KAFKA-10200: Fix testability of PAPI with windowed stores (#8927) [github] KAFKA-4996: Fix findbugs multithreaded correctness

Re: Running system tests on mac

2020-06-30 Thread Colin McCabe
Ducktape runs on Python 2. You can't use it with Python 3, as you are trying to do here. If anyone's interested in porting it to Python 3 it would be a good change. Otherwise, using docker as suggested here seems to be the best way to go. best, Colin On Mon, Jun 29, 2020, at 02:14, Gokul

Re: [DISCUSS] KIP-308: GetOffsetShell: new KafkaConsumer API, support for multiple topics, minimize the number of requests to server

2020-06-30 Thread Dániel Urbán
That's a good question. In the PR I submitted, it would result in a list of partitions contained by a topic for which the user has DESCRIBE privilege. The tool utilizes Consumer.listTopics, so unauthorized topics are not present in the response at all. The current version in trunk simply reports

????,????????????kafka-API??????????????????.

2020-06-30 Thread Koray
,??API,??,. ?? : KStreamBuilder??,??from??. ??kafka 10.0..,??,??. :

回复: [DISCUSS] KIP-308: GetOffsetShell: new KafkaConsumer API, support for multiple topics, minimize the number of requests to server

2020-06-30 Thread Hu Xi
That's a great KIP for GetOffsetShell tool. I have a question about the multiple-topic lookup situation. In a secured environment, what does the tool output if it has DESCRIBE privileges for some topics but hasn't for others? 发件人: Dániel Urbán 发送时间: 2020年6月30日

Re: [DISCUSS] KIP-308: GetOffsetShell: new KafkaConsumer API, support for multiple topics, minimize the number of requests to server

2020-06-30 Thread Dániel Urbán
Hi Manikumar, Thanks, went ahead and assigned a new ID, it is KIP-635 now: https://cwiki.apache.org/confluence/display/KAFKA/KIP-635%3A+GetOffsetShell%3A+support+for+multiple+topics+and+consumer+configuration+override Daniel Manikumar ezt írta (időpont: 2020. jún. 30., K, 16:03): > Hi, > > Yes,

Re: [DISCUSS] KIP-308: GetOffsetShell: new KafkaConsumer API, support for multiple topics, minimize the number of requests to server

2020-06-30 Thread Manikumar
Hi, Yes, we can assign new id to this KIP. Thanks. On Tue, Jun 30, 2020 at 6:59 PM Dániel Urbán wrote: > Hi, > > To help with the discussion, I also have a PR for this KIP now. reflecting > the current state of the KIP: https://github.com/apache/kafka/pull/8957. > I would like to ask a

Build failed in Jenkins: kafka-trunk-jdk8 #4679

2020-06-30 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9893: Configurable TCP connection timeout and improve the initial -- [...truncated 3.16 MB...]

Build failed in Jenkins: kafka-trunk-jdk14 #255

2020-06-30 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9893: Configurable TCP connection timeout and improve the initial -- [...truncated 3.18 MB...]

Re: [DISCUSS] KIP-308: GetOffsetShell: new KafkaConsumer API, support for multiple topics, minimize the number of requests to server

2020-06-30 Thread Dániel Urbán
Hi, To help with the discussion, I also have a PR for this KIP now. reflecting the current state of the KIP: https://github.com/apache/kafka/pull/8957. I would like to ask a committer to start the test job on it. One thing I realised though is that there is a KIP id collision, there is another

[jira] [Created] (KAFKA-10221) Backport fix for KAFKA-9603 to 2.5

2020-06-30 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-10221: - Summary: Backport fix for KAFKA-9603 to 2.5 Key: KAFKA-10221 URL: https://issues.apache.org/jira/browse/KAFKA-10221 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-10220) NPE when describing resources

2020-06-30 Thread Edoardo Comar (Jira)
Edoardo Comar created KAFKA-10220: - Summary: NPE when describing resources Key: KAFKA-10220 URL: https://issues.apache.org/jira/browse/KAFKA-10220 Project: Kafka Issue Type: Bug

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-06-30 Thread Yuriy Badalyantc
Hi everybody! Looks like a discussion about KIP-513 could take a while. I think we should move forward with KIP-616 without waiting for KIP-513. I created a new pull request for KIP-616: https://github.com/apache/kafka/pull/8955. It contains a new

Build failed in Jenkins: kafka-2.5-jdk8 #161

2020-06-30 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10212: Describing a topic with the TopicCommand fails if -- [...truncated 2.93 MB...] org.apache.kafka.streams.test.OutputVerifierTest >

Re: [VOTE] KIP-623: Add "internal-topics" option to streams application reset tool

2020-06-30 Thread Bruno Cadonna
Hi, I have already brought this up in the discussion thread. Should we not run a dry-run in any case to avoid inadvertently deleting topics of other applications? I know it is a backward incompatible change if users use it in scripts, but I think it is still worth discussing it. I would to hear

Build failed in Jenkins: kafka-2.4-jdk8 #230

2020-06-30 Thread Apache Jenkins Server
See Changes: [konstantine] KAFKA-9509: Increase timeout when consuming records to fix flaky test in -- [...truncated 7.68 MB...]

Build failed in Jenkins: kafka-trunk-jdk14 #254

2020-06-30 Thread Apache Jenkins Server
See Changes: [github] make produce-sync flush (#8925) -- [...truncated 3.19 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

Re: [DISCUSS] KIP-308: GetOffsetShell: new KafkaConsumer API, support for multiple topics, minimize the number of requests to server

2020-06-30 Thread Dániel Urbán
Hi Manikumar, Thanks for the comments. 1. Will change this - thought that "command-config" is used for admin clients. 2. It's not necessary, just felt like a nice quality-of-life feature - will remove it. Thanks, Daniel On Tue, Jun 30, 2020 at 4:16 AM Manikumar wrote: > Hi Daniel, > > Thanks

Build failed in Jenkins: kafka-trunk-jdk8 #4678

2020-06-30 Thread Apache Jenkins Server
See Changes: [github] make produce-sync flush (#8925) -- [...truncated 3.16 MB...] org.apache.kafka.streams.test.ConsumerRecordFactoryTest >

Build failed in Jenkins: kafka-trunk-jdk11 #1606

2020-06-30 Thread Apache Jenkins Server
See Changes: [github] make produce-sync flush (#8925) -- [...truncated 3.18 MB...] org.apache.kafka.streams.test.OutputVerifierTest >