Re: [DISCUSS] KIP-812: Introduce another form of the `KafkaStreams.close()` API that forces the member to leave the consumer group

2022-02-18 Thread Luke Chen
Hi Seung-chan,

Thanks for the update.

One more comment:
3. Could you explain if user doesn't provide the `timeout` value in
`CloseOptions`. What will happen? I saw the default value for `timeout`
variable in `CloseOptions` is `null`. What doest null `timeout` mean?

Thank you.
Luke

On Sat, Feb 19, 2022 at 3:31 PM Seung-chan Ahn 
wrote:

> Hi Luke,
>
> Thanks for reviewing this KIP!
>
> 1. You are right, I'm only adding one method with `CloseOptions`. Let me
> comment in the KIP! -> Done
> 2. I think it makes more sense to have `Duration` type option which is
> aligned to the existing public method. Let me update the KIP! -> Done
>
> Seung-chan
>
> On Fri, Feb 18, 2022 at 3:21 PM Luke Chen  wrote:
>
> > Hi Seung-chan,
> >
> > Thanks for the KIP!
> > I like the optimization to have a close method to force leave group.
> > Some minor comments:
> > 1. in the "public interfaces", there are many `close` method listed, but
> I
> > believe you only want to introduce the one with `CloseOptions`, right?
> > Could you add some comment to it to point it out this is what you added
> in
> > this KIP?
> > 2. In the CloseOptions class variable, I saw there's a `timeoutMs` as
> type
> > `Integer`. Could you explain why we use `Integer` here, instead of `int`,
> > or `Duration`?
> >
> > Thank you.
> > Luke
> >
> > On Tue, Feb 15, 2022 at 11:19 PM Seung-chan Ahn  >
> > wrote:
> >
> > > Hi Guozhang,
> > >
> > >
> > > Thanks a lot for sharing your idea and guiding me.
> > >
> > >
> > > Thanks to your support, we've reached this voting:
> > >
> > > https://lists.apache.org/thread/toq5pg799ctd7lwdcd6g7zk6xn73h26r
> > >
> > >
> > > Seung-chan
> > >
> > > On Wed, Jan 12, 2022 at 10:50 PM Seung-chan Ahn <
> dev.issea1...@gmail.com
> > >
> > > wrote:
> > >
> > > > Hi team,
> > > >
> > > > Here's the new KIP
> > > > <
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-812%3A+Introduce+another+form+of+the+%60KafkaStreams.close%28%29%60+API+that+forces+the+member+to+leave+the+consumer+group
> > > >
> > > >  for this issue .
> > > >
> > > > The title says pretty much what this KIP is for. Even though it's my
> > > first
> > > > draft, as A. Sophie Blee-Goldman has written rich descriptions and
> > > already
> > > > the solutions in the issue thread, I've enjoyed following up on the
> > idea.
> > > >
> > > > Please feel free to review on any point!
> > > >
> > >
> >
>


Re: [DISCUSS] KIP-812: Introduce another form of the `KafkaStreams.close()` API that forces the member to leave the consumer group

2022-02-18 Thread Seung-chan Ahn
Hi Luke,

Thanks for reviewing this KIP!

1. You are right, I'm only adding one method with `CloseOptions`. Let me
comment in the KIP! -> Done
2. I think it makes more sense to have `Duration` type option which is
aligned to the existing public method. Let me update the KIP! -> Done

Seung-chan

On Fri, Feb 18, 2022 at 3:21 PM Luke Chen  wrote:

> Hi Seung-chan,
>
> Thanks for the KIP!
> I like the optimization to have a close method to force leave group.
> Some minor comments:
> 1. in the "public interfaces", there are many `close` method listed, but I
> believe you only want to introduce the one with `CloseOptions`, right?
> Could you add some comment to it to point it out this is what you added in
> this KIP?
> 2. In the CloseOptions class variable, I saw there's a `timeoutMs` as type
> `Integer`. Could you explain why we use `Integer` here, instead of `int`,
> or `Duration`?
>
> Thank you.
> Luke
>
> On Tue, Feb 15, 2022 at 11:19 PM Seung-chan Ahn 
> wrote:
>
> > Hi Guozhang,
> >
> >
> > Thanks a lot for sharing your idea and guiding me.
> >
> >
> > Thanks to your support, we've reached this voting:
> >
> > https://lists.apache.org/thread/toq5pg799ctd7lwdcd6g7zk6xn73h26r
> >
> >
> > Seung-chan
> >
> > On Wed, Jan 12, 2022 at 10:50 PM Seung-chan Ahn  >
> > wrote:
> >
> > > Hi team,
> > >
> > > Here's the new KIP
> > > <
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-812%3A+Introduce+another+form+of+the+%60KafkaStreams.close%28%29%60+API+that+forces+the+member+to+leave+the+consumer+group
> > >
> > >  for this issue .
> > >
> > > The title says pretty much what this KIP is for. Even though it's my
> > first
> > > draft, as A. Sophie Blee-Goldman has written rich descriptions and
> > already
> > > the solutions in the issue thread, I've enjoyed following up on the
> idea.
> > >
> > > Please feel free to review on any point!
> > >
> >
>


Re: [DISCUSS] KIP-794: Strictly Uniform Sticky Partitioner

2022-02-18 Thread Artem Livshits
Hello Luke, Jun,

Thank you for your feedback.  I've added the Rejected Alternative section
that may clarify some of the questions w.r.t. returning -1.

1. I've elaborated on the -1 in the KIP.  The problem is that a significant
part of the logic needs to be in the producer (because it now uses
information about brokers that only the producer knows), so encapsulation
of the logic within the default partitioner isn't as clean.   I've added
the Rejected Alternative section that documents an attempt to keep the
encapsulation by providing new callbacks to the partitioner.

2. The meaning of the partitioner.sticky.batch.size is explained in the
Uniform Sticky Batch Size section.  Basically, we track the amount of bytes
produced to the partition and if it exceeds partitioner.sticky.batch.size
then we switch to the next partition.  As far as the reason to make it
different from batch.size, I think Luke answered this with the question #3
-- what if the load pattern is such that 15.5KB would be more efficient
than 16KB?

3. I think it's hard to have one size that would fit all patterns.  E.g. if
the load pattern is such that there is linger and the app fills the batch
before linger expires, then having 16KB would most likely synchronize
batching and partition switching, so each partition would get a full
batch.  If load pattern is such that there are a few non-complete batches
go out before a larger batch starts to fill, then it may actually be
beneficial to make slightly larger (e.g. linger=0, first few records go in
the first batch, then next few records go to second batch, and so on, until
5 in-flight, then larger batch would form while waiting for broker to
respond, but the partition switch would happen before the larger batch is
full).

4. There are a couple of reasons for introducing
partition.availability.timeout.ms.  Luke's an Jun's questions are slightly
different, so I'm going to separate replies.
(Luke) Is the queue size a good enough signal?  I think it's a good default
signal as it tries to preserve general fairness and not overreact on the
broker's state at each moment in time.  But because it's smooth, it may not
be reactive enough to instantaneous latency jumps.  For latency-sensitive
workloads, it may be desirable to react faster when a broker becomes
unresponsive (but that may make the distribution really choppy), so
partition.availability.timeout.ms provides an opportunity to tune
adaptiveness.

(Jun) Can we just not assign partitions to brokers that are not ready?
Switching partitions purely based on current broker readiness information
can really skew workload I think (or at least I couldn't build a model that
proves that over time it's going to be generally fair), I feel that the
algorithm should try to be fair in general and use smoother signals by
default (e.g. a broker with choppier latency may get much less load even
though it can handle throughput, it then may potentially skew consumption),
note that the queue-size-based logic uses probabilities (so we don't fully
remove brokers, just make it less likely) and relative info rather than a
threshold (so if all brokers are heavily, but equally loaded, they will get
equal distribution, rather than get removed because they exceed some
threshold).  So at the very least, I would like this logic to be turned off
by default as it's hard to predict what it could do with different patterns
(which means that there would need to be some configuration).  We could
just not use brokers that are not ready, but again, I think that it's good
to try to be fair under normal circumstances, so if normally brokers can
respond under some partition.availability.timeout.ms threshold and the
application works well with those latencies, then we could distribute data
equally between brokers that don't exceed the latencies.  The value, of
course, would depend on the environment and app requirements, hence it's
configurable.

10. Added a statement at the beginning of the proposed changes.

-Artem


On Thu, Feb 17, 2022 at 3:46 PM Jun Rao  wrote:

> Hi, Artem,
>
> Thanks for the KIP. A few comments below.
>
> 1. I agree with Luke that having the partitioner returning -1 is kind of
> weird. Could we just change the implementation of DefaultPartitioner to the
> new behavior?
>
> 2. partitioner.sticky.batch.size: Similar question to Luke. I am not sure
> why we want to introduce this new configuration. Could we just use the
> existing batch.size?
>
> 4. I also agree with Luke that it's not clear why we need
> partition.availability.timeout.ms. The KIP says the broker "would not be
> chosen until the broker is able to accept the next ready batch from the
> partition". If we are keeping track of this, could we just avoid assigning
> records to partitions whose leader is not able to accept the next batch? If
> we do that, perhaps we don't need partition.availability.timeout.ms.
>
> 10. Currently, partitioner.class defaults to DefaultPartitioner, which uses
> StickyPartitioner w

Re: Call for review: KIP-618

2022-02-18 Thread Chris Egerton
Thanks Luke! Appreciate it.

On Thu, Feb 17, 2022 at 10:02 PM Luke Chen  wrote:

> Hi Chris,
>
> Thanks for all the PRs (wow, 10K lines of changes).
> EOS on source connector is a great feature in Kafka connect!
>
> I will start to review this PR next week:
> Producer fencing API for admin client:
> https://github.com/apache/kafka/pull/11777
>
> Thank you.
> Luke
>
> On Fri, Feb 18, 2022 at 2:36 AM Chris Egerton  >
> wrote:
>
> > Hi all,
> >
> > It's been a little over eight months since KIP-618 (
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-618%3A+Exactly-Once+Support+for+Source+Connectors
> > )
> > was approved, and nearly as long since a feature- and test-complete
> > implementation (initially published in
> > https://github.com/apache/kafka/pull/10907) has been ready for review.
> >
> > Since then, there have been two rounds of review that, combined, cover
> less
> > than half of the code base. At this rate, I estimate it won't receive a
> > complete review--let alone be approved and merged--until October this
> year,
> > at the earliest.
> >
> > I understand that review for moderately large features like this one (the
> > current diff stacks up at a little over 10K additions) is time-consuming,
> > but would it be possible to expedite things a little here?
> >
> > I've taken the suggestion of Mickael Maison and split the old, monolithic
> > PR up into several smaller ones that should be easier to review.
> >
> > These PRs are ready to review right now:
> > - Producer fencing API for admin client:
> > https://github.com/apache/kafka/pull/11777
> > - Metrics for source task transactions:
> > https://github.com/apache/kafka/pull/11772
> > - Source connector APIs: https://github.com/apache/kafka/pull/11773
> > - Utility methods used in downstream PRs:
> > https://github.com/apache/kafka/pull/11774
> >
> > These require changes either from PRs in the above list, or from others
> in
> > this list:
> > - User-facing config properties:
> > https://github.com/apache/kafka/pull/11775
> > - Preflight validation: https://github.com/apache/kafka/pull/11776
> > - Protected config topic writes by the leader:
> > https://github.com/apache/kafka/pull/11778
> > - Zombie fencing: https://github.com/apache/kafka/pull/11779
> > - Transactional source tasks: https://github.com/apache/kafka/pull/11780
> > - Per-connector offsets topics:
> https://github.com/apache/kafka/pull/11781
> >
> > And these bring together all of the above and add end-to-end testing:
> > - Embedded integration tests: https://github.com/apache/kafka/pull/11782
> > - System tests: https://github.com/apache/kafka/pull/11783
> >
> > I'd like it if we could try to get this feature merged in time for the
> > upcoming 3.2 release. If anyone believes this isn't possible, please let
> me
> > know.
> >
> > Cheers,
> >
> > Chris
> >
>


Re: [DISCUSS] KIP-820: Extend KStream process with new Processor API

2022-02-18 Thread John Roesler
Hello all,

While I sympathize with Matthias’s desire to wipe the slate clean and redesign 
the dsl with full knowledge of everything we’ve learned in the past few years, 
that would also be a pretty intense project on its own. It seems better to 
leave that project for someone who is motivated to take it on.

Reading between the lines, it seems like Jorge’s motivation is more along the 
lines of removing a few specific pain points. I appreciate Matthias extending 
the offer, but if Jorge doesn’t want to redesign the dsl right now, we’re 
better off just accepting the work he’s willing to do.

Specifically, this KIP is quite a nice improvement. Looking at the KStream 
interface, roughly half of it is devoted to various flavors of “transform”, 
which makes it really hard on users to figure out which they are supposed to 
use for what purpose. This kip let us drop all that complexity in favor of just 
two methods, thanks to the fact that we now have the ability for processors to 
specify their forwarding type. 

By the way, I really like Matthias’s suggestion to set the KOut generic bound 
to Void for processValues. Then, instead of doing an equality check on the key 
during forward, you’d just set the key back to the one saved before processing 
(with setRecordKey). This is both more efficient (because we don’t have the 
equality check) and more foolproof for users (because it’s enforced by the 
compiler instead of the runtime).

Thanks, all!
-John

On Fri, Feb 18, 2022, at 00:43, Jorge Esteban Quilcate Otoya wrote:
> On Fri, 18 Feb 2022 at 02:16, Matthias J. Sax  wrote:
>
>> > It probably deserves its own thread to start discussing ideas.
>>
>> Yes. My question was: if we think it's time to do a DSL 2.0, should we
>> drop this KIP and just fix via DSL 2.0 instead?
>>
>>
> Good question. Would love to hear what others think about this.
>
> I've stated my position about this here:
>
>> For this KIP specifically, I think about it as a continuation from
> KIP-478. Therefore, it could make sense to have it as part of the current
> version of the DSL.
>
> I'd even add that if this KIP is adopted, I would not be that disappointed
> if KIP-634 is dropped in favor of a DSL v2.0 as the access to headers
> provided by KIP-478's via Record API is much better than previous
> `.context().headers()`.
>
> But happy to reconsider if there is an agreement to focus efforts towards a
> DSL 2.0.
>
>
>> > You're right. I'm not proposing the method signature.
>>
>> What signature do you propose? I don't see an update on the KIP.
>>
>> My bad. I have clarified this in the KIP's public interfaces now:
>
> ```
>
> New methods:
>
>- KStream KStream#processValues(ProcessorSupplier
>processorSupplier, String... stateStoreNames)
>- KStream KStream#processValues(ProcessorSupplier
>processorSupplier, Named named, String... stateStoreNames)
>
> Modified methods:
>
>- KStream KStream#process(ProcessorSupplier
>processorSupplier, String... stateStoreNames)
>- KStream KStream#process(ProcessorSupplier
>processorSupplier, Named named, String... stateStoreNames)
>
> ```
>
>
>>
>> > Not sure if I understand how this would look like. Do you mean checking
>> it
>> > on the Record itself or somewhere else?
>>
>> @Guozhang: I am not worried about the runtime overhead. I am worries
>> about user experience. It's not clear from the method signature, that
>> you are not allowed to change the key, what seems to be bad API desig.
>> Even if I understand the desire to keep the API surface ares small -- I
>> would rather have a compile time enforcement than a runtime check.
>>
>> For example, we have `map()` and `mapValues()` and `mapValues()` returns
>> a `Value V` (enforces that that key is not changes) instead of a
>> `KeyValue` and we use a runtime check to check that the key is
>> not changed.
>>
>> Naively, could we enforce something similar by setting the output key
>> type as `Void`.
>>
>>KStream#processValue(ProcessorSupplier
>> processorSupplier)
>>
>> Not sure if this would work or not?
>>
>> Or it might be worth to add a new interface, `ValueProcessorSupplier`
>> that ensures that the key is not modified?
>>
>>
> This is an important discussion, even more so with a DSL v2.0.
>
> At the moment, the DSL just flags whether partitioning is required based on
> the DSL operation. As mentioned, `mapValues()` enforces only the value has
> changed through the DSL, though the only _guarantee_ we have is that Kafka
> Streams "owns" the implementation, and we can flag this properly.
>
> With a hypothetical v2.0 based on Record API, this will be harder to
> enforce with the current APIs. e.g. with `mapValues(Record record)`,
> nothing would stop users from using `record.withKey("needs_partitioning")`.
>
> The approach defined on this KIP is similar to what we have at the moment
> on `ValueTransformer*` where it validates at runtime that the users are not
> calling `forward` with `ForwardingDisabledProcessorContext`.
> `V

Jenkins build is unstable: Kafka » Kafka Branch Builder » trunk #706

2022-02-18 Thread Apache Jenkins Server
See 




[jira] [Created] (KAFKA-13676) When processing in ALOS we might as well commit progress made other tasks on a task specific exception

2022-02-18 Thread Walker Carlson (Jira)
Walker Carlson created KAFKA-13676:
--

 Summary: When processing in ALOS we might as well commit progress 
made other tasks on a task specific exception
 Key: KAFKA-13676
 URL: https://issues.apache.org/jira/browse/KAFKA-13676
 Project: Kafka
  Issue Type: Improvement
  Components: streams
Reporter: Walker Carlson
Assignee: Walker Carlson


When processing in ALOS we might as well commit progress made other tasks on a 
task specific exception. If one task has an issue and we have already 
successfully completed processing on at least one task it would be good to 
commit those successfully processed tasks. This should prevent limit the 
duplicated records downstream and also be more efficient.

Also if one task is having lots of issues the other tasks can at least make 
progress. When we introduced the thread replacement mechanism this optimization 
became possible. 



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


[jira] [Resolved] (KAFKA-13666) Tests should not ignore exceptions for supported OS

2022-02-18 Thread Bruno Cadonna (Jira)


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

Bruno Cadonna resolved KAFKA-13666.
---
Resolution: Fixed

> Tests should not ignore exceptions for supported OS
> ---
>
> Key: KAFKA-13666
> URL: https://issues.apache.org/jira/browse/KAFKA-13666
> Project: Kafka
>  Issue Type: Test
>  Components: streams
>Affects Versions: 3.0.0
>Reporter: Rob Leland
>Priority: Minor
> Fix For: 3.2.0
>
> Attachments: kafka test changes.png
>
>
> A few of the tests are swallowing exceptions for all operations systems 
> because they might fail on windows. This could mask regressions in supported 
> OS. When using the testDrivers change this so exceptions are only ignored for 
> Windows OS.
> Please see: https://github.com/apache/kafka/pull/11752



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


Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #705

2022-02-18 Thread Apache Jenkins Server
See 


Changes:


--
[...truncated 441971 lines...]
[2022-02-18T11:26:26.750Z] + mvn archetype:generate -DarchetypeCatalog=local 
-DarchetypeGroupId=org.apache.kafka 
-DarchetypeArtifactId=streams-quickstart-java -DarchetypeVersion=3.2.0-SNAPSHOT 
-DgroupId=streams.examples -DartifactId=streams.examples -Dversion=0.1 
-Dpackage=myapps
[2022-02-18T11:26:28.826Z] [INFO] Scanning for projects...
[2022-02-18T11:26:28.826Z] [INFO] 
[2022-02-18T11:26:28.826Z] [INFO] --< 
org.apache.maven:standalone-pom >---
[2022-02-18T11:26:28.826Z] [INFO] Building Maven Stub Project (No POM) 1
[2022-02-18T11:26:28.826Z] [INFO] [ pom 
]-
[2022-02-18T11:26:28.826Z] [INFO] 
[2022-02-18T11:26:28.826Z] [INFO] >>> maven-archetype-plugin:3.2.1:generate 
(default-cli) > generate-sources @ standalone-pom >>>
[2022-02-18T11:26:28.826Z] [INFO] 
[2022-02-18T11:26:28.826Z] [INFO] <<< maven-archetype-plugin:3.2.1:generate 
(default-cli) < generate-sources @ standalone-pom <<<
[2022-02-18T11:26:28.826Z] [INFO] 
[2022-02-18T11:26:28.826Z] [INFO] 
[2022-02-18T11:26:28.826Z] [INFO] --- maven-archetype-plugin:3.2.1:generate 
(default-cli) @ standalone-pom ---
[2022-02-18T11:26:30.907Z] [INFO] Generating project in Interactive mode
[2022-02-18T11:26:30.907Z] [WARNING] Archetype not found in any catalog. 
Falling back to central repository.
[2022-02-18T11:26:30.907Z] [WARNING] Add a repository with id 'archetype' in 
your settings.xml if archetype's repository is elsewhere.
[2022-02-18T11:26:30.907Z] [INFO] Using property: groupId = streams.examples
[2022-02-18T11:26:30.907Z] [INFO] Using property: artifactId = streams.examples
[2022-02-18T11:26:30.907Z] [INFO] Using property: version = 0.1
[2022-02-18T11:26:30.907Z] [INFO] Using property: package = myapps
[2022-02-18T11:26:30.907Z] Confirm properties configuration:
[2022-02-18T11:26:30.907Z] groupId: streams.examples
[2022-02-18T11:26:30.907Z] artifactId: streams.examples
[2022-02-18T11:26:30.907Z] version: 0.1
[2022-02-18T11:26:30.907Z] package: myapps
[2022-02-18T11:26:30.907Z]  Y: : [INFO] 

[2022-02-18T11:26:30.907Z] [INFO] Using following parameters for creating 
project from Archetype: streams-quickstart-java:3.2.0-SNAPSHOT
[2022-02-18T11:26:30.907Z] [INFO] 

[2022-02-18T11:26:30.907Z] [INFO] Parameter: groupId, Value: streams.examples
[2022-02-18T11:26:30.907Z] [INFO] Parameter: artifactId, Value: streams.examples
[2022-02-18T11:26:30.907Z] [INFO] Parameter: version, Value: 0.1
[2022-02-18T11:26:30.907Z] [INFO] Parameter: package, Value: myapps
[2022-02-18T11:26:30.907Z] [INFO] Parameter: packageInPathFormat, Value: myapps
[2022-02-18T11:26:30.907Z] [INFO] Parameter: package, Value: myapps
[2022-02-18T11:26:30.907Z] [INFO] Parameter: version, Value: 0.1
[2022-02-18T11:26:30.907Z] [INFO] Parameter: groupId, Value: streams.examples
[2022-02-18T11:26:30.907Z] [INFO] Parameter: artifactId, Value: streams.examples
[2022-02-18T11:26:30.907Z] [INFO] Project created from Archetype in dir: 
/home/jenkins/jenkins-agent/workspace/Kafka_kafka_trunk/streams/quickstart/test-streams-archetype/streams.examples
[2022-02-18T11:26:30.907Z] [INFO] 

[2022-02-18T11:26:30.907Z] [INFO] BUILD SUCCESS
[2022-02-18T11:26:30.907Z] [INFO] 

[2022-02-18T11:26:30.907Z] [INFO] Total time:  2.355 s
[2022-02-18T11:26:30.907Z] [INFO] Finished at: 2022-02-18T11:26:30Z
[2022-02-18T11:26:30.907Z] [INFO] 

[Pipeline] dir
[2022-02-18T11:26:31.428Z] Running in 
/home/jenkins/jenkins-agent/workspace/Kafka_kafka_trunk/streams/quickstart/test-streams-archetype/streams.examples
[Pipeline] {
[Pipeline] sh
[2022-02-18T11:26:34.376Z] + mvn compile
[2022-02-18T11:26:36.448Z] [INFO] Scanning for projects...
[2022-02-18T11:26:36.448Z] [INFO] 
[2022-02-18T11:26:36.448Z] [INFO] -< 
streams.examples:streams.examples >--
[2022-02-18T11:26:36.448Z] [INFO] Building Kafka Streams Quickstart :: Java 0.1
[2022-02-18T11:26:36.448Z] [INFO] [ jar 
]-
[2022-02-18T11:26:36.448Z] [INFO] 
[2022-02-18T11:26:36.448Z] [INFO] --- maven-resources-plugin:2.6:resources 
(default-resources) @ streams.examples ---
[2022-02-18T11:26:37.483Z] [INFO] Using 'UTF-8' encoding to copy filtered 
resources.
[2022-02-18T11:26:37.483Z] [INFO] Copying 1 resource
[2022-02-18T11:26:37.483Z] [INFO] 
[2022-02-18T11:26:37.483Z] [INFO] --- maven-compiler-plugin:3.1:compile 
(default-compile) @ streams.examples ---

Jenkins build is unstable: Kafka » Kafka Branch Builder » trunk #704

2022-02-18 Thread Apache Jenkins Server
See