Re: [DISCUSS] Apache Kafka 3.0.0 release plan with new updated dates

2021-07-22 Thread Sophie Blee-Goldman
Hey Konstantine,

A javadocs ticket of ours was demoted to a non-blocker earlier this week
due to lack of action,
but I now have a PR ready and under review. It's picking up some essential
followup that was
missed during the implementation of KIP-633 and is pretty essential. I
tagged you on the PR,
it's technically touching on a few things that aren't just docs, but only
to add a handful of checks
that already existed on the old APIs and just got missed on the new APIs.
Anything beyond that
I left as a TODO to follow up on after 3.0.

KAFKA-13021  ---
https://github.com/apache/kafka/pull/4

I think we should be able to get it merged by tomorrow. Assuming we do, can
I promote it back
to blocker status and pick the fix to the 3.0 branch?

Thanks!
Sophie

On Thu, Jul 22, 2021 at 4:29 PM Konstantine Karantasis
 wrote:

> Thanks for raising this John.
>
> While we are working to eliminate the existing blockers I think it would be
> great to use this time in order to test the upgrade path that you mention.
>
> Before we approve a release candidate (once such a RC is generated) we
> should confirm that the upgrade works as expected.
> So, I agree with you that this is not an RC generation blocker per se but
> it's a release blocker overall.
>
> Konstantine
>
>
> On Thu, Jul 22, 2021 at 4:21 PM John Roesler  wrote:
>
> > Hello Konstantine,
> >
> > Someone just called to my attention that KAFKA-12724 had not
> > been marked as a 3.0 blocker. We never added 2.8 to the
> > Streams upgrade system test suite. This isn't a blocker in
> > that it is a problem, but we should make sure that Streams
> > is actually upgradable before releasing 3.0.
> >
> > I'm sorry for the oversight. For what it's worth, I think we
> > could proceed with a release candidate while we continue to
> > address the missing system test.
> >
> > Thanks,
> > -John
> >
> > https://issues.apache.org/jira/browse/KAFKA-12724
> >
> > On Wed, 2021-07-21 at 14:00 -0700, Konstantine Karantasis
> > wrote:
> > > Thanks for the heads up Colin.
> > >
> > > KAFKA-13112 seems important and of course relevant to what we ship with
> > > 3.0.
> > > Same for the test failures captured by KAFKA-13095 and KAFKA-12851.
> > Fixing
> > > those will increase the stability of our builds.
> > >
> > > Therefore, considering these tickets as blockers currently makes sense
> to
> > > me.
> > >
> > > Konstantine
> > >
> > >
> > > On Wed, Jul 21, 2021 at 11:46 AM Colin McCabe 
> > wrote:
> > >
> > > > Hi Konstantine,
> > > >
> > > > Thanks for your work on this release! We discovered three blocker
> bugs
> > > > which are worth bringing up here:
> > > >
> > > > 1. KAFKA-13112: Controller's committed offset get out of sync with
> raft
> > > > client listener context
> > > > 2. KAFKA-13095: TransactionsTest is failing in kraft mode
> > > > 3. KAFKA-12851: Flaky Test
> > > > RaftEventSimulationTest.canMakeProgressIfMajorityIsReachable
> > > >
> > > > There are two subtasks for #1 which we are working on. We suspect
> that
> > #3
> > > > has been fixed by a previous fix we made... we're looking into it.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > > On Mon, Jul 19, 2021, at 20:23, Konstantine Karantasis wrote:
> > > > > Hi all,
> > > > >
> > > > > Since last week, we have reached the stage of Code Freeze for the
> > 3.0.0
> > > > > Apache Kafka release.
> > > > >
> > > > > From this point forward and until the official release of 3.0.0,
> only
> > > > > critical fixes for blocker issues should be merged to the 3.0
> release
> > > > > branch.
> > > > >
> > > > > The release plan currently includes ten (10) such known blockers.
> > > > >
> > > > >
> https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+3.0.0
> > > > >
> > > > > Besides these issues, any new issue that potentially gets
> discovered
> > will
> > > > > need to be reported on dev@kafka.apache.org (under this thread)
> and
> > be
> > > > > evaluated as a release blocker. At this point, the bar for such
> > issues is
> > > > > high; they need to be regressions or critical issues without an
> > > > acceptable
> > > > > workaround to be considered as release blockers.
> > > > >
> > > > > Exceptions of changes that may be merged to the 3.0 branch without
> a
> > > > > mention on the dev mailing list are fixes for test failures that
> will
> > > > help
> > > > > stabilize the build and small documentation changes.
> > > > >
> > > > > If by the end of this week we are down to zero blockers and have
> > green
> > > > > builds and passing system tests, I will attempt to generate the
> first
> > > > > Release Candidate (RC) for 3.0.0 on Friday.
> > > > >
> > > > > Thank you all for the hard work so far.
> > > > > Konstantine
> > > > >
> > > > >
> > > > > On Mon, Jul 12, 2021 at 1:59 PM Konstantine Karantasis
> > > > >  wrote:
> > > > >
> > > > > > Hi all,
> > > > > >
> > > > > > This is a reminder that Code Freeze for Apache Kafka 3.0 is
> coming
> > up
> > > > 

[jira] [Created] (KAFKA-13129) Fix broken system tests relate to the ConfigCommand change

2021-07-22 Thread Luke Chen (Jira)
Luke Chen created KAFKA-13129:
-

 Summary: Fix broken system tests relate to the ConfigCommand change
 Key: KAFKA-13129
 URL: https://issues.apache.org/jira/browse/KAFKA-13129
 Project: Kafka
  Issue Type: Bug
Reporter: Luke Chen
Assignee: Luke Chen
 Fix For: 3.0.0


After KAFKA-12598, the system tests failed in {{upgrade_test}}, 
{{zookeeper_tls_encrypt_only_test.py,}} and {{zookeeper_tls_test.py}}. Fix them.



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


[jira] [Created] (KAFKA-13128) Flaky Test StoreQueryIntegrationTest.shouldQueryStoresAfterAddingAndRemovingStreamThread

2021-07-22 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-13128:
--

 Summary: Flaky Test 
StoreQueryIntegrationTest.shouldQueryStoresAfterAddingAndRemovingStreamThread
 Key: KAFKA-13128
 URL: https://issues.apache.org/jira/browse/KAFKA-13128
 Project: Kafka
  Issue Type: Bug
  Components: streams
Affects Versions: 3.1.0
Reporter: A. Sophie Blee-Goldman


h3. Stacktrace

java.lang.AssertionError: Expected: is not null but: was null 
  at org.hamcrest.MatcherAssert.assertThat(MatcherAssert.java:20) 
  at org.hamcrest.MatcherAssert.assertThat(MatcherAssert.java:6)
  at 
org.apache.kafka.streams.integration.StoreQueryIntegrationTest.lambda$shouldQueryStoresAfterAddingAndRemovingStreamThread$19(StoreQueryIntegrationTest.java:461)
  at 
org.apache.kafka.streams.integration.StoreQueryIntegrationTest.until(StoreQueryIntegrationTest.java:506)
  at 
org.apache.kafka.streams.integration.StoreQueryIntegrationTest.shouldQueryStoresAfterAddingAndRemovingStreamThread(StoreQueryIntegrationTest.java:455)

 

https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-11085/5/testReport/org.apache.kafka.streams.integration/StoreQueryIntegrationTest/Build___JDK_16_and_Scala_2_13___shouldQueryStoresAfterAddingAndRemovingStreamThread_2/



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


[jira] [Created] (KAFKA-13127) Fix stray partition lookup logic

2021-07-22 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-13127:
---

 Summary: Fix stray partition lookup logic
 Key: KAFKA-13127
 URL: https://issues.apache.org/jira/browse/KAFKA-13127
 Project: Kafka
  Issue Type: Bug
Reporter: Jason Gustafson
Assignee: Jason Gustafson
 Fix For: 3.0.0


The result of `BrokerMetadataPublisher.findGhostReplicas` is inverted. It 
returns all of the non-stray replicas. This causes all off these partitions to 
get deleted on startup by mistake.



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


Re: [DISCUSS] Apache Kafka 3.0.0 release plan with new updated dates

2021-07-22 Thread Konstantine Karantasis
Thanks for raising this John.

While we are working to eliminate the existing blockers I think it would be
great to use this time in order to test the upgrade path that you mention.

Before we approve a release candidate (once such a RC is generated) we
should confirm that the upgrade works as expected.
So, I agree with you that this is not an RC generation blocker per se but
it's a release blocker overall.

Konstantine


On Thu, Jul 22, 2021 at 4:21 PM John Roesler  wrote:

> Hello Konstantine,
>
> Someone just called to my attention that KAFKA-12724 had not
> been marked as a 3.0 blocker. We never added 2.8 to the
> Streams upgrade system test suite. This isn't a blocker in
> that it is a problem, but we should make sure that Streams
> is actually upgradable before releasing 3.0.
>
> I'm sorry for the oversight. For what it's worth, I think we
> could proceed with a release candidate while we continue to
> address the missing system test.
>
> Thanks,
> -John
>
> https://issues.apache.org/jira/browse/KAFKA-12724
>
> On Wed, 2021-07-21 at 14:00 -0700, Konstantine Karantasis
> wrote:
> > Thanks for the heads up Colin.
> >
> > KAFKA-13112 seems important and of course relevant to what we ship with
> > 3.0.
> > Same for the test failures captured by KAFKA-13095 and KAFKA-12851.
> Fixing
> > those will increase the stability of our builds.
> >
> > Therefore, considering these tickets as blockers currently makes sense to
> > me.
> >
> > Konstantine
> >
> >
> > On Wed, Jul 21, 2021 at 11:46 AM Colin McCabe 
> wrote:
> >
> > > Hi Konstantine,
> > >
> > > Thanks for your work on this release! We discovered three blocker bugs
> > > which are worth bringing up here:
> > >
> > > 1. KAFKA-13112: Controller's committed offset get out of sync with raft
> > > client listener context
> > > 2. KAFKA-13095: TransactionsTest is failing in kraft mode
> > > 3. KAFKA-12851: Flaky Test
> > > RaftEventSimulationTest.canMakeProgressIfMajorityIsReachable
> > >
> > > There are two subtasks for #1 which we are working on. We suspect that
> #3
> > > has been fixed by a previous fix we made... we're looking into it.
> > >
> > > best,
> > > Colin
> > >
> > > On Mon, Jul 19, 2021, at 20:23, Konstantine Karantasis wrote:
> > > > Hi all,
> > > >
> > > > Since last week, we have reached the stage of Code Freeze for the
> 3.0.0
> > > > Apache Kafka release.
> > > >
> > > > From this point forward and until the official release of 3.0.0, only
> > > > critical fixes for blocker issues should be merged to the 3.0 release
> > > > branch.
> > > >
> > > > The release plan currently includes ten (10) such known blockers.
> > > >
> > > > https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+3.0.0
> > > >
> > > > Besides these issues, any new issue that potentially gets discovered
> will
> > > > need to be reported on dev@kafka.apache.org (under this thread) and
> be
> > > > evaluated as a release blocker. At this point, the bar for such
> issues is
> > > > high; they need to be regressions or critical issues without an
> > > acceptable
> > > > workaround to be considered as release blockers.
> > > >
> > > > Exceptions of changes that may be merged to the 3.0 branch without a
> > > > mention on the dev mailing list are fixes for test failures that will
> > > help
> > > > stabilize the build and small documentation changes.
> > > >
> > > > If by the end of this week we are down to zero blockers and have
> green
> > > > builds and passing system tests, I will attempt to generate the first
> > > > Release Candidate (RC) for 3.0.0 on Friday.
> > > >
> > > > Thank you all for the hard work so far.
> > > > Konstantine
> > > >
> > > >
> > > > On Mon, Jul 12, 2021 at 1:59 PM Konstantine Karantasis
> > > >  wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > This is a reminder that Code Freeze for Apache Kafka 3.0 is coming
> up
> > > this
> > > > > week and is set to take place by the end of day Wednesday, July
> 14th.
> > > > >
> > > > > Currently in the project we have 22 blocker issues for 3.0, out of
> 41
> > > total
> > > > > tickets targeting 3.0.
> > > > >
> > > > > You may find the list of open issues in the release plan page:
> > > > >
> https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+3.0.0
> > > > >
> > > > > Thanks for all the hard work so far and for reducing the number of
> open
> > > > > issues in the recent days.
> > > > > Please take another look and help us resolve all the blockers for
> this
> > > > > upcoming major release.
> > > > >
> > > > > Best,
> > > > > Konstantine
> > > > >
> > > > > On Mon, Jul 12, 2021 at 1:57 PM Konstantine Karantasis <
> > > > > konstant...@confluent.io> wrote:
> > > > >
> > > > > >
> > > > > > Thanks for the update Levani,
> > > > > >
> > > > > > KIP-708 is now on the list of postponed KIPs.
> > > > > >
> > > > > > Konstantine
> > > > > >
> > > > > > On Thu, Jul 1, 2021 at 10:48 PM Levani Kokhreidze <
> > > > > levani.co...@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi 

Re: [DISCUSS] Apache Kafka 3.0.0 release plan with new updated dates

2021-07-22 Thread John Roesler
Hello Konstantine,

Someone just called to my attention that KAFKA-12724 had not
been marked as a 3.0 blocker. We never added 2.8 to the
Streams upgrade system test suite. This isn't a blocker in
that it is a problem, but we should make sure that Streams
is actually upgradable before releasing 3.0.

I'm sorry for the oversight. For what it's worth, I think we
could proceed with a release candidate while we continue to
address the missing system test.

Thanks,
-John

https://issues.apache.org/jira/browse/KAFKA-12724

On Wed, 2021-07-21 at 14:00 -0700, Konstantine Karantasis
wrote:
> Thanks for the heads up Colin.
> 
> KAFKA-13112 seems important and of course relevant to what we ship with
> 3.0.
> Same for the test failures captured by KAFKA-13095 and KAFKA-12851. Fixing
> those will increase the stability of our builds.
> 
> Therefore, considering these tickets as blockers currently makes sense to
> me.
> 
> Konstantine
> 
> 
> On Wed, Jul 21, 2021 at 11:46 AM Colin McCabe  wrote:
> 
> > Hi Konstantine,
> > 
> > Thanks for your work on this release! We discovered three blocker bugs
> > which are worth bringing up here:
> > 
> > 1. KAFKA-13112: Controller's committed offset get out of sync with raft
> > client listener context
> > 2. KAFKA-13095: TransactionsTest is failing in kraft mode
> > 3. KAFKA-12851: Flaky Test
> > RaftEventSimulationTest.canMakeProgressIfMajorityIsReachable
> > 
> > There are two subtasks for #1 which we are working on. We suspect that #3
> > has been fixed by a previous fix we made... we're looking into it.
> > 
> > best,
> > Colin
> > 
> > On Mon, Jul 19, 2021, at 20:23, Konstantine Karantasis wrote:
> > > Hi all,
> > > 
> > > Since last week, we have reached the stage of Code Freeze for the 3.0.0
> > > Apache Kafka release.
> > > 
> > > From this point forward and until the official release of 3.0.0, only
> > > critical fixes for blocker issues should be merged to the 3.0 release
> > > branch.
> > > 
> > > The release plan currently includes ten (10) such known blockers.
> > > 
> > > https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+3.0.0
> > > 
> > > Besides these issues, any new issue that potentially gets discovered will
> > > need to be reported on dev@kafka.apache.org (under this thread) and be
> > > evaluated as a release blocker. At this point, the bar for such issues is
> > > high; they need to be regressions or critical issues without an
> > acceptable
> > > workaround to be considered as release blockers.
> > > 
> > > Exceptions of changes that may be merged to the 3.0 branch without a
> > > mention on the dev mailing list are fixes for test failures that will
> > help
> > > stabilize the build and small documentation changes.
> > > 
> > > If by the end of this week we are down to zero blockers and have green
> > > builds and passing system tests, I will attempt to generate the first
> > > Release Candidate (RC) for 3.0.0 on Friday.
> > > 
> > > Thank you all for the hard work so far.
> > > Konstantine
> > > 
> > > 
> > > On Mon, Jul 12, 2021 at 1:59 PM Konstantine Karantasis
> > >  wrote:
> > > 
> > > > Hi all,
> > > > 
> > > > This is a reminder that Code Freeze for Apache Kafka 3.0 is coming up
> > this
> > > > week and is set to take place by the end of day Wednesday, July 14th.
> > > > 
> > > > Currently in the project we have 22 blocker issues for 3.0, out of 41
> > total
> > > > tickets targeting 3.0.
> > > > 
> > > > You may find the list of open issues in the release plan page:
> > > > https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+3.0.0
> > > > 
> > > > Thanks for all the hard work so far and for reducing the number of open
> > > > issues in the recent days.
> > > > Please take another look and help us resolve all the blockers for this
> > > > upcoming major release.
> > > > 
> > > > Best,
> > > > Konstantine
> > > > 
> > > > On Mon, Jul 12, 2021 at 1:57 PM Konstantine Karantasis <
> > > > konstant...@confluent.io> wrote:
> > > > 
> > > > > 
> > > > > Thanks for the update Levani,
> > > > > 
> > > > > KIP-708 is now on the list of postponed KIPs.
> > > > > 
> > > > > Konstantine
> > > > > 
> > > > > On Thu, Jul 1, 2021 at 10:48 PM Levani Kokhreidze <
> > > > levani.co...@gmail.com>
> > > > > wrote:
> > > > > 
> > > > > > Hi Konstantine,
> > > > > > 
> > > > > > FYI, I don’t think we will be able to have KIP-708 ready on time.
> > > > > > Feel free to remove it from the release plan.
> > > > > > 
> > > > > > Best,
> > > > > > Levani
> > > > > > 
> > > > > > > On 1. Jul 2021, at 01:27, Konstantine Karantasis <
> > > > > > konstant...@confluent.io.INVALID> wrote:
> > > > > > > 
> > > > > > > Hi all,
> > > > > > > 
> > > > > > > Today we have reached the Feature Freeze milestone for Apache
> > Kafka
> > > > 3.0.
> > > > > > > Exciting!
> > > > > > > 
> > > > > > > I'm going to allow for any pending changes to settle within the
> > next
> > > > > > couple
> > > > > > > of days.
> > > > > > > I trust that we all approve and merge 

Build failed in Jenkins: Kafka » Kafka Branch Builder » 2.8 #55

2021-07-22 Thread Apache Jenkins Server
See 


Changes:


--
[...truncated 465733 lines...]
[2021-07-22T22:25:04.818Z] KafkaZkClientTest > 
testZNodeChangeHandlerForDataChange() STARTED
[2021-07-22T22:25:04.818Z] 
[2021-07-22T22:25:04.818Z] KafkaZkClientTest > 
testZNodeChangeHandlerForDataChange() PASSED
[2021-07-22T22:25:04.818Z] 
[2021-07-22T22:25:04.818Z] KafkaZkClientTest > 
testCreateAndGetTopicPartitionStatesRaw() STARTED
[2021-07-22T22:25:05.857Z] 
[2021-07-22T22:25:05.857Z] KafkaZkClientTest > 
testCreateAndGetTopicPartitionStatesRaw() PASSED
[2021-07-22T22:25:05.857Z] 
[2021-07-22T22:25:05.857Z] KafkaZkClientTest > testLogDirGetters() STARTED
[2021-07-22T22:25:05.857Z] 
[2021-07-22T22:25:05.857Z] KafkaZkClientTest > testLogDirGetters() PASSED
[2021-07-22T22:25:05.857Z] 
[2021-07-22T22:25:05.857Z] KafkaZkClientTest > 
testSetGetAndDeletePartitionReassignment() STARTED
[2021-07-22T22:25:05.857Z] 
[2021-07-22T22:25:05.857Z] KafkaZkClientTest > 
testSetGetAndDeletePartitionReassignment() PASSED
[2021-07-22T22:25:05.857Z] 
[2021-07-22T22:25:05.857Z] KafkaZkClientTest > 
testIsrChangeNotificationsDeletion() STARTED
[2021-07-22T22:25:07.068Z] 
[2021-07-22T22:25:07.068Z] KafkaZkClientTest > 
testIsrChangeNotificationsDeletion() PASSED
[2021-07-22T22:25:07.068Z] 
[2021-07-22T22:25:07.068Z] KafkaZkClientTest > testGetDataAndVersion() STARTED
[2021-07-22T22:25:07.068Z] 
[2021-07-22T22:25:07.068Z] KafkaZkClientTest > testGetDataAndVersion() PASSED
[2021-07-22T22:25:07.068Z] 
[2021-07-22T22:25:07.068Z] KafkaZkClientTest > testGetChildren() STARTED
[2021-07-22T22:25:07.068Z] 
[2021-07-22T22:25:07.068Z] KafkaZkClientTest > testGetChildren() PASSED
[2021-07-22T22:25:07.068Z] 
[2021-07-22T22:25:07.068Z] KafkaZkClientTest > testSetAndGetConsumerOffset() 
STARTED
[2021-07-22T22:25:07.068Z] 
[2021-07-22T22:25:07.068Z] KafkaZkClientTest > testSetAndGetConsumerOffset() 
PASSED
[2021-07-22T22:25:07.068Z] 
[2021-07-22T22:25:07.068Z] KafkaZkClientTest > testClusterIdMethods() STARTED
[2021-07-22T22:25:08.279Z] 
[2021-07-22T22:25:08.279Z] KafkaZkClientTest > testClusterIdMethods() PASSED
[2021-07-22T22:25:08.279Z] 
[2021-07-22T22:25:08.279Z] KafkaZkClientTest > 
testEntityConfigManagementMethods() STARTED
[2021-07-22T22:25:08.279Z] 
[2021-07-22T22:25:08.279Z] KafkaZkClientTest > 
testEntityConfigManagementMethods() PASSED
[2021-07-22T22:25:08.279Z] 
[2021-07-22T22:25:08.279Z] KafkaZkClientTest > testUpdateLeaderAndIsr() STARTED
[2021-07-22T22:25:08.279Z] 
[2021-07-22T22:25:08.279Z] KafkaZkClientTest > testUpdateLeaderAndIsr() PASSED
[2021-07-22T22:25:08.279Z] 
[2021-07-22T22:25:08.279Z] KafkaZkClientTest > testUpdateBrokerInfo() STARTED
[2021-07-22T22:25:09.316Z] 
[2021-07-22T22:25:09.316Z] KafkaZkClientTest > testUpdateBrokerInfo() PASSED
[2021-07-22T22:25:09.316Z] 
[2021-07-22T22:25:09.316Z] KafkaZkClientTest > testCreateRecursive() STARTED
[2021-07-22T22:25:09.316Z] 
[2021-07-22T22:25:09.316Z] KafkaZkClientTest > testCreateRecursive() PASSED
[2021-07-22T22:25:09.316Z] 
[2021-07-22T22:25:09.316Z] KafkaZkClientTest > testGetConsumerOffsetNoData() 
STARTED
[2021-07-22T22:25:09.316Z] 
[2021-07-22T22:25:09.316Z] KafkaZkClientTest > testGetConsumerOffsetNoData() 
PASSED
[2021-07-22T22:25:09.316Z] 
[2021-07-22T22:25:09.316Z] KafkaZkClientTest > testDeleteTopicPathMethods() 
STARTED
[2021-07-22T22:25:10.357Z] 
[2021-07-22T22:25:10.357Z] KafkaZkClientTest > testDeleteTopicPathMethods() 
PASSED
[2021-07-22T22:25:10.357Z] 
[2021-07-22T22:25:10.357Z] KafkaZkClientTest > testSetTopicPartitionStatesRaw() 
STARTED
[2021-07-22T22:25:10.357Z] 
[2021-07-22T22:25:10.357Z] KafkaZkClientTest > testSetTopicPartitionStatesRaw() 
PASSED
[2021-07-22T22:25:10.357Z] 
[2021-07-22T22:25:10.357Z] KafkaZkClientTest > testAclManagementMethods() 
STARTED
[2021-07-22T22:25:10.357Z] 
[2021-07-22T22:25:10.357Z] KafkaZkClientTest > testAclManagementMethods() PASSED
[2021-07-22T22:25:10.357Z] 
[2021-07-22T22:25:10.357Z] KafkaZkClientTest > 
testPreferredReplicaElectionMethods() STARTED
[2021-07-22T22:25:11.394Z] 
[2021-07-22T22:25:11.394Z] KafkaZkClientTest > 
testPreferredReplicaElectionMethods() PASSED
[2021-07-22T22:25:11.394Z] 
[2021-07-22T22:25:11.394Z] KafkaZkClientTest > testPropagateLogDir() STARTED
[2021-07-22T22:25:11.394Z] 
[2021-07-22T22:25:11.394Z] KafkaZkClientTest > testPropagateLogDir() PASSED
[2021-07-22T22:25:11.394Z] 
[2021-07-22T22:25:11.394Z] KafkaZkClientTest > testGetDataAndStat() STARTED
[2021-07-22T22:25:12.605Z] 
[2021-07-22T22:25:12.605Z] KafkaZkClientTest > testGetDataAndStat() PASSED
[2021-07-22T22:25:12.605Z] 
[2021-07-22T22:25:12.605Z] KafkaZkClientTest > 
testReassignPartitionsInProgress() STARTED
[2021-07-22T22:25:12.605Z] 
[2021-07-22T22:25:12.605Z] KafkaZkClientTest > 
testReassignPartitionsInProgress() PASSED
[2021-07-22T22:25:12.605Z] 
[2021-07-22T22:25:12.605Z] KafkaZkClientTest > testCreateTopLevelPaths() STARTED
[2021-07-22T22:25:12.605Z] 

[jira] [Created] (KAFKA-13126) Overflow in joinGroupTimeoutMs when max.poll.interval.ms is MAX_VALUE leads to missing rebalances

2021-07-22 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-13126:
--

 Summary: Overflow in joinGroupTimeoutMs when max.poll.interval.ms 
is MAX_VALUE leads to missing rebalances
 Key: KAFKA-13126
 URL: https://issues.apache.org/jira/browse/KAFKA-13126
 Project: Kafka
  Issue Type: Bug
  Components: consumer
Reporter: A. Sophie Blee-Goldman
Assignee: A. Sophie Blee-Goldman
 Fix For: 3.1.0


In older versions of Kafka Streams, the {{max.poll.interval.ms}} config was 
overridden by default to {{Integer.MAX_VALUE}}. Even after we removed this 
override, users of both the plain consumer client and kafka streams still set 
the poll interval to MAX_VALUE somewhat often. Unfortunately, this causes an 
overflow when computing the {{joinGroupTimeoutMs}} and results in it being set 
to the {{request.timeout.ms}} instead, which is much lower.

This can easily make consumers drop out of the group, since they must rejoin 
now within 30s (by default) yet have no obligation to almost ever call poll() 
given the high {{max.poll.interval.ms}}. We just need to check for overflow and 
fix it to {{Integer.MAX_VALUE}} when it occurs.



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


Build failed in Jenkins: Kafka » kafka-2.7-jdk8 #168

2021-07-22 Thread Apache Jenkins Server
See 


Changes:

[Manikumar Reddy] KAFKA-12985: Upgrade jetty to 9.4.42 (#10919)

[Manikumar Reddy] MINOR: checkstyle version upgrade: 8.20 -> 8.36.2 (#10656)


--
[...truncated 3.46 MB...]

org.apache.kafka.streams.test.MockProcessorContextAPITest > 
shouldCaptureOutputRecords STARTED

org.apache.kafka.streams.test.MockProcessorContextAPITest > 
shouldCaptureOutputRecords PASSED

org.apache.kafka.streams.test.MockProcessorContextAPITest > 
fullConstructorShouldSetAllExpectedAttributes STARTED

org.apache.kafka.streams.test.MockProcessorContextAPITest > 
fullConstructorShouldSetAllExpectedAttributes PASSED

org.apache.kafka.streams.test.MockProcessorContextAPITest > 
shouldCaptureCommitsAndAllowReset STARTED

org.apache.kafka.streams.test.MockProcessorContextAPITest > 
shouldCaptureCommitsAndAllowReset PASSED

org.apache.kafka.streams.test.MockProcessorContextAPITest > 
shouldCaptureApplicationAndRecordMetadata STARTED

org.apache.kafka.streams.test.MockProcessorContextAPITest > 
shouldCaptureApplicationAndRecordMetadata PASSED

org.apache.kafka.streams.test.MockProcessorContextAPITest > 
shouldCaptureRecordsOutputToChildByName STARTED

org.apache.kafka.streams.test.MockProcessorContextAPITest > 
shouldCaptureRecordsOutputToChildByName PASSED

org.apache.kafka.streams.test.MockProcessorContextAPITest > 
shouldCapturePunctuator STARTED

org.apache.kafka.streams.test.MockProcessorContextAPITest > 
shouldCapturePunctuator PASSED

org.apache.kafka.streams.test.wordcount.WindowedWordCountProcessorTest > 
shouldWorkWithInMemoryStore STARTED

org.apache.kafka.streams.test.wordcount.WindowedWordCountProcessorTest > 
shouldWorkWithInMemoryStore PASSED

org.apache.kafka.streams.test.wordcount.WindowedWordCountProcessorTest > 
shouldWorkWithPersistentStore STARTED

org.apache.kafka.streams.test.wordcount.WindowedWordCountProcessorTest > 
shouldWorkWithPersistentStore PASSED

org.apache.kafka.streams.MockTimeTest > shouldGetNanosAsMillis STARTED

org.apache.kafka.streams.MockTimeTest > shouldGetNanosAsMillis PASSED

org.apache.kafka.streams.MockTimeTest > shouldSetStartTime STARTED

org.apache.kafka.streams.MockTimeTest > shouldSetStartTime PASSED

org.apache.kafka.streams.MockTimeTest > shouldNotAllowNegativeSleep STARTED

org.apache.kafka.streams.MockTimeTest > shouldNotAllowNegativeSleep PASSED

org.apache.kafka.streams.MockTimeTest > shouldAdvanceTimeOnSleep STARTED

org.apache.kafka.streams.MockTimeTest > shouldAdvanceTimeOnSleep PASSED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > shouldReturnIsOpen 
STARTED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > shouldReturnIsOpen 
PASSED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > 
shouldDeleteAndReturnPlainValue STARTED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > 
shouldDeleteAndReturnPlainValue PASSED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > shouldReturnName 
STARTED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > shouldReturnName 
PASSED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > 
shouldPutWithUnknownTimestamp STARTED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > 
shouldPutWithUnknownTimestamp PASSED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > 
shouldPutAllWithUnknownTimestamp STARTED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > 
shouldPutAllWithUnknownTimestamp PASSED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > 
shouldReturnIsPersistent STARTED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > 
shouldReturnIsPersistent PASSED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > 
shouldForwardDeprecatedInit STARTED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > 
shouldForwardDeprecatedInit PASSED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > 
shouldPutIfAbsentWithUnknownTimestamp STARTED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > 
shouldPutIfAbsentWithUnknownTimestamp PASSED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > shouldForwardClose 
STARTED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > shouldForwardClose 
PASSED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > shouldForwardFlush 
STARTED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > shouldForwardFlush 
PASSED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > shouldForwardInit 
STARTED

org.apache.kafka.streams.internals.KeyValueStoreFacadeTest > shouldForwardInit 
PASSED

> Task :streams:upgrade-system-tests-0100:spotbugsMain NO-SOURCE
> Task :streams:upgrade-system-tests-0100:test
> Task :streams:upgrade-system-tests-0101:compileJava NO-SOURCE
> Task :streams:upgrade-system-tests-0101:processResources NO-SOURCE
> Task 

Jenkins build is still unstable: Kafka » Kafka Branch Builder » 2.8 #54

2021-07-22 Thread Apache Jenkins Server
See 




Jenkins build is still unstable: Kafka » Kafka Branch Builder » 3.0 #58

2021-07-22 Thread Apache Jenkins Server
See 




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

2021-07-22 Thread Apache Jenkins Server
See 


Changes:


--
[...truncated 486476 lines...]
[2021-07-22T19:36:30.313Z] TopicCommandIntegrationTest > 
testDescribeWhenTopicDoesntExist() PASSED
[2021-07-22T19:36:30.313Z] 
[2021-07-22T19:36:30.313Z] TopicCommandIntegrationTest > 
testDescribeDoesNotFailWhenListingReassignmentIsUnauthorized() STARTED
[2021-07-22T19:36:34.111Z] 
[2021-07-22T19:36:34.111Z] TopicCommandIntegrationTest > 
testCreateWithReplicaAssignment() PASSED
[2021-07-22T19:36:34.111Z] 
[2021-07-22T19:36:34.111Z] TopicCommandIntegrationTest > 
testAlterWithInvalidPartitionCount() STARTED
[2021-07-22T19:36:35.514Z] 
[2021-07-22T19:36:35.514Z] TopicCommandIntegrationTest > 
testDescribeDoesNotFailWhenListingReassignmentIsUnauthorized() PASSED
[2021-07-22T19:36:35.514Z] 
[2021-07-22T19:36:35.514Z] TopicCommandIntegrationTest > 
testAlterAssignmentWithMoreAssignmentThanPartitions() STARTED
[2021-07-22T19:36:38.126Z] 
[2021-07-22T19:36:38.127Z] TopicCommandIntegrationTest > 
testAlterWithInvalidPartitionCount() PASSED
[2021-07-22T19:36:38.127Z] 
[2021-07-22T19:36:38.127Z] TopicCommandIntegrationTest > testCreate() STARTED
[2021-07-22T19:36:39.639Z] 
[2021-07-22T19:36:39.639Z] TopicCommandIntegrationTest > 
testAlterAssignmentWithMoreAssignmentThanPartitions() PASSED
[2021-07-22T19:36:39.639Z] 
[2021-07-22T19:36:39.639Z] TopicCommandIntegrationTest > 
testDescribeWhenTopicDoesntExistWithIfExists() STARTED
[2021-07-22T19:36:43.443Z] 
[2021-07-22T19:36:43.443Z] TopicCommandIntegrationTest > testCreate() PASSED
[2021-07-22T19:36:43.443Z] 
[2021-07-22T19:36:43.443Z] TopicCommandIntegrationTest > 
testDescribeUnderMinIsrPartitionsMixed() STARTED
[2021-07-22T19:36:46.424Z] 
[2021-07-22T19:36:46.424Z] TopicCommandIntegrationTest > 
testDescribeWhenTopicDoesntExistWithIfExists() PASSED
[2021-07-22T19:36:46.424Z] 
[2021-07-22T19:36:46.424Z] TopicCommandIntegrationTest > 
testCreateWithDefaultPartitions() STARTED
[2021-07-22T19:36:50.494Z] 
[2021-07-22T19:36:50.494Z] TopicCommandIntegrationTest > 
testDescribeUnderMinIsrPartitionsMixed() PASSED
[2021-07-22T19:36:50.494Z] 
[2021-07-22T19:36:50.494Z] TopicCommandIntegrationTest > 
testCreateWhenAlreadyExists() STARTED
[2021-07-22T19:36:50.661Z] 
[2021-07-22T19:36:50.661Z] TopicCommandIntegrationTest > 
testCreateWithDefaultPartitions() PASSED
[2021-07-22T19:36:50.661Z] 
[2021-07-22T19:36:50.661Z] TopicCommandIntegrationTest > testListTopics() 
STARTED
[2021-07-22T19:36:55.015Z] 
[2021-07-22T19:36:55.015Z] TopicCommandIntegrationTest > 
testCreateWhenAlreadyExists() PASSED
[2021-07-22T19:36:55.015Z] 
[2021-07-22T19:36:55.015Z] TopicCommandIntegrationTest > testDescribe() STARTED
[2021-07-22T19:36:55.182Z] 
[2021-07-22T19:36:55.182Z] TopicCommandIntegrationTest > testListTopics() PASSED
[2021-07-22T19:36:55.182Z] 
[2021-07-22T19:36:55.182Z] TopicCommandIntegrationTest > 
testDeleteWhenTopicDoesntExistWithIfExists() STARTED
[2021-07-22T19:36:59.573Z] 
[2021-07-22T19:36:59.573Z] TopicCommandIntegrationTest > testDescribe() PASSED
[2021-07-22T19:36:59.573Z] 
[2021-07-22T19:36:59.573Z] TopicCommandIntegrationTest > testAlterAssignment() 
STARTED
[2021-07-22T19:36:59.740Z] 
[2021-07-22T19:36:59.741Z] TopicCommandIntegrationTest > 
testDeleteWhenTopicDoesntExistWithIfExists() PASSED
[2021-07-22T19:36:59.741Z] 
[2021-07-22T19:36:59.741Z] TopicCommandIntegrationTest > 
testCreateWhenAlreadyExistsWithIfNotExists() STARTED
[2021-07-22T19:37:02.206Z] 
[2021-07-22T19:37:02.206Z] TopicCommandIntegrationTest > 
testCreateWhenAlreadyExistsWithIfNotExists() PASSED
[2021-07-22T19:37:02.206Z] 
[2021-07-22T19:37:02.206Z] TopicCommandIntegrationTest > 
testDeleteInternalTopic() STARTED
[2021-07-22T19:37:02.877Z] 
[2021-07-22T19:37:02.877Z] TopicCommandIntegrationTest > testAlterAssignment() 
PASSED
[2021-07-22T19:37:02.877Z] 
[2021-07-22T19:37:02.877Z] ReplicationUtilsTest > testUpdateLeaderAndIsr() 
STARTED
[2021-07-22T19:37:03.930Z] 
[2021-07-22T19:37:03.930Z] ReplicationUtilsTest > testUpdateLeaderAndIsr() 
PASSED
[2021-07-22T19:37:03.930Z] 
[2021-07-22T19:37:03.930Z] ListOffsetsIntegrationTest > 
testMaxTimestampOffset() STARTED
[2021-07-22T19:37:05.937Z] 
[2021-07-22T19:37:05.937Z] ListOffsetsIntegrationTest > 
testMaxTimestampOffset() PASSED
[2021-07-22T19:37:05.937Z] 
[2021-07-22T19:37:05.937Z] ListOffsetsIntegrationTest > testLatestOffset() 
STARTED
[2021-07-22T19:37:06.607Z] 
[2021-07-22T19:37:06.607Z] TopicCommandIntegrationTest > 
testDeleteInternalTopic() PASSED
[2021-07-22T19:37:06.607Z] 
[2021-07-22T19:37:06.607Z] TopicCommandIntegrationTest > 
testInvalidTopicLevelConfig() STARTED
[2021-07-22T19:37:08.112Z] 
[2021-07-22T19:37:08.112Z] ListOffsetsIntegrationTest > testLatestOffset() 
PASSED
[2021-07-22T19:37:08.112Z] 
[2021-07-22T19:37:08.112Z] ListOffsetsIntegrationTest > testEarliestOffset() 
STARTED
[2021-07-22T19:37:11.128Z] 
[2021-07-22T19:37:11.128Z] ListOffsetsIntegrationTest > 

Re: [DISCUSS] KIP-655: Windowed "Distinct" Operation for KStream

2021-07-22 Thread John Roesler
Hi Ivan,

Sorry for the late amendment, but I was just reviewing the
KIP again for your vote thread, and it struck me that, if
this is a stateful operation, we need a few more overloads.

Following the example from the other stateful windowed
operators, we should have:

distinct()
distinct(Named)
distinct(Materialized)

It's a small update, but an important one, since people will
inevitably need to customize the state store for the
operation.

Thanks,
John

On Thu, 2021-07-22 at 13:58 -0500, John Roesler wrote:
> Hi Ivan,
> 
> Thanks for the reply.
> 
> 1. I think I might have gotten myself confused. I was
> thinking of this operation as stateless, but now I'm not
> sure what I was thinking... This operator has to be
> stateful, right? In that case, I agree that comparing
> serialized values seems to be way to do it.
> 
> 2. Thanks for the confirmation
> 
> 3. I continue to be satisfied to let you all hash it out.
> 
> Thanks,
> -John
> 
> On Tue, 2021-07-20 at 11:42 +0300, Ivan Ponomarev wrote:
> > Hi all,
> > 
> > 1. Actually I always thought about the serialized byte array only -- at 
> > least this is what local stores depend upon, and what Kafka itself 
> > depends upon when doing log compaction.
> > 
> > I can imagine a case where two different byte arrays deserialize to 
> > objects which are `equals` to each other. But I think we can ignore this 
> > for now because IMO the risks related to buggy `equals` implementations 
> > outweigh the potential benefits.
> > 
> > I will mention the duplicate definition in the KIP.
> > 
> > 2. I agree with John, he got my point.
> > 
> > 3. Let me gently insist on `distinct`. I believe that an exception to 
> > the rule is appropriate here, because the name `distinct()` is ubiquitous.
> > 
> > It's not only about Java Streams API (or .NET LINQ, which appeared 
> > earlier and also has `Distinct`): Spark's DataFrame has `distinct()` 
> > method, Hazelcast Jet has `distinct()` method, and I bet I can find more 
> > examples if I search. When we teach KStreams, we always say that 
> > KStreams are just like other streaming APIs, and they have roots in SQL 
> > queries. Users know what `distinct` is and they expect it to be in the API.
> > 
> > 
> > Regards,
> > 
> > Ivan
> > 
> > 
> > 13.07.2021 0:10, John Roesler пишет:
> > > Hi all,
> > > 
> > > Bruno raised some very good points. I’d like to chime in with additional 
> > > context.
> > > 
> > > 1. Great point. We faced a similar problem defining KIP-557. For 557, we 
> > > chose to use the serialized byte array instead of the equals() method, 
> > > but I think the situation in KIP-655 is a bit different. I think it might 
> > > make sense to use the equals() method here, but am curious what Ivan 
> > > thinks.
> > > 
> > > 2. I figured we'd do nothing. I thought Ivan was just saying that it 
> > > doesn't make a ton of sense to use it, which I agree with, but it doesn't 
> > > seem like that means we should prohibit it.
> > > 
> > > 3. FWIW, I don't have a strong feeling either way.
> > > 
> > > Thanks,
> > > -John
> > > 
> > > On Mon, Jul 12, 2021, at 09:14, Bruno Cadonna wrote:
> > > > Hi Ivan,
> > > > 
> > > > Thank you for the KIP!
> > > > 
> > > > Some aspects are not clear to me from the KIP and I have a proposal.
> > > > 
> > > > 1. The KIP does not describe the criteria that define a duplicate. Could
> > > > you add a definition of duplicate to the KIP?
> > > > 
> > > > 2. The KIP does not describe what happens if distinct() is applied on a
> > > > hopping window. On the DSL level, I do not see how you can avoid that
> > > > users apply distinct() on a hopping window, i.e., you cannot avoid it at
> > > > compile time, you need to check it at runtime and throw an exception. Is
> > > > this correct or am I missing something?
> > > > 
> > > > 3. I would also like to back a proposal by Sophie. She proposed to use
> > > > deduplicate() instead of distinct(), since the other DSL operations are
> > > > also verbs. I do not think that SQL and the Java Stream API are good
> > > > arguments to not use a verb.
> > > > 
> > > > Best,
> > > > Bruno
> > > > 
> > > > 
> > > > On 10.07.21 19:11, John Roesler wrote:
> > > > > Hi Ivan,
> > > > > 
> > > > > Sorry for the silence!
> > > > > 
> > > > > I have just re-read the proposal.
> > > > > 
> > > > > To summarize, you are now only proposing the zero-arg distict() 
> > > > > method to be added to TimeWindowedKStream and SessionWindowedKStream, 
> > > > > right?
> > > > > 
> > > > > I’m in favor of this proposal.
> > > > > 
> > > > > Thanks,
> > > > > John
> > > > > 
> > > > > On Sat, Jul 10, 2021, at 10:18, Ivan Ponomarev wrote:
> > > > > > Hello everyone,
> > > > > > 
> > > > > > I would like to remind you about KIP-655 and KIP-759 just in case 
> > > > > > they
> > > > > > got lost in your inbox.
> > > > > > 
> > > > > > Now the initial proposal is split into two independent and smaller 
> > > > > > ones,
> > > > > > so it must be easier to review them. Of course, 

Re: [DISCUSS] KIP-655: Windowed "Distinct" Operation for KStream

2021-07-22 Thread John Roesler
Hi Ivan,

Thanks for the reply.

1. I think I might have gotten myself confused. I was
thinking of this operation as stateless, but now I'm not
sure what I was thinking... This operator has to be
stateful, right? In that case, I agree that comparing
serialized values seems to be way to do it.

2. Thanks for the confirmation

3. I continue to be satisfied to let you all hash it out.

Thanks,
-John

On Tue, 2021-07-20 at 11:42 +0300, Ivan Ponomarev wrote:
> Hi all,
> 
> 1. Actually I always thought about the serialized byte array only -- at 
> least this is what local stores depend upon, and what Kafka itself 
> depends upon when doing log compaction.
> 
> I can imagine a case where two different byte arrays deserialize to 
> objects which are `equals` to each other. But I think we can ignore this 
> for now because IMO the risks related to buggy `equals` implementations 
> outweigh the potential benefits.
> 
> I will mention the duplicate definition in the KIP.
> 
> 2. I agree with John, he got my point.
> 
> 3. Let me gently insist on `distinct`. I believe that an exception to 
> the rule is appropriate here, because the name `distinct()` is ubiquitous.
> 
> It's not only about Java Streams API (or .NET LINQ, which appeared 
> earlier and also has `Distinct`): Spark's DataFrame has `distinct()` 
> method, Hazelcast Jet has `distinct()` method, and I bet I can find more 
> examples if I search. When we teach KStreams, we always say that 
> KStreams are just like other streaming APIs, and they have roots in SQL 
> queries. Users know what `distinct` is and they expect it to be in the API.
> 
> 
> Regards,
> 
> Ivan
> 
> 
> 13.07.2021 0:10, John Roesler пишет:
> > Hi all,
> > 
> > Bruno raised some very good points. I’d like to chime in with additional 
> > context.
> > 
> > 1. Great point. We faced a similar problem defining KIP-557. For 557, we 
> > chose to use the serialized byte array instead of the equals() method, but 
> > I think the situation in KIP-655 is a bit different. I think it might make 
> > sense to use the equals() method here, but am curious what Ivan thinks.
> > 
> > 2. I figured we'd do nothing. I thought Ivan was just saying that it 
> > doesn't make a ton of sense to use it, which I agree with, but it doesn't 
> > seem like that means we should prohibit it.
> > 
> > 3. FWIW, I don't have a strong feeling either way.
> > 
> > Thanks,
> > -John
> > 
> > On Mon, Jul 12, 2021, at 09:14, Bruno Cadonna wrote:
> > > Hi Ivan,
> > > 
> > > Thank you for the KIP!
> > > 
> > > Some aspects are not clear to me from the KIP and I have a proposal.
> > > 
> > > 1. The KIP does not describe the criteria that define a duplicate. Could
> > > you add a definition of duplicate to the KIP?
> > > 
> > > 2. The KIP does not describe what happens if distinct() is applied on a
> > > hopping window. On the DSL level, I do not see how you can avoid that
> > > users apply distinct() on a hopping window, i.e., you cannot avoid it at
> > > compile time, you need to check it at runtime and throw an exception. Is
> > > this correct or am I missing something?
> > > 
> > > 3. I would also like to back a proposal by Sophie. She proposed to use
> > > deduplicate() instead of distinct(), since the other DSL operations are
> > > also verbs. I do not think that SQL and the Java Stream API are good
> > > arguments to not use a verb.
> > > 
> > > Best,
> > > Bruno
> > > 
> > > 
> > > On 10.07.21 19:11, John Roesler wrote:
> > > > Hi Ivan,
> > > > 
> > > > Sorry for the silence!
> > > > 
> > > > I have just re-read the proposal.
> > > > 
> > > > To summarize, you are now only proposing the zero-arg distict() method 
> > > > to be added to TimeWindowedKStream and SessionWindowedKStream, right?
> > > > 
> > > > I’m in favor of this proposal.
> > > > 
> > > > Thanks,
> > > > John
> > > > 
> > > > On Sat, Jul 10, 2021, at 10:18, Ivan Ponomarev wrote:
> > > > > Hello everyone,
> > > > > 
> > > > > I would like to remind you about KIP-655 and KIP-759 just in case they
> > > > > got lost in your inbox.
> > > > > 
> > > > > Now the initial proposal is split into two independent and smaller 
> > > > > ones,
> > > > > so it must be easier to review them. Of course, if you have time.
> > > > > 
> > > > > Regards,
> > > > > 
> > > > > Ivan
> > > > > 
> > > > > 
> > > > > 24.06.2021 18:11, Ivan Ponomarev пишет:
> > > > > > Hello all,
> > > > > > 
> > > > > > I have rewritten the KIP-655 summarizing what was agreed upon during
> > > > > > this discussion (now the proposal is much simpler and less 
> > > > > > invasive).
> > > > > > 
> > > > > > I have also created KIP-759 (cancelRepartition operation) and 
> > > > > > started a
> > > > > > discussion for it.
> > > > > > 
> > > > > > Regards,
> > > > > > 
> > > > > > Ivan.
> > > > > > 
> > > > > > 
> > > > > > 
> > > > > > 04.06.2021 8:15, Matthias J. Sax пишет:
> > > > > > > Just skimmed over the thread -- first of all, I am glad that we 
> > > > > > > could
> > > > > > > merge KIP-418 and 

[jira] [Resolved] (KAFKA-12985) CVE-2021-28169 - Upgrade jetty to 9.4.41

2021-07-22 Thread Manikumar (Jira)


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

Manikumar resolved KAFKA-12985.
---
Fix Version/s: 2.8.1
   2.7.2
   3.0.0
   Resolution: Fixed

> CVE-2021-28169 - Upgrade jetty to 9.4.41
> 
>
> Key: KAFKA-12985
> URL: https://issues.apache.org/jira/browse/KAFKA-12985
> Project: Kafka
>  Issue Type: Task
>  Components: security
>Reporter: Dongjin Lee
>Assignee: Dongjin Lee
>Priority: Minor
> Fix For: 3.0.0, 2.7.2, 2.8.1
>
>
> CVE-2021-28169 vulnerability affects Jetty versions up to 9.4.40. For more 
> information see https://nvd.nist.gov/vuln/detail/CVE-2021-28169
> Upgrading to Jetty version 9.4.41 should address this issue 
> (https://github.com/eclipse/jetty.project/security/advisories/GHSA-gwcr-j4wh-j3cq).



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


Re: [VOTE] KIP-763: Range queries with open endpoints

2021-07-22 Thread John Roesler
Thank you, Patrick,

+1 (binding) from me as well!

Thanks,
-John

On Thu, 2021-07-22 at 10:40 +0200, Bruno Cadonna wrote:
> Hi Patrick,
> 
> Thank you for the KIP!
> 
> +1 (binding)
> 
> Best,
> Bruno
> 
> On 22.07.21 03:47, Luke Chen wrote:
> > Hi Patrick,
> > I like this KIP!
> > 
> > +1 (non-binding)
> > 
> > Luke
> > 
> > On Thu, Jul 22, 2021 at 7:04 AM Matthias J. Sax  wrote:
> > 
> > > Thanks for the KIP.
> > > 
> > > +1 (binding)
> > > 
> > > 
> > > -Matthias
> > > 
> > > On 7/21/21 1:18 PM, Patrick Stuedi wrote:
> > > > Hi all,
> > > > 
> > > > Thanks for the feedback on the KIP, I have updated the KIP and would 
> > > > like
> > > > to start the voting.
> > > > 
> > > > The KIP can be found here:
> > > > 
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-763%3A+Range+queries+with+open+endpoints
> > > > 
> > > > Please vote in this thread.
> > > > 
> > > > Thanks!
> > > > -Patrick
> > > > 
> > > 
> > 




Re: Jira contributor access

2021-07-22 Thread Boyang Chen
Done, thanks for your interest in Kafka!

On Thu, Jul 22, 2021 at 10:37 AM Knowles Atchison Jr 
wrote:

> Requesting Jira contributor access.
>
> Username: katchison
>


Re: [DISCUSS] KIP-714: Client metrics and observability

2021-07-22 Thread Feng Min
On Wed, Jul 21, 2021 at 6:17 PM Colin McCabe  wrote:

> On Tue, Jun 29, 2021, at 07:22, Magnus Edenhill wrote:
> > Den tors 17 juni 2021 kl 00:52 skrev Colin McCabe :
> > > A few critiques:
> > >
> > > - As I wrote above, I think this could benefit a lot by being split
> into
> > > several RPCs. A registration RPC, a report RPC, and an unregister RPC
> seem
> > > like logical choices.
> > >
> >
> > Responded to this in your previous mail, but in short I think a single
> > request is sufficient and keeps the implementation complexity / state
> down.
> >
>
> Hi Magnus,
>
> I still suspect that trying to do everything with a single RPC is more
> complex than using multiple RPCs.
>
> Can you go into more detail about how the client learns what metrics it
> should send? This was the purpose of the "registration" step in my scheme
> above.
>
> It seems quite awkward to combine an RPC for reporting metrics with and
> RPC for finding out what metrics are configured to be reported. For
> example, how would you build a tool to check what metrics are configured to
> be reported? Does the tool have to report fake metrics, just because
> there's no other way to get back that information? Seems wrong. (It would
> be a bit like combining createTopics and listTopics for "simplicity")
>

 +1 on separate RPC on metric discovery and metric report. I actually think
it makes complexity/state down compared with single RPC.


>
> > > - I don't think the client should be able to choose its own UUID. This
> > > adds complexity and introduces a chance that clients will choose an ID
> that
> > > is not unique. We already have an ID that the client itself supplies
> > > (clientID) so there is no need to introduce another such ID.
> > >
> >
> > The CLIENT_INSTANCE_ID (which is a combination of the client.id and a
> UUID)
> > is actually generated by the receiving broker on first contact.
> > The need for a new unique semi-random id is outlined in the KIP, but in
> > short; the client.id is not unique, and we need something unique that
> still
> > is prefix-matchable to the client.id so that we can add subscriptions
> > either using prefix-matching of just the client.id (which may match one
> or
> > more client instances), and exact matching which will match a one
> specific
> > client instance.
>
> Hmm... the client id is already sent in every RPC as part of the header.
> It's not necessary to send it again as part of one of the other RPC fields,
> right?
>
> More generally, why does the client instance ID need to be
> prefix-matchable? That seems like an implementation detail of the metrics
> collection system used on the broker side. Maybe someone wants to group by
> things other than client IDs -- perhaps client versions, for instance. By
> the same argument, we should put the client version string in the client
> instance ID, since someone might want to group by that. Or maybe we should
> include the hostname, and the IP, and, and, and You see the issue here.
> I think we shouldn't get involved in this kind of decision -- if we just
> pass a UUID, the broker-side software can group it or prefix it however it
> wants internally.
>
> > > - In general the schema seems to have a bad case of string-itis. UUID,
> > > content type, and requested metrics are all strings. Since these
> messages
> > > will be sent very frequently, it's quite costly to use strings for all
> > > these things. We have a type for UUID, which uses 16 bytes -- let's use
> > > that type for client instance ID, rather than a string which will be
> much
> > > larger. Also, since we already send clientID in the message header,
> there
> > > is no need to include it again in the instance ID.
> > >
> >
> > As explained above we need the client.id in the CLIENT_INSTANCE_ID. And
> I
> > don't think the overhead of this one string per request is going to be
> much
> > of an issue,
> > typical metric push intervals are probably in the >60s range.
> > If this becomes a problem we could use a per-connection identifier that
> the
> > broker translates to the client instance id before pushing metrics
> upwards
> > in the system.
> >
>
> This is actually an interesting design question -- why not use a
> per-TCP-connection identifier, rather than a per-client-instance
> identifier? If we are grouping by other things anyway (clientID, principal,
> etc.) on the server side, do we need to maintain a per-process identifier
> rather than a per-connection one?
>
> >
> > > - I think it would also be nice to have an enum or something for
> > > AcceptedContentTypes, RequestedMetrics, etc. We know that new
> additions to
> > > these categories will require KIPs, so it should be straightforward
> for the
> > > project to just have an enum that allows us to communicate these as
> ints.
> > >
> >
> > I'm thinking this might be overly constraining. The broker doesn't parse
> or
> > handle the received metrics data itself but just pushes it to the metrics
> > plugin, using an enum would 

Jira contributor access

2021-07-22 Thread Knowles Atchison Jr
Requesting Jira contributor access.

Username: katchison


[GitHub] [kafka-site] hrittikhere opened a new pull request #364: Add p3r (https://www.p3r.one/) to the list of the "Powered By ❤️"

2021-07-22 Thread GitBox


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


   [p3r](https://www.p3r.one/) uses Kafka as part of our Kubernetes Platform. 
We deliver it alongside Linkerd for development teams building event-based 
microservices. 
   
   @hachikuji could you help merge this? (Noticed you helped the last traveller 
to these parts :) )
   
   Feel free to contact me with any further questions.
   
   


-- 
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.

To unsubscribe, e-mail: dev-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




Re: [DISCUSS] KIP-761: Add total blocked time metric to streams

2021-07-22 Thread Bruno Cadonna

Hi Rohan,

Thank you for the KIP!

I agree that the KIP is well-motivated.

What is not very clear is the metadata like type, group, and tags of the 
metrics. For example, there is not application-id tag in Streams and 
there is also no producer-id tag. The clients, i.e., producer, admin, 
consumer, and also Streams have a client-id tag, that corresponds to the 
producer-id, consumer-id, etc you use in the KIP.


For examples of metadata used in Streams you can look at the following KIPs:

- 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-444%3A+Augment+metrics+for+Kafka+Streams
- 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-471%3A+Expose+RocksDB+Metrics+in+Kafka+Streams
- 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-607%3A+Add+Metrics+to+Kafka+Streams+to+Report+Properties+of+RocksDB
- 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-613%3A+Add+end-to-end+latency+metrics+to+Streams



Best,
Bruno

On 22.07.21 09:42, Rohan Desai wrote:

re sophie:

The intent here was to include all blocked time (not just `RUNNING`). The
caller can window the total blocked time themselves, and that can be
compared with a timeseries of the state to understand the ratio in
different states. I'll update the KIP to include `committed`. The admin API
calls should be accounted for by the admin client iotime/iowaittime
metrics.

On Tue, Jul 20, 2021 at 11:49 PM Rohan Desai 
wrote:


I remember now that we moved the round-trip PID's txn completion logic

into
init-transaction and commit/abort-transaction. So I think we'd count time
as in StreamsProducer#initTransaction as well (admittedly it is in most
cases a one-time thing).

Makes sense - I'll update the KIP

On Tue, Jul 20, 2021 at 11:48 PM Rohan Desai 
wrote:




I had a question - it seems like from the descriptionsof

`txn-commit-time-total` and `offset-commit-time-total` that they measure
similar processes for ALOS and EOS, but only `txn-commit-time-total` is
included in `blocked-time-total`. Why isn't `offset-commit-time-total` also
included?

I've updated the KIP to include it.


Aside from `flush-time-total`, `txn-commit-time-total` and

`offset-commit-time-total`, which will be producer/consumer client
metrics,
the rest of the metrics will be streams metrics that will be thread level,
is that right?

Based on the feedback from Guozhang, I've updated the KIP to reflect that
the lower-level metrics are all client metrics that are then summed to
compute the blocked time metric, which is a Streams metric.

On Tue, Jul 20, 2021 at 11:58 AM Rohan Desai 
wrote:


Similarly, I think "txn-commit-time-total" and

"offset-commit-time-total" may better be inside producer and consumer
clients respectively.

I agree for offset-commit-time-total. For txn-commit-time-total I'm
proposing we measure `StreamsProducer.commitTransaction`, which wraps
multiple producer calls (sendOffsets, commitTransaction)


For "txn-commit-time-total" specifically, besides

producer.commitTxn.
other txn-related calls may also be blocking, including
producer.beginTxn/abortTxn, I saw you mentioned "txn-begin-time-total"
later in the doc, but did not include it as a separate metric, and
similarly, should we have a `txn-abort-time-total` as well? If yes,
could
you update the KIP page accordingly.

`beginTransaction` is not blocking - I meant to remove that from that
doc. I'll add something for abort.

On Mon, Jul 19, 2021 at 11:55 PM Rohan Desai 
wrote:


Thanks for the review Guozhang! responding to your feedback inline:


1) I agree that the current ratio metrics is just "snapshot in

point", and
more flexible metrics that would allow reporters to calculate based on
window intervals are better. However, the current mechanism of the
proposed
metrics assumes the thread->clients mapping as of today, where each
thread
would own exclusively one main consumer, restore consumer, producer and
an
admin client. But this mapping may be subject to change in the future.
Have
you thought about how this metric can be extended when, e.g. the
embedded
clients and stream threads are de-coupled?

Of course this depends on how exactly we refactor the runtime -
assuming that we plan to factor out consumers into an "I/O" layer that is
responsible for receiving records and enqueuing them to be processed by
processing threads, then I think it should be reasonable to count the time
we spend blocked on this internal queue(s) as blocked. The main concern
there to me is that the I/O layer would be doing something expensive like
decompression that shouldn't be counted as "blocked". But if that really is
so expensive that it starts to throw off our ratios then it's probably
indicative of a larger problem that the "i/o layer" is a bottleneck and it
would be worth refactoring so that decompression (or insert other expensive
thing here) can also be done on the processing threads.


2) [This and all below are minor comments] The "flush-time-total" may

better be a producer client metric, as 

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

2021-07-22 Thread Apache Jenkins Server
See 




[jira] [Resolved] (KAFKA-13069) Add magic number to DefaultKafkaPrincipalBuilder.KafkaPrincipalSerde

2021-07-22 Thread Ron Dagostino (Jira)


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

Ron Dagostino resolved KAFKA-13069.
---
Resolution: Invalid

Flexible fields are sufficient as per KIP-590 VOTE email thread, so a magic 
number will not be needed.

> Add magic number to DefaultKafkaPrincipalBuilder.KafkaPrincipalSerde
> 
>
> Key: KAFKA-13069
> URL: https://issues.apache.org/jira/browse/KAFKA-13069
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.0.0, 2.8.0
>Reporter: Ron Dagostino
>Assignee: Ron Dagostino
>Priority: Critical
> Fix For: 3.1.0
>
>




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


Re: [VOTE] KIP-763: Range queries with open endpoints

2021-07-22 Thread Bruno Cadonna

Hi Patrick,

Thank you for the KIP!

+1 (binding)

Best,
Bruno

On 22.07.21 03:47, Luke Chen wrote:

Hi Patrick,
I like this KIP!

+1 (non-binding)

Luke

On Thu, Jul 22, 2021 at 7:04 AM Matthias J. Sax  wrote:


Thanks for the KIP.

+1 (binding)


-Matthias

On 7/21/21 1:18 PM, Patrick Stuedi wrote:

Hi all,

Thanks for the feedback on the KIP, I have updated the KIP and would like
to start the voting.

The KIP can be found here:


https://cwiki.apache.org/confluence/display/KAFKA/KIP-763%3A+Range+queries+with+open+endpoints


Please vote in this thread.

Thanks!
-Patrick







Re: [DISCUSS] KIP-761: Add total blocked time metric to streams

2021-07-22 Thread Rohan Desai
re sophie:

The intent here was to include all blocked time (not just `RUNNING`). The
caller can window the total blocked time themselves, and that can be
compared with a timeseries of the state to understand the ratio in
different states. I'll update the KIP to include `committed`. The admin API
calls should be accounted for by the admin client iotime/iowaittime
metrics.

On Tue, Jul 20, 2021 at 11:49 PM Rohan Desai 
wrote:

> > I remember now that we moved the round-trip PID's txn completion logic
> into
> init-transaction and commit/abort-transaction. So I think we'd count time
> as in StreamsProducer#initTransaction as well (admittedly it is in most
> cases a one-time thing).
>
> Makes sense - I'll update the KIP
>
> On Tue, Jul 20, 2021 at 11:48 PM Rohan Desai 
> wrote:
>
>>
>> > I had a question - it seems like from the descriptionsof
>> `txn-commit-time-total` and `offset-commit-time-total` that they measure
>> similar processes for ALOS and EOS, but only `txn-commit-time-total` is
>> included in `blocked-time-total`. Why isn't `offset-commit-time-total` also
>> included?
>>
>> I've updated the KIP to include it.
>>
>> > Aside from `flush-time-total`, `txn-commit-time-total` and
>> `offset-commit-time-total`, which will be producer/consumer client
>> metrics,
>> the rest of the metrics will be streams metrics that will be thread level,
>> is that right?
>>
>> Based on the feedback from Guozhang, I've updated the KIP to reflect that
>> the lower-level metrics are all client metrics that are then summed to
>> compute the blocked time metric, which is a Streams metric.
>>
>> On Tue, Jul 20, 2021 at 11:58 AM Rohan Desai 
>> wrote:
>>
>>> > Similarly, I think "txn-commit-time-total" and
>>> "offset-commit-time-total" may better be inside producer and consumer
>>> clients respectively.
>>>
>>> I agree for offset-commit-time-total. For txn-commit-time-total I'm
>>> proposing we measure `StreamsProducer.commitTransaction`, which wraps
>>> multiple producer calls (sendOffsets, commitTransaction)
>>>
>>> > > For "txn-commit-time-total" specifically, besides
>>> producer.commitTxn.
>>> other txn-related calls may also be blocking, including
>>> producer.beginTxn/abortTxn, I saw you mentioned "txn-begin-time-total"
>>> later in the doc, but did not include it as a separate metric, and
>>> similarly, should we have a `txn-abort-time-total` as well? If yes,
>>> could
>>> you update the KIP page accordingly.
>>>
>>> `beginTransaction` is not blocking - I meant to remove that from that
>>> doc. I'll add something for abort.
>>>
>>> On Mon, Jul 19, 2021 at 11:55 PM Rohan Desai 
>>> wrote:
>>>
 Thanks for the review Guozhang! responding to your feedback inline:

 > 1) I agree that the current ratio metrics is just "snapshot in
 point", and
 more flexible metrics that would allow reporters to calculate based on
 window intervals are better. However, the current mechanism of the
 proposed
 metrics assumes the thread->clients mapping as of today, where each
 thread
 would own exclusively one main consumer, restore consumer, producer and
 an
 admin client. But this mapping may be subject to change in the future.
 Have
 you thought about how this metric can be extended when, e.g. the
 embedded
 clients and stream threads are de-coupled?

 Of course this depends on how exactly we refactor the runtime -
 assuming that we plan to factor out consumers into an "I/O" layer that is
 responsible for receiving records and enqueuing them to be processed by
 processing threads, then I think it should be reasonable to count the time
 we spend blocked on this internal queue(s) as blocked. The main concern
 there to me is that the I/O layer would be doing something expensive like
 decompression that shouldn't be counted as "blocked". But if that really is
 so expensive that it starts to throw off our ratios then it's probably
 indicative of a larger problem that the "i/o layer" is a bottleneck and it
 would be worth refactoring so that decompression (or insert other expensive
 thing here) can also be done on the processing threads.

 > 2) [This and all below are minor comments] The "flush-time-total" may
 better be a producer client metric, as "flush-wait-time-total", than a
 streams metric, though the streams-level "total-blocked" can still
 leverage
 it. Similarly, I think "txn-commit-time-total" and
 "offset-commit-time-total" may better be inside producer and consumer
 clients respectively.

 Good call - I'll update the KIP

 > 3) The doc was not very clear on how "thread-start-time" would be
 needed
 when calculating streams utilization along with total-blocked time,
 could
 you elaborate a bit more in the KIP?

 Yes, will do.

 > For "txn-commit-time-total" specifically, besides producer.commitTxn.
 other txn-related calls may also be 

[jira] [Created] (KAFKA-13125) Close KeyValueIterator implemented instance in internal tests (cont.)

2021-07-22 Thread Luke Chen (Jira)
Luke Chen created KAFKA-13125:
-

 Summary: Close KeyValueIterator implemented instance in internal 
tests (cont.)
 Key: KAFKA-13125
 URL: https://issues.apache.org/jira/browse/KAFKA-13125
 Project: Kafka
  Issue Type: Sub-task
Reporter: Luke Chen
Assignee: Luke Chen






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


[jira] [Created] (KAFKA-13124) Cl

2021-07-22 Thread Luke Chen (Jira)
Luke Chen created KAFKA-13124:
-

 Summary: Cl
 Key: KAFKA-13124
 URL: https://issues.apache.org/jira/browse/KAFKA-13124
 Project: Kafka
  Issue Type: Sub-task
Reporter: Luke Chen






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


[DISCUSS] KIP-764 Configurable backlog size for creating Acceptor

2021-07-22 Thread Haruki Okada
Hi, Kafka.

I proposed KIP-764, which tries to add new KafkaConfig to adjust Acceptor's
backlog size.
As described in the KIP and the ticket KAFKA-9648, currently backlog size
is fixed value (50) and it may not be enough to handle incoming connections
from massive clients.

So we would like to make it configurable.

https://cwiki.apache.org/confluence/display/KAFKA/KIP-764%3A+Configurable+backlog+size+for+creating+Acceptor

-- 

Okada Haruki
ocadar...@gmail.com



[jira] [Created] (KAFKA-13123) Close KeyValueIterator implemented instance in example codes and some tests

2021-07-22 Thread Luke Chen (Jira)
Luke Chen created KAFKA-13123:
-

 Summary: Close KeyValueIterator implemented instance in example 
codes and some tests
 Key: KAFKA-13123
 URL: https://issues.apache.org/jira/browse/KAFKA-13123
 Project: Kafka
  Issue Type: Sub-task
Reporter: Luke Chen






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


[jira] [Created] (KAFKA-13122) Close KeyValueIterator implemented instance to avoid resource leak

2021-07-22 Thread Luke Chen (Jira)
Luke Chen created KAFKA-13122:
-

 Summary: Close KeyValueIterator implemented instance to avoid 
resource leak
 Key: KAFKA-13122
 URL: https://issues.apache.org/jira/browse/KAFKA-13122
 Project: Kafka
  Issue Type: Bug
  Components: streams
Reporter: Luke Chen
Assignee: Luke Chen


Found there are "many" KeyValueIterator implemented instances don't explicitly 
get closed, which will cause resource leak. This issue mostly happen in tests 
because we usually query state store to get result iterator, and then do 
verification, but forgot close it. This issue also appear in the example code 
in our developer guide docs.



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