[jira] [Created] (KAFKA-10203) Rolling upgrade from 2.1.1 to trunk (2.7.x) doesn't work

2020-06-25 Thread John Roesler (Jira)
John Roesler created KAFKA-10203:


 Summary: Rolling upgrade from 2.1.1 to trunk (2.7.x) doesn't work
 Key: KAFKA-10203
 URL: https://issues.apache.org/jira/browse/KAFKA-10203
 Project: Kafka
  Issue Type: Bug
Reporter: John Roesler


As part of KAFKA-10173, I converted the upgrade test to use the SmokeTest 
application and also added new upgrade paths, including from 2.1.1 to trunk.

It is a rolling bounce scenario.

After the first instance upgrades from 2.1.1 to trunk, I observe the following 
on _both_ instances that are still on 2.1.1:
{code:java}
org.apache.kafka.streams.errors.TaskAssignmentException: stream-thread 
[SmokeTest-ed7632fc-3465-4534-8a26-2f6ad76ff80f-StreamThread-2-consumer] Number 
of assigned partitions 11 is not equal to the number of active taskIds 7, 
assignmentInfo=[version=4, supported version=4, active tasks=[0_0, 0_2, 2_0, 
1_1, 1_3, 2_2, 3_3], standby tasks={0_0=[data-0], 1_0=[max-0, min-0], 
0_1=[data-1], 0_2=[data-2], 2_0=[sum-0, cnt-0], 1_1=[max-1, min-1], 1_2=[min-2, 
max-2], 0_3=[data-3], 2_1=[cnt-1, sum-1], 
3_0=[SmokeTest-cntByCnt-repartition-0], 0_4=[data-4], 
3_1=[SmokeTest-cntByCnt-repartition-1], 2_2=[cnt-2, sum-2], 1_3=[min-3, max-3], 
3_2=[SmokeTest-cntByCnt-repartition-2], 1_4=[max-4, min-4], 2_3=[sum-3, cnt-3], 
2_4=[sum-4, cnt-4], 3_3=[SmokeTest-cntByCnt-repartition-3], 
3_4=[SmokeTest-cntByCnt-repartition-4]}, global assignment={}]
at 
org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor.processVersionOneAssignment(StreamsPartitionAssignor.java:892)
at 
org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor.processVersionTwoAssignment(StreamsPartitionAssignor.java:908)
at 
org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor.processVersionThreeAssignment(StreamsPartitionAssignor.java:925)
at 
org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor.processVersionFourAssignment(StreamsPartitionAssignor.java:932)
at 
org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor.onAssignment(StreamsPartitionAssignor.java:872)
at 
org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.onJoinComplete(ConsumerCoordinator.java:281)
at 
org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:406)
at 
org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:340)
at 
org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:341)
at 
org.apache.kafka.clients.consumer.KafkaConsumer.updateAssignmentMetadataIfNeeded(KafkaConsumer.java:1214)
at 
org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1179)
at 
org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1164)
at 
org.apache.kafka.streams.processor.internals.StreamThread.pollRequests(StreamThread.java:913)
at 
org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:818)
at 
org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:777)
at 
org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:747)
{code}
the other 2.1.1 instance reports the same exception, with "Number of assigned 
partitions 9 is not equal to the number of active taskIds 6"



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


[jira] [Created] (KAFKA-10202) SmokeTest should provide a machanism to generate continuously AND verify the result

2020-06-25 Thread John Roesler (Jira)
John Roesler created KAFKA-10202:


 Summary: SmokeTest should provide a machanism to generate 
continuously AND verify the result
 Key: KAFKA-10202
 URL: https://issues.apache.org/jira/browse/KAFKA-10202
 Project: Kafka
  Issue Type: Improvement
  Components: streams, system tests
Reporter: John Roesler


Several system tests use the SmokeTestDriver, but they have to choose between 
generating for a fixed period of time (say, two minutes) before verifying 
results OR generating until cancelled with no verification.

It's not impossible to implement "generate until cancelled and then verify", 
and doing so would both speed up tests that are just blocked by the two-minute 
generation period AND decrease flakiness in other tests, when they can't 
complete all their operations within the deadline before verification begins.

One ides is to revamp the smoke test driver's verification logic to make it 
consume the input and determine the expected results, so that the verifier can 
be run as a separate process instead of passing the expected results straight 
from the generator to the verifier. This is how the RelationalSmokeTest works.

An alternative is to register a signal handler so that we can signal the driver 
to stop generating and start verifying.

 



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


Re: [VOTE] KIP-627: Expose Trogdor-specific JMX Metrics for Tasks and Agents

2020-06-25 Thread Colin McCabe
+1 (binding).

Thanks, Sam.

best,
Colin


On Thu, Jun 25, 2020, at 18:05, Gwen Shapira wrote:
> +1 (binding)
> 
> Thank you, Sam. It is great to see Trogdor getting the care it deserves.
> 
> On Mon, Jun 22, 2020, 1:46 PM Sam Pal  wrote:
> 
> > Hi all,
> >
> > I would like to start a vote for KIP-627, which adds metrics about active
> > agents and the number of created, running, and done tasks in a Trogdor
> > cluster:
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-627%3A+Expose+Trogdor-specific+JMX+Metrics+for+Tasks+and+Agents
> >
> > Looking forward to hearing from you all!
> >
> > Best,
> > Sam
> >
> >
>


Re: [VOTE] KIP-629: Use racially neutral terms in our codebase

2020-06-25 Thread Jay Kreps
+1

On Thu, Jun 25, 2020 at 6:39 PM Bill Bejeck  wrote:

> Thanks for this KIP Xavier.
>
> +1(binding)
>
> -Bill
>
> On Thu, Jun 25, 2020 at 9:04 PM Gwen Shapira  wrote:
>
> > +1 (binding)
> >
> > Thank you Xavier!
> >
> > On Thu, Jun 25, 2020, 3:44 PM Xavier Léauté  wrote:
> >
> > > Hi Everyone,
> > >
> > > I would like to initiate the voting process for KIP-629.
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-629%3A+Use+racially+neutral+terms+in+our+codebase
> > >
> > > Thank you,
> > > Xavier
> > >
> >
>


Re: [VOTE] KIP-629: Use racially neutral terms in our codebase

2020-06-25 Thread Bill Bejeck
Thanks for this KIP Xavier.

+1(binding)

-Bill

On Thu, Jun 25, 2020 at 9:04 PM Gwen Shapira  wrote:

> +1 (binding)
>
> Thank you Xavier!
>
> On Thu, Jun 25, 2020, 3:44 PM Xavier Léauté  wrote:
>
> > Hi Everyone,
> >
> > I would like to initiate the voting process for KIP-629.
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-629%3A+Use+racially+neutral+terms+in+our+codebase
> >
> > Thank you,
> > Xavier
> >
>


Re: [VOTE] KIP-627: Expose Trogdor-specific JMX Metrics for Tasks and Agents

2020-06-25 Thread Gwen Shapira
+1 (binding)

Thank you, Sam. It is great to see Trogdor getting the care it deserves.

On Mon, Jun 22, 2020, 1:46 PM Sam Pal  wrote:

> Hi all,
>
> I would like to start a vote for KIP-627, which adds metrics about active
> agents and the number of created, running, and done tasks in a Trogdor
> cluster:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-627%3A+Expose+Trogdor-specific+JMX+Metrics+for+Tasks+and+Agents
>
> Looking forward to hearing from you all!
>
> Best,
> Sam
>
>


Re: [VOTE] KIP-629: Use racially neutral terms in our codebase

2020-06-25 Thread Gwen Shapira
+1 (binding)

Thank you Xavier!

On Thu, Jun 25, 2020, 3:44 PM Xavier Léauté  wrote:

> Hi Everyone,
>
> I would like to initiate the voting process for KIP-629.
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-629%3A+Use+racially+neutral+terms+in+our+codebase
>
> Thank you,
> Xavier
>


Re: [VOTE] KIP-629: Use racially neutral terms in our codebase

2020-06-25 Thread Guozhang Wang
+1 (binding).


On Thu, Jun 25, 2020 at 4:32 PM John Roesler  wrote:

> Thanks, Xavier!
>
> I’m +1 (binding)
>
> -John
>
> On Thu, Jun 25, 2020, at 17:43, Xavier Léauté wrote:
> > Hi Everyone,
> >
> > I would like to initiate the voting process for KIP-629.
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-629%3A+Use+racially+neutral+terms+in+our+codebase
> >
> > Thank you,
> > Xavier
> >
>


-- 
-- Guozhang


Access to create KIP

2020-06-25 Thread Charlie Getzen
Hello,

I would like to create a KIP around
https://github.com/apache/kafka/pull/8930 as it changes the public
interface.

My username is wiki cgetzen and it is registered under this email.

Thanks!
Charlie


Re: [VOTE] KIP-629: Use racially neutral terms in our codebase

2020-06-25 Thread John Roesler
Thanks, Xavier!

I’m +1 (binding)

-John

On Thu, Jun 25, 2020, at 17:43, Xavier Léauté wrote:
> Hi Everyone,
> 
> I would like to initiate the voting process for KIP-629.
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-629%3A+Use+racially+neutral+terms+in+our+codebase
> 
> Thank you,
> Xavier
>


[VOTE] KIP-629: Use racially neutral terms in our codebase

2020-06-25 Thread Xavier Léauté
Hi Everyone,

I would like to initiate the voting process for KIP-629.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-629%3A+Use+racially+neutral+terms+in+our+codebase

Thank you,
Xavier


[jira] [Created] (KAFKA-10201) Update codebase to use more inclusive terms

2020-06-25 Thread Jira
Xavier Léauté created KAFKA-10201:
-

 Summary: Update codebase to use more inclusive terms
 Key: KAFKA-10201
 URL: https://issues.apache.org/jira/browse/KAFKA-10201
 Project: Kafka
  Issue Type: Improvement
Reporter: Xavier Léauté
 Fix For: 2.7.0


see the corresponding KIP 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-629:+Use+racially+neutral+terms+in+our+codebase



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


KAFKA-10194 and contributor list

2020-06-25 Thread Mohamed Chebbi

Hi Kafka Team

i want to be added to the contributor list so i can Work on KAFKA-10194.

my JIRA Username is mhmdchebbi.


Cdt

Mohamed Chebbi



Re: [DISCUSS] KIP-629: Use racially neutral terms in our codebase

2020-06-25 Thread Xavier Léauté
> Probably obvious but is documentation and website considered as well as
> part of the KIP?
>

Documentation and website changes don't require a KIP to my knowledge,
however we should also update them as needed (beyond the obvious
documentation updates for the configuration names).


[jira] [Created] (KAFKA-10200) MockProcessorContext doesn't work with WindowStores

2020-06-25 Thread John Roesler (Jira)
John Roesler created KAFKA-10200:


 Summary: MockProcessorContext doesn't work with WindowStores
 Key: KAFKA-10200
 URL: https://issues.apache.org/jira/browse/KAFKA-10200
 Project: Kafka
  Issue Type: Bug
  Components: streams, streams-test-utils
Reporter: John Roesler
Assignee: John Roesler


The recommended pattern for testing custom Processor implementations is to use 
the test-utils MockProcessorContext. If a Processor implementation needs a 
store, the store also has to be initialized with the same context. However, the 
existing (in-memory and persistent) Windowed store implementations perform 
internal casts that result in class cast exceptions if you attempt to 
initialize them with the MockProcessorContext.

A workaround is to instead embed the processor in an application and use the 
TopologyTestDriver instead.



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


[jira] [Created] (KAFKA-10199) Separate state restoration into separate threads

2020-06-25 Thread Guozhang Wang (Jira)
Guozhang Wang created KAFKA-10199:
-

 Summary: Separate state restoration into separate threads
 Key: KAFKA-10199
 URL: https://issues.apache.org/jira/browse/KAFKA-10199
 Project: Kafka
  Issue Type: Improvement
  Components: streams
Reporter: Guozhang Wang
Assignee: Guozhang Wang


As part of the restoration optimization effort, we would like to move the 
restoration process to separate threads such that:

1. Stream threads would not be restricted by the main consumer `poll` frequency 
to keep as part of the group.

2. We can allow larger batches of data to be written into the restoration.

Besides this, we'd also like to fix the known issues that for piggy-backed 
source topics as changelog topics, the serde exception / extra processing logic 
would be skipped.

We would also cleanup the global update tasks as part of this effort to 
consolidate to the separate restoration threads, and would also gear them up 
with corresponding monitoring metrics (KIPs in progress).



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


[GitHub] [kafka-site] wkodate opened a new pull request #271: MINOR: Fix table of contents in protocol page

2020-06-25 Thread GitBox


wkodate opened a new pull request #271:
URL: https://github.com/apache/kafka-site/pull/271


   I deleted a part of the table of contents in protocol.html because this page 
doesn't seem to contain "Evolving the Protocol" section.
   https://kafka.apache.org/protocol.html#protocol_evolution



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




Re: [DISCUSS] Apache Kafka 2.6.0 release

2020-06-25 Thread Randall Hauch
Thanks for the update, folks!

Based upon Jira [1], we currently have 4 issues that are considered
blockers for the 2.6.0 release and production of RCs:

   - https://issues.apache.org/jira/browse/KAFKA-10134 - High CPU issue
   during rebalance in Kafka consumer after upgrading to 2.5 (unassigned)
   - https://issues.apache.org/jira/browse/KAFKA-10143 - Can no longer
   change replication throttle with reassignment tool (Jason G)
   - https://issues.apache.org/jira/browse/KAFKA-10166 - Excessive
   TaskCorruptedException seen in testing (Sophie, Bruno)
   - https://issues.apache.org/jira/browse/KAFKA-10173
   - BufferUnderflowException during Kafka Streams Upgrade (John R)

and one critical issue that may be a regression that at this time will not
block production of RCs:

   - https://issues.apache.org/jira/browse/KAFKA-10017 - Flaky Test
   EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta (Matthias)

and one build/release issue we'd like to fix if possible but will not block
RCs or the release:

   - https://issues.apache.org/jira/browse/KAFKA-9381
   - kafka-streams-scala: Javadocs + Scaladocs not published on maven central
   (me)

I'm working with the assignees and reporters of these issues (via comments
on the issues) to identify an ETA and to track progress. Anyone is welcome
to chime in on those issues.

At this time, no other changes (other than PRs that only fix/improve tests)
should be merged to the `2.6` branch. If you think you've identified a new
blocker issue or believe another existing issue should be treated as a
blocker for 2.6.0, please mark the issue's `fix version` as `2.6.0` _and_
respond to this thread with details, and I will work with you to determine
whether it is indeed a blocker.

As always, let me know here if you have any questions/concerns.

Best regards,

Randall

[1] https://issues.apache.org/jira/projects/KAFKA/versions/12346918

On Thu, Jun 25, 2020 at 8:27 AM Mario Molina  wrote:

> Hi Randal,
>
> Ticket https://issues.apache.org/jira/browse/KAFKA-9018 is not a blocker
> so
> it can be moved to the 2.7.0 version.
>
> Mario
>
> On Wed, 24 Jun 2020 at 20:22, Boyang Chen 
> wrote:
>
> > Hey Randal,
> >
> > There was another spotted blocker:
> > https://issues.apache.org/jira/browse/KAFKA-10173
> > As of current, John is working on a fix.
> >
> > Boyang
> >
> > On Wed, Jun 24, 2020 at 4:08 PM Sophie Blee-Goldman  >
> > wrote:
> >
> > > Hey all,
> > >
> > > Just a heads up that we discovered a new blocker. The fix is pretty
> > > straightforward
> > > and there's already a PR for it so it should be resolved quickly.
> > >
> > > Here's the ticket: https://issues.apache.org/jira/browse/KAFKA-10198
> > >
> > > On Sat, May 30, 2020 at 12:52 PM Randall Hauch 
> wrote:
> > >
> > > > Hi, Kowshik,
> > > >
> > > > Thanks for the update on KIP-584. This is listed on the "Postponed"
> > > section
> > > > of the AK 2.6.0 release plan (
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=152113430
> > > > ).
> > > >
> > > > Best regards,
> > > >
> > > > Randall
> > > >
> > > > On Fri, May 29, 2020 at 4:51 PM Kowshik Prakasam <
> > kpraka...@confluent.io
> > > >
> > > > wrote:
> > > >
> > > > > Hi Randall,
> > > > >
> > > > > We have to remove KIP-584 from the release plan, as this item will
> > not
> > > be
> > > > > completed for 2.6 release (although KIP is accepted). We plan to
> > > include
> > > > it
> > > > > in a next release.
> > > > >
> > > > >
> > > > > Cheers,
> > > > > Kowshik
> > > > >
> > > > >
> > > > > On Fri, May 29, 2020 at 11:43 AM Maulin Vasavada <
> > > > > maulin.vasav...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi Randall Hauch
> > > > > >
> > > > > > Can we add KIP-519 to 2.6? It was merged to Trunk already in
> April
> > -
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=128650952
> > > > > > .
> > > > > >
> > > > > > Thanks
> > > > > > Maulin
> > > > > >
> > > > > > On Fri, May 29, 2020 at 11:01 AM Randall Hauch  >
> > > > wrote:
> > > > > >
> > > > > > > Here's an update on the AK 2.6.0 release.
> > > > > > >
> > > > > > > Code freeze was Wednesday, and the release plan [1] has been
> > > updated
> > > > to
> > > > > > > reflect all of the KIPs that made the release. We've also cut
> the
> > > > `2.6`
> > > > > > > branch that we'll use for the release; see separate email
> > > announcing
> > > > > the
> > > > > > > new branch.
> > > > > > >
> > > > > > > The next important date for the 2.6.0 release is CODE FREEZE on
> > > JUNE
> > > > > 10,
> > > > > > > and until that date all bug fixes are still welcome on the
> > release
> > > > > > branch.
> > > > > > > But after that, only blocker bugs can be merged to the release
> > > > branch.
> > > > > > >
> > > > > > > If you have any questions or concerns, please contact me or
> > (better
> > > > > yet)
> > > > > > > reply to this thread.
> > > > > > >
> > > > > > > Thanks, and best regards!
> > > > 

回复: Re:Re: [ANNOUNCE] New committer: Xi Hu

2020-06-25 Thread Hu Xi
Thank you, everyone. It is my great honor to be a part of the community. Will 
make a greater contribution in the coming days.


发件人: Roc Marshal 
发送时间: 2020年6月25日 10:20
收件人: us...@kafka.apache.org 
主题: Re:Re: [ANNOUNCE] New committer: Xi Hu

Congratulations ! Xi Hu.


Best,
Roc Marshal.














At 2020-06-25 01:30:33, "Boyang Chen"  wrote:
>Congratulations Xi! Well deserved.
>
>On Wed, Jun 24, 2020 at 10:10 AM AJ Chen  wrote:
>
>> Congratulations, Xi.
>> -aj
>>
>>
>>
>> On Wed, Jun 24, 2020 at 9:27 AM Guozhang Wang  wrote:
>>
>> > The PMC for Apache Kafka has invited Xi Hu as a committer and we are
>> > pleased to announce that he has accepted!
>> >
>> > Xi Hu has been actively contributing to Kafka since 2016, and is well
>> > recognized especially for his non-code contributions: he maintains a tech
>> > blog post evangelizing Kafka in the Chinese speaking community (
>> > https://www.cnblogs.com/huxi2b/), and is one of the most active
>> answering
>> > member in Zhihu (Chinese Reddit / StackOverflow) Kafka topic. He has
>> > presented in Kafka meetup events in the past and authored a
>> > book deep-diving on Kafka architecture design and operations as well (
>> > https://www.amazon.cn/dp/B07JH9G2FL). Code wise, he has contributed 75
>> > patches so far.
>> >
>> >
>> > Thanks for all the contributions Xi. Congratulations!
>> >
>> > -- Guozhang, on behalf of the Apache Kafka PMC
>> >
>>


Re: [DISCUSS] Apache Kafka 2.6.0 release

2020-06-25 Thread Mario Molina
Hi Randal,

Ticket https://issues.apache.org/jira/browse/KAFKA-9018 is not a blocker so
it can be moved to the 2.7.0 version.

Mario

On Wed, 24 Jun 2020 at 20:22, Boyang Chen 
wrote:

> Hey Randal,
>
> There was another spotted blocker:
> https://issues.apache.org/jira/browse/KAFKA-10173
> As of current, John is working on a fix.
>
> Boyang
>
> On Wed, Jun 24, 2020 at 4:08 PM Sophie Blee-Goldman 
> wrote:
>
> > Hey all,
> >
> > Just a heads up that we discovered a new blocker. The fix is pretty
> > straightforward
> > and there's already a PR for it so it should be resolved quickly.
> >
> > Here's the ticket: https://issues.apache.org/jira/browse/KAFKA-10198
> >
> > On Sat, May 30, 2020 at 12:52 PM Randall Hauch  wrote:
> >
> > > Hi, Kowshik,
> > >
> > > Thanks for the update on KIP-584. This is listed on the "Postponed"
> > section
> > > of the AK 2.6.0 release plan (
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=152113430
> > > ).
> > >
> > > Best regards,
> > >
> > > Randall
> > >
> > > On Fri, May 29, 2020 at 4:51 PM Kowshik Prakasam <
> kpraka...@confluent.io
> > >
> > > wrote:
> > >
> > > > Hi Randall,
> > > >
> > > > We have to remove KIP-584 from the release plan, as this item will
> not
> > be
> > > > completed for 2.6 release (although KIP is accepted). We plan to
> > include
> > > it
> > > > in a next release.
> > > >
> > > >
> > > > Cheers,
> > > > Kowshik
> > > >
> > > >
> > > > On Fri, May 29, 2020 at 11:43 AM Maulin Vasavada <
> > > > maulin.vasav...@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi Randall Hauch
> > > > >
> > > > > Can we add KIP-519 to 2.6? It was merged to Trunk already in April
> -
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=128650952
> > > > > .
> > > > >
> > > > > Thanks
> > > > > Maulin
> > > > >
> > > > > On Fri, May 29, 2020 at 11:01 AM Randall Hauch 
> > > wrote:
> > > > >
> > > > > > Here's an update on the AK 2.6.0 release.
> > > > > >
> > > > > > Code freeze was Wednesday, and the release plan [1] has been
> > updated
> > > to
> > > > > > reflect all of the KIPs that made the release. We've also cut the
> > > `2.6`
> > > > > > branch that we'll use for the release; see separate email
> > announcing
> > > > the
> > > > > > new branch.
> > > > > >
> > > > > > The next important date for the 2.6.0 release is CODE FREEZE on
> > JUNE
> > > > 10,
> > > > > > and until that date all bug fixes are still welcome on the
> release
> > > > > branch.
> > > > > > But after that, only blocker bugs can be merged to the release
> > > branch.
> > > > > >
> > > > > > If you have any questions or concerns, please contact me or
> (better
> > > > yet)
> > > > > > reply to this thread.
> > > > > >
> > > > > > Thanks, and best regards!
> > > > > >
> > > > > > Randall
> > > > > >
> > > > > > [1] AK 2.6.0 Release Plan:
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=152113430
> > > > > >
> > > > > >
> > > > > > On Wed, May 27, 2020 at 5:53 PM Matthias J. Sax <
> mj...@apache.org>
> > > > > wrote:
> > > > > >
> > > > > > > Thanks Randall!
> > > > > > >
> > > > > > > I added missing KIP-594.
> > > > > > >
> > > > > > >
> > > > > > > For the postponed KIP section: I removed KIP-441 and KIP-444 as
> > > both
> > > > > are
> > > > > > > completed.
> > > > > > >
> > > > > > >
> > > > > > > -Matthias
> > > > > > >
> > > > > > > On 5/27/20 2:31 PM, Randall Hauch wrote:
> > > > > > > > Hey everyone, just a quick update on the 2.6.0 release.
> > > > > > > >
> > > > > > > > Based on the release plan (
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=152113430
> > > > > > > ),
> > > > > > > > today (May 27) is feature freeze. Any major feature work that
> > is
> > > > not
> > > > > > > > already complete will need to push out to the next release
> > > (either
> > > > > 2.7
> > > > > > or
> > > > > > > > 3.0). There are a few PRs for KIPs that are nearing
> completion,
> > > and
> > > > > > we're
> > > > > > > > having some Jenkins build issues. I will send another email
> > later
> > > > > today
> > > > > > > or
> > > > > > > > early tomorrow with an update, and I plan to cut the release
> > > branch
> > > > > > > shortly
> > > > > > > > thereafter.
> > > > > > > >
> > > > > > > > I have also updated the list of planned KIPs on the release
> > plan
> > > > > page (
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=152113430
> > > > > > > ),
> > > > > > > > and I've moved to the "Postponed" table any KIP that looks
> like
> > > it
> > > > is
> > > > > > not
> > > > > > > > going to be complete today. If any KIP is in the wrong table,
> > > > please
> > > > > > let
> > > > > > > me
> > > > > > > > know.
> > > > > > > >
> > > > > > > > If you have any questions or concerns, please feel free to

Re: [ANNOUNCE] New committer: Boyang Chen

2020-06-25 Thread Viktor Somogyi-Vass
Congrats :)

On Thu, Jun 25, 2020 at 12:28 AM Liquan Pei  wrote:

> Congrats!
>
> On Wed, Jun 24, 2020 at 9:42 AM Raymond Ng  wrote:
>
> > Congrats Boyang! Look forward to more awesome contributions from you in
> the
> > future.
> >
> > Regards,
> > Ray
> >
> > On Wed, Jun 24, 2020 at 6:07 AM Ismael Juma  wrote:
> >
> > > Congratulations Boyang!
> > >
> > > Ismael
> > >
> > > On Mon, Jun 22, 2020 at 4:26 PM Guozhang Wang 
> > wrote:
> > >
> > > > The PMC for Apache Kafka has invited Boyang Chen as a committer and
> we
> > > are
> > > > pleased to announce that he has accepted!
> > > >
> > > > Boyang has been active in the Kafka community more than two years
> ago.
> > > > Since then he has presented his experience operating with Kafka
> Streams
> > > at
> > > > Pinterest as well as several feature development including rebalance
> > > > improvements (KIP-345) and exactly-once scalability improvements
> > > (KIP-447)
> > > > in various Kafka Summit and Kafka Meetups. More recently he's also
> been
> > > > participating in Kafka broker development including post-Zookeeper
> > > > controller design (KIP-500). Besides all the code contributions,
> Boyang
> > > has
> > > > also helped reviewing even more PRs and KIPs than his own.
> > > >
> > > > Thanks for all the contributions Boyang! And look forward to more
> > > > collaborations with you on Apache Kafka.
> > > >
> > > >
> > > > -- Guozhang, on behalf of the Apache Kafka PMC
> > > >
> > >
> >
> --
> Liquan Pei
> Software Engineer, Confluent Inc
>


Jenkins build is back to normal : kafka-2.6-jdk8 #60

2020-06-25 Thread Apache Jenkins Server
See 




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

2020-06-25 Thread Apache Jenkins Server
See 


Changes:

[github] KAFKA-10198: guard against recycling dirty state (#8924)


--
[...truncated 6.36 MB...]

org.apache.kafka.streams.TopologyTestDriverTest > shouldPopulateGlobalStore[Eos 
enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldThrowIfPersistentBuiltInStoreIsAccessedWithUntypedMethod[Eos enabled = 
false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldThrowIfPersistentBuiltInStoreIsAccessedWithUntypedMethod[Eos enabled = 
false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldAllowPrePopulatingStatesStoresWithCachingEnabled[Eos enabled = false] 
STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldAllowPrePopulatingStatesStoresWithCachingEnabled[Eos enabled = false] 
PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldReturnCorrectPersistentStoreTypeOnly[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldReturnCorrectPersistentStoreTypeOnly[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > shouldRespectTaskIdling[Eos 
enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > shouldRespectTaskIdling[Eos 
enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldUseSourceSpecificDeserializers[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldUseSourceSpecificDeserializers[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > shouldReturnAllStores[Eos 
enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > shouldReturnAllStores[Eos 
enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldNotCreateStateDirectoryForStatelessTopology[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldNotCreateStateDirectoryForStatelessTopology[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldApplyGlobalUpdatesCorrectlyInRecursiveTopologies[Eos enabled = false] 
STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldApplyGlobalUpdatesCorrectlyInRecursiveTopologies[Eos enabled = false] 
PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldReturnAllStoresNames[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldReturnAllStoresNames[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldPassRecordHeadersIntoSerializersAndDeserializers[Eos enabled = false] 
STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldPassRecordHeadersIntoSerializersAndDeserializers[Eos enabled = false] 
PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessConsumerRecordList[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessConsumerRecordList[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldUseSinkSpecificSerializers[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldUseSinkSpecificSerializers[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldFlushStoreForFirstInput[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldFlushStoreForFirstInput[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessFromSourceThatMatchPattern[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessFromSourceThatMatchPattern[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldCaptureSinkTopicNamesIfWrittenInto[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldCaptureSinkTopicNamesIfWrittenInto[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldUpdateStoreForNewKey[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldUpdateStoreForNewKey[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldSendRecordViaCorrectSourceTopicDeprecated[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldSendRecordViaCorrectSourceTopicDeprecated[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldPunctuateOnWallClockTime[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldPunctuateOnWallClockTime[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > shouldSetRecordMetadata[Eos 
enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > shouldSetRecordMetadata[Eos 
enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldNotUpdateStoreForLargerValue[Eos enabled = false] 

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

2020-06-25 Thread Apache Jenkins Server
See 


Changes:

[github] KAFKA-10198: guard against recycling dirty state (#8924)


--
[...truncated 6.36 MB...]
org.apache.kafka.streams.TestTopicsTest > testTimestamp PASSED

org.apache.kafka.streams.TestTopicsTest > testWithHeaders STARTED

org.apache.kafka.streams.TestTopicsTest > testWithHeaders PASSED

org.apache.kafka.streams.TestTopicsTest > testKeyValue STARTED

org.apache.kafka.streams.TestTopicsTest > testKeyValue PASSED

org.apache.kafka.streams.TestTopicsTest > 
shouldNotAllowToCreateTopicWithNullTopicName STARTED

org.apache.kafka.streams.TestTopicsTest > 
shouldNotAllowToCreateTopicWithNullTopicName PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordsFromKeyValuePairs STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordsFromKeyValuePairs PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithNullKeyAndDefaultTimestamp 
STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithNullKeyAndDefaultTimestamp 
PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithDefaultTimestamp 
STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithDefaultTimestamp 
PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithNullKey STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithNullKey PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateNullKeyConsumerRecordWithOtherTopicNameAndTimestampWithTimetamp 
STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateNullKeyConsumerRecordWithOtherTopicNameAndTimestampWithTimetamp 
PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithTimestamp STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithTimestamp PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullHeaders STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullHeaders PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithDefaultTimestamp STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithDefaultTimestamp PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicName STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicName PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithNullKey STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithNullKey PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecord STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecord PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateNullKeyConsumerRecord STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateNullKeyConsumerRecord PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithOtherTopicName STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithOtherTopicName PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > shouldAdvanceTime 
STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > shouldAdvanceTime 
PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithKeyValuePairs STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithKeyValuePairs PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithKeyValuePairsAndCustomTimestamps
 STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithKeyValuePairsAndCustomTimestamps
 PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithKeyValuePairs 
STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithKeyValuePairs PASSED


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

2020-06-25 Thread William Bottrell
Thanks, John! I appreciate you adjusting my lingo. I made the change to the
KIP. I will add the note about system time to the javadoc.

On Wed, Jun 24, 2020 at 6:52 PM John Roesler  wrote:

> Hi Will,
>
> This proposal looks good to me overall. Thanks for the contribution!
>
> Just a couple of minor notes:
>
> The system time method would return a cached timestamp that Streams looks
> up once when it starts processing a record. This may be confusing, so it
> might be good to state it in the javadoc.
>
> I thought the javadoc for the stream time might be a bit confusing. We
> normally talk about “Tasks” not “partition groups” in the public api. Maybe
> just saying that it’s “the maximum timestamp of any record yet processed by
> the task” would be both high level and accurate.
>
> Thanks again!
> -John
>
> On Mon, Jun 22, 2020, at 02:10, William Bottrell wrote:
> > Thanks, Bruno. I updated the KIP, so hopefully it makes more sense.
> Thanks
> > to Matthias J. Sax and Piotr Smolinski for helping with details.
> >
> > I welcome more feedback. Let me know if something doesn't make sense or I
> > need to provide more detail. Also, feel free to enlighten me. Thanks!
> >
> > On Thu, Jun 11, 2020 at 1:11 PM Bruno Cadonna 
> wrote:
> >
> > > Hi Will,
> > >
> > > Thank you for the KIP.
> > >
> > > 1. Could you elaborate a bit more on the motivation in the KIP? An
> > > example would make the motivation clearer.
> > >
> > > 2. In section "Proposed Changes" you do not need to show the
> > > implementation and describe internals. A description of the expected
> > > behavior of the newly added methods should suffice.
> > >
> > > 3. In "Compatibility, Deprecation, and Migration Plan" you should
> > > state that the change is backward compatible because the two methods
> > > will be added and no other method will be changed or removed.
> > >
> > > Best,
> > > Bruno
> > >
> > > On Wed, Jun 10, 2020 at 10:06 AM William Bottrell  >
> > > wrote:
> > > >
> > > > Add currentSystemTimeMs and currentStreamTimeMs to ProcessorContext
> > > > <
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-622%3A+Add+currentSystemTimeMs+and+currentStreamTimeMs+to+ProcessorContext
> > > >
> > > >
> > > > I am extremely new to Kafka, but thank you to John Roesler and
> Matthias
> > > J.
> > > > Sax for pointing me in the right direction. I accept any and all
> > > feedback.
> > > >
> > > > Thanks,
> > > > Will
> > >
> >
>