You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Ismael Juma <is...@juma.me.uk> on 2023/04/14 13:44:53 UTC

Test failures

Hi team,

It looks like there are a lot of test failures in the master branch. I
don't know which commits introduced them, but can you please check if
commit(s) you merged or contributed are the reason and fix it asap? If it's
easy to fix the tests, let's do that - otherwise we should revert the
faulty commit. And let's please be more careful going forward when it comes
to the PRs we merge.

An example from one of the builds, but there are many like this:

Build / JDK 17 and Scala 2.13 /
kafka.api.TransactionsBounceTest.testWithGroupMetadata()
Build / JDK 17 and Scala 2.13 /
kafka.api.TransactionsBounceTest.testWithGroupId()
Build / JDK 17 and Scala 2.13 /
kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
Build / JDK 17 and Scala 2.13 /
kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
Build / JDK 8 and Scala 2.12 /
kafka.admin.TopicCommandIntegrationTest.testDescribeUnderMinIsrPartitionsMixed(String).quorum=zk
Build / JDK 8 and Scala 2.12 /
kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
Build / JDK 8 and Scala 2.12 /
kafka.security.authorizer.AuthorizerTest.testDeleteAclOnPrefixedResource(String).quorum=kraft
Build / JDK 8 and Scala 2.12 /
kafka.security.authorizer.AuthorizerTest.testNoAclFoundOverride(String).quorum=kraft
Build / JDK 8 and Scala 2.12 /
kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
Build / JDK 8 and Scala 2.12 /
kafka.security.authorizer.AuthorizerTest.testGetAclsPrincipal(String).quorum=kraft
Build / JDK 8 and Scala 2.12 /
kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
Build / JDK 8 and Scala 2.12 /
kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testSyncTopicConfigs()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testSyncTopicConfigs()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTransactionsTest.testReplicateSourceDefault()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testConnectorReconfiguration
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsetsCustomOffsetsTopic
Build / JDK 11 and Scala 2.13 /
kafka.api.TransactionsTest.testFailureToFenceEpoch(String).quorum=kraft
Build / JDK 11 and Scala 2.13 /
kafka.security.authorizer.AuthorizerTest.testDeleteAllAclOnWildcardResource(String).quorum=kraft
Build / JDK 11 and Scala 2.13 /
kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
Build / JDK 11 and Scala 2.13 /
kafka.server.metadata.BrokerMetadataPublisherTest.testReloadUpdatedFilesWithoutConfigChange()
Build / JDK 11 and Scala 2.13 / kafka.zk.ZkMigrationIntegrationTest.[1]
Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0, Security=PLAINTEXT
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.streams.integration.AdjustStreamThreadCountTest.shouldAddAndRemoveThreadsMultipleTimes()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.trogdor.coordinator.CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testRestartReplication()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithFrequentOffsetSyncs()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForPrincipalPolicy
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.integration.ExampleConnectIntegrationTest.testSinkConnector
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
Build / JDK 11 and Scala 2.12 /
integration.kafka.server.FetchFromFollowerIntegrationTest.testRackAwareRangeAssignor()
Build / JDK 11 and Scala 2.12 /
kafka.security.authorizer.AuthorizerTest.testSuperUserHasAccess(String).quorum=kraft
Build / JDK 11 and Scala 2.12 /
kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
Build / JDK 11 and Scala 2.12 /
kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
Build / JDK 11 and Scala 2.12 /
kafka.network.SocketServerTest.testConnectionRateLimit()
Build / JDK 11 and Scala 2.12 /
kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
Build / JDK 11 and Scala 2.12 / kafka.zk.ZkMigrationIntegrationTest.[1]
Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0, Security=PLAINTEXT
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.controller.QuorumControllerTest.testDelayedConfigurationOperations()
Build / JDK 17 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
Build / JDK 17 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
Build / JDK 17 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
Build / JDK 17 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
Build / JDK 17 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
Build / JDK 17 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
Build / JDK 8 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
Build / JDK 8 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
Build / JDK 8 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
Build / JDK 8 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
Build / JDK 8 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
Build / JDK 8 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
Build / JDK 11 and Scala 2.13 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
Build / JDK 11 and Scala 2.12 /
org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()

https://ci-builds.apache.org/job/Kafka/job/kafka/job/trunk/1764/#showFailuresLink

Ismael

Re: Test failures

Posted by Sagar <sa...@gmail.com>.
Hey Greg,

I see https://issues.apache.org/jira/browse/KAFKA-14905 is marked as
resolved but I saw testSyncTopicConfigs() failed in
org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.
Here is the build :

https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka-pr/detail/PR-13453/6/tests
.

Do you think this is a flaky test which needs to be looked at?

Thanks!
Sagar.

On Thu, Apr 27, 2023 at 2:05 AM Ismael Juma <is...@juma.me.uk> wrote:

> Thanks!
>
> Ismael
>
> On Wed, Apr 26, 2023, 9:12 AM Sagar <sa...@gmail.com> wrote:
>
> > Hi,
> >
> > FYI the pr which fixes the flaky test for connect is merged to trunk
> >
> > Sagar.
> >
> > On Wed, 26 Apr 2023 at 2:45 AM, Manyanda Chitimbo <
> > manyanda.chitimbo@gmail.com> wrote:
> >
> > > HI Ismail,
> > >
> > > There is a PR to fix the failure in
> > > https://github.com/apache/kafka/pull/13634
> > >
> > > On Tue, Apr 25, 2023 at 9:40 PM Ismael Juma <is...@juma.me.uk> wrote:
> > >
> > > > I was looking at the CI builds and I came across the following test
> > > failure
> > > > that seems to be clear and consistent:
> > > >
> > > > org.mockito.exceptions.verification.TooFewActualInvocations:
> > > > kafkaBasedLog.send(<any>, <any>, <any>);
> > > > Wanted 2 times:
> > > > -> at
> > > >
> > org.apache.kafka.connect.util.KafkaBasedLog.send(KafkaBasedLog.java:376)
> > > > But was 1 time:
> > > > -> at
> > > >
> > >
> >
> org.apache.kafka.connect.storage.KafkaStatusBackingStore.sendTopicStatus(KafkaStatusBackingStore.java:315)
> > > >
> > > >
> > > >
> > >
> >
> https://ci-builds.apache.org/job/Kafka/job/kafka/job/trunk/1795/testReport/junit/org.apache.kafka.connect.storage/KafkaStatusBackingStoreFormatTest/Build___JDK_11_and_Scala_2_13___putTopicStateRetriableFailure/
> > > >
> > > > :(
> > > >
> > > > Ismael
> > > >
> > > > On Fri, Apr 21, 2023 at 5:39 PM Sagar <sa...@gmail.com>
> > wrote:
> > > >
> > > > > Hi Greg,
> > > > >
> > > > > The fix for 14929 has already been included as part of this pr :
> > > > > https://github.com/apache/kafka/pull/13594
> > > > >
> > > > > I can create a separate pr just for that flaky test if needed. Let
> me
> > > > know
> > > > > .
> > > > >
> > > > > Sagar.
> > > > >
> > > > > On Sat, 22 Apr 2023 at 3:20 AM, Greg Harris
> > > <greg.harris@aiven.io.invalid
> > > > >
> > > > > wrote:
> > > > >
> > > > > > Hey all,
> > > > > >
> > > > > > We just landed a fix for
> > > > > https://issues.apache.org/jira/browse/KAFKA-14905
> > > > > > which was causing all of
> > > > > > those MirrorConnectorsWithCustomForwardingAdminIntegrationTest
> > > > failures,
> > > > > > and we will monitor the build for any re-occurrances.
> > > > > > Unfortunately we discovered another test flake that was
> introduced
> > > > > recently
> > > > > > but that should have a straightforward resolution:
> > > > > > https://issues.apache.org/jira/browse/KAFKA-14929
> > > > > > Thanks Ismael for merging a fix for
> > > > > > https://issues.apache.org/jira/browse/KAFKA-8115 but it appears
> > that
> > > > > there
> > > > > > is still more investigation needed there, as the test is still
> > > failing
> > > > > > occasionally.
> > > > > >
> > > > > > Thanks,
> > > > > > Greg
> > > > > >
> > > > > > On Fri, Apr 14, 2023 at 12:18 PM Ismael Juma <is...@juma.me.uk>
> > > > wrote:
> > > > > >
> > > > > > > Thanks Greg! I really appreciate the help.
> > > > > > >
> > > > > > > Ismael
> > > > > > >
> > > > > > > On Fri, Apr 14, 2023 at 12:08 PM Greg Harris
> > > > > > <greg.harris@aiven.io.invalid
> > > > > > > >
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hey Ismael,
> > > > > > > >
> > > > > > > > We're working to stabilize the Connect/MM2 tests with the
> > > following
> > > > > > > issues:
> > > > > > > >
> > > > > > > > * https://issues.apache.org/jira/browse/KAFKA-14905 to
> address
> > > > > > > > MirrorConectorsWithCustomForwardingAdminIntegrationTest with
> > > > > tentative
> > > > > > > open
> > > > > > > > PR
> > > > > > > > * https://issues.apache.org/jira/browse/KAFKA-14901 to
> address
> > > > > > > > ExactlyOnceSourceIntegrationTest caused by an (apparent)
> kafka
> > > bug
> > > > > > > >
> > > > > > > > Looking at the other failures in Connect/MM2 for that build
> in
> > > > > > > particular,
> > > > > > > > it appears that most of them include Embedded Kafka not
> coming
> > > > > > > up/shutting
> > > > > > > > down cleanly:
> > > > > > > > * MirrorConnectorsIntegrationBaseTest
> > > > > > > > * MirrorConnectorsIntegrationExactlyOnceTest
> > > > > > > > * MirrorConnectorsIntegrationSSLTest
> > > > > > > > * ConnectorClientPolicyIntegrationTest
> > > > > > > > * ConnectorTopicsIntegrationTest
> > > > > > > > * ExactlyOnceSourceIntegrationTest
> > > > > > > > * OffsetsApiIntegrationTest
> > > > > > > > I'll start investigating these failures to learn more.
> > > > > > > >
> > > > > > > > I also have a few older flaky test improvements that have not
> > > been
> > > > > > > reviewed
> > > > > > > > or merged yet:
> > > > > > > > * https://issues.apache.org/jira/browse/KAFKA-8115 to
> > > > > > > > address CoordinatorTest (reappeared in the linked build)
> > > > > > > > * https://issues.apache.org/jira/browse/KAFKA-14345 to
> address
> > > > > > > > (Dynamic)ConnectionQuotaTest
> > > > > > > >
> > > > > > > > It also appears that the flakey AuthorizerTest should be
> > > addressed
> > > > by
> > > > > > > > https://github.com/apache/kafka/pull/13543 which is on trunk
> > now
> > > > but
> > > > > > > > wasn't
> > > > > > > > at the time of the above run.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Greg
> > > > > > > >
> > > > > > > > On Fri, Apr 14, 2023 at 10:25 AM Ismael Juma <
> > ismael@juma.me.uk>
> > > > > > wrote:
> > > > > > > >
> > > > > > > > > Thanks Justine!
> > > > > > > > >
> > > > > > > > > Ismael
> > > > > > > > >
> > > > > > > > > On Fri, Apr 14, 2023 at 9:53 AM Justine Olshan
> > > > > > > > > <jo...@confluent.io.invalid>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hey Ismael -- thanks for bringing this up.
> > > > > > > > > > I've filed
> > https://issues.apache.org/jira/browse/KAFKA-14904
> > > > and
> > > > > > am
> > > > > > > > > > working
> > > > > > > > > > on it now.
> > > > > > > > > >
> > > > > > > > > > I hope the other tests get fixed soon.
> > > > > > > > > >
> > > > > > > > > > On Fri, Apr 14, 2023 at 6:47 AM Ismael Juma <
> > > ismael@juma.me.uk
> > > > >
> > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi team,
> > > > > > > > > > >
> > > > > > > > > > > It looks like there are a lot of test failures in the
> > > master
> > > > > > > branch.
> > > > > > > > I
> > > > > > > > > > > don't know which commits introduced them, but can you
> > > please
> > > > > > check
> > > > > > > if
> > > > > > > > > > > commit(s) you merged or contributed are the reason and
> > fix
> > > it
> > > > > > asap?
> > > > > > > > If
> > > > > > > > > > it's
> > > > > > > > > > > easy to fix the tests, let's do that - otherwise we
> > should
> > > > > revert
> > > > > > > the
> > > > > > > > > > > faulty commit. And let's please be more careful going
> > > forward
> > > > > > when
> > > > > > > it
> > > > > > > > > > comes
> > > > > > > > > > > to the PRs we merge.
> > > > > > > > > > >
> > > > > > > > > > > An example from one of the builds, but there are many
> > like
> > > > > this:
> > > > > > > > > > >
> > > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > > >
> kafka.api.TransactionsBounceTest.testWithGroupMetadata()
> > > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > > > kafka.api.TransactionsBounceTest.testWithGroupId()
> > > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.admin.TopicCommandIntegrationTest.testDescribeUnderMinIsrPartitionsMixed(String).quorum=zk
> > > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAclOnPrefixedResource(String).quorum=kraft
> > > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testNoAclFoundOverride(String).quorum=kraft
> > > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testGetAclsPrincipal(String).quorum=kraft
> > > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testSyncTopicConfigs()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testSyncTopicConfigs()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTransactionsTest.testReplicateSourceDefault()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testConnectorReconfiguration
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsetsCustomOffsetsTopic
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > >
> > > > >
> > kafka.api.TransactionsTest.testFailureToFenceEpoch(String).quorum=kraft
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAllAclOnWildcardResource(String).quorum=kraft
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.server.metadata.BrokerMetadataPublisherTest.testReloadUpdatedFilesWithoutConfigChange()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > kafka.zk.ZkMigrationIntegrationTest.[1]
> > > > > > > > > > > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0,
> > > > > > > > > Security=PLAINTEXT
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.streams.integration.AdjustStreamThreadCountTest.shouldAddAndRemoveThreadsMultipleTimes()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.trogdor.coordinator.CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testRestartReplication()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForPrincipalPolicy
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExampleConnectIntegrationTest.testSinkConnector
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> integration.kafka.server.FetchFromFollowerIntegrationTest.testRackAwareRangeAssignor()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSuperUserHasAccess(String).quorum=kraft
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> kafka.network.SocketServerTest.testConnectionRateLimit()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > kafka.zk.ZkMigrationIntegrationTest.[1]
> > > > > > > > > > > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0,
> > > > > > > > > Security=PLAINTEXT
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.controller.QuorumControllerTest.testDelayedConfigurationOperations()
> > > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://ci-builds.apache.org/job/Kafka/job/kafka/job/trunk/1764/#showFailuresLink
> > > > > > > > > > >
> > > > > > > > > > > Ismael
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> > >
> > > --
> > > Manyanda Chitimbo.
> > >
> >
>

Re: Test failures

Posted by Ismael Juma <is...@juma.me.uk>.
Thanks!

Ismael

On Wed, Apr 26, 2023, 9:12 AM Sagar <sa...@gmail.com> wrote:

> Hi,
>
> FYI the pr which fixes the flaky test for connect is merged to trunk
>
> Sagar.
>
> On Wed, 26 Apr 2023 at 2:45 AM, Manyanda Chitimbo <
> manyanda.chitimbo@gmail.com> wrote:
>
> > HI Ismail,
> >
> > There is a PR to fix the failure in
> > https://github.com/apache/kafka/pull/13634
> >
> > On Tue, Apr 25, 2023 at 9:40 PM Ismael Juma <is...@juma.me.uk> wrote:
> >
> > > I was looking at the CI builds and I came across the following test
> > failure
> > > that seems to be clear and consistent:
> > >
> > > org.mockito.exceptions.verification.TooFewActualInvocations:
> > > kafkaBasedLog.send(<any>, <any>, <any>);
> > > Wanted 2 times:
> > > -> at
> > >
> org.apache.kafka.connect.util.KafkaBasedLog.send(KafkaBasedLog.java:376)
> > > But was 1 time:
> > > -> at
> > >
> >
> org.apache.kafka.connect.storage.KafkaStatusBackingStore.sendTopicStatus(KafkaStatusBackingStore.java:315)
> > >
> > >
> > >
> >
> https://ci-builds.apache.org/job/Kafka/job/kafka/job/trunk/1795/testReport/junit/org.apache.kafka.connect.storage/KafkaStatusBackingStoreFormatTest/Build___JDK_11_and_Scala_2_13___putTopicStateRetriableFailure/
> > >
> > > :(
> > >
> > > Ismael
> > >
> > > On Fri, Apr 21, 2023 at 5:39 PM Sagar <sa...@gmail.com>
> wrote:
> > >
> > > > Hi Greg,
> > > >
> > > > The fix for 14929 has already been included as part of this pr :
> > > > https://github.com/apache/kafka/pull/13594
> > > >
> > > > I can create a separate pr just for that flaky test if needed. Let me
> > > know
> > > > .
> > > >
> > > > Sagar.
> > > >
> > > > On Sat, 22 Apr 2023 at 3:20 AM, Greg Harris
> > <greg.harris@aiven.io.invalid
> > > >
> > > > wrote:
> > > >
> > > > > Hey all,
> > > > >
> > > > > We just landed a fix for
> > > > https://issues.apache.org/jira/browse/KAFKA-14905
> > > > > which was causing all of
> > > > > those MirrorConnectorsWithCustomForwardingAdminIntegrationTest
> > > failures,
> > > > > and we will monitor the build for any re-occurrances.
> > > > > Unfortunately we discovered another test flake that was introduced
> > > > recently
> > > > > but that should have a straightforward resolution:
> > > > > https://issues.apache.org/jira/browse/KAFKA-14929
> > > > > Thanks Ismael for merging a fix for
> > > > > https://issues.apache.org/jira/browse/KAFKA-8115 but it appears
> that
> > > > there
> > > > > is still more investigation needed there, as the test is still
> > failing
> > > > > occasionally.
> > > > >
> > > > > Thanks,
> > > > > Greg
> > > > >
> > > > > On Fri, Apr 14, 2023 at 12:18 PM Ismael Juma <is...@juma.me.uk>
> > > wrote:
> > > > >
> > > > > > Thanks Greg! I really appreciate the help.
> > > > > >
> > > > > > Ismael
> > > > > >
> > > > > > On Fri, Apr 14, 2023 at 12:08 PM Greg Harris
> > > > > <greg.harris@aiven.io.invalid
> > > > > > >
> > > > > > wrote:
> > > > > >
> > > > > > > Hey Ismael,
> > > > > > >
> > > > > > > We're working to stabilize the Connect/MM2 tests with the
> > following
> > > > > > issues:
> > > > > > >
> > > > > > > * https://issues.apache.org/jira/browse/KAFKA-14905 to address
> > > > > > > MirrorConectorsWithCustomForwardingAdminIntegrationTest with
> > > > tentative
> > > > > > open
> > > > > > > PR
> > > > > > > * https://issues.apache.org/jira/browse/KAFKA-14901 to address
> > > > > > > ExactlyOnceSourceIntegrationTest caused by an (apparent) kafka
> > bug
> > > > > > >
> > > > > > > Looking at the other failures in Connect/MM2 for that build in
> > > > > > particular,
> > > > > > > it appears that most of them include Embedded Kafka not coming
> > > > > > up/shutting
> > > > > > > down cleanly:
> > > > > > > * MirrorConnectorsIntegrationBaseTest
> > > > > > > * MirrorConnectorsIntegrationExactlyOnceTest
> > > > > > > * MirrorConnectorsIntegrationSSLTest
> > > > > > > * ConnectorClientPolicyIntegrationTest
> > > > > > > * ConnectorTopicsIntegrationTest
> > > > > > > * ExactlyOnceSourceIntegrationTest
> > > > > > > * OffsetsApiIntegrationTest
> > > > > > > I'll start investigating these failures to learn more.
> > > > > > >
> > > > > > > I also have a few older flaky test improvements that have not
> > been
> > > > > > reviewed
> > > > > > > or merged yet:
> > > > > > > * https://issues.apache.org/jira/browse/KAFKA-8115 to
> > > > > > > address CoordinatorTest (reappeared in the linked build)
> > > > > > > * https://issues.apache.org/jira/browse/KAFKA-14345 to address
> > > > > > > (Dynamic)ConnectionQuotaTest
> > > > > > >
> > > > > > > It also appears that the flakey AuthorizerTest should be
> > addressed
> > > by
> > > > > > > https://github.com/apache/kafka/pull/13543 which is on trunk
> now
> > > but
> > > > > > > wasn't
> > > > > > > at the time of the above run.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Greg
> > > > > > >
> > > > > > > On Fri, Apr 14, 2023 at 10:25 AM Ismael Juma <
> ismael@juma.me.uk>
> > > > > wrote:
> > > > > > >
> > > > > > > > Thanks Justine!
> > > > > > > >
> > > > > > > > Ismael
> > > > > > > >
> > > > > > > > On Fri, Apr 14, 2023 at 9:53 AM Justine Olshan
> > > > > > > > <jo...@confluent.io.invalid>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hey Ismael -- thanks for bringing this up.
> > > > > > > > > I've filed
> https://issues.apache.org/jira/browse/KAFKA-14904
> > > and
> > > > > am
> > > > > > > > > working
> > > > > > > > > on it now.
> > > > > > > > >
> > > > > > > > > I hope the other tests get fixed soon.
> > > > > > > > >
> > > > > > > > > On Fri, Apr 14, 2023 at 6:47 AM Ismael Juma <
> > ismael@juma.me.uk
> > > >
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi team,
> > > > > > > > > >
> > > > > > > > > > It looks like there are a lot of test failures in the
> > master
> > > > > > branch.
> > > > > > > I
> > > > > > > > > > don't know which commits introduced them, but can you
> > please
> > > > > check
> > > > > > if
> > > > > > > > > > commit(s) you merged or contributed are the reason and
> fix
> > it
> > > > > asap?
> > > > > > > If
> > > > > > > > > it's
> > > > > > > > > > easy to fix the tests, let's do that - otherwise we
> should
> > > > revert
> > > > > > the
> > > > > > > > > > faulty commit. And let's please be more careful going
> > forward
> > > > > when
> > > > > > it
> > > > > > > > > comes
> > > > > > > > > > to the PRs we merge.
> > > > > > > > > >
> > > > > > > > > > An example from one of the builds, but there are many
> like
> > > > this:
> > > > > > > > > >
> > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > > kafka.api.TransactionsBounceTest.testWithGroupMetadata()
> > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > > kafka.api.TransactionsBounceTest.testWithGroupId()
> > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.admin.TopicCommandIntegrationTest.testDescribeUnderMinIsrPartitionsMixed(String).quorum=zk
> > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAclOnPrefixedResource(String).quorum=kraft
> > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testNoAclFoundOverride(String).quorum=kraft
> > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testGetAclsPrincipal(String).quorum=kraft
> > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testSyncTopicConfigs()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testSyncTopicConfigs()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTransactionsTest.testReplicateSourceDefault()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testConnectorReconfiguration
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsetsCustomOffsetsTopic
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > >
> > > >
> kafka.api.TransactionsTest.testFailureToFenceEpoch(String).quorum=kraft
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAllAclOnWildcardResource(String).quorum=kraft
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.server.metadata.BrokerMetadataPublisherTest.testReloadUpdatedFilesWithoutConfigChange()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > kafka.zk.ZkMigrationIntegrationTest.[1]
> > > > > > > > > > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0,
> > > > > > > > Security=PLAINTEXT
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.streams.integration.AdjustStreamThreadCountTest.shouldAddAndRemoveThreadsMultipleTimes()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.trogdor.coordinator.CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testRestartReplication()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForPrincipalPolicy
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExampleConnectIntegrationTest.testSinkConnector
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> integration.kafka.server.FetchFromFollowerIntegrationTest.testRackAwareRangeAssignor()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSuperUserHasAccess(String).quorum=kraft
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > > kafka.network.SocketServerTest.testConnectionRateLimit()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > kafka.zk.ZkMigrationIntegrationTest.[1]
> > > > > > > > > > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0,
> > > > > > > > Security=PLAINTEXT
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.controller.QuorumControllerTest.testDelayedConfigurationOperations()
> > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://ci-builds.apache.org/job/Kafka/job/kafka/job/trunk/1764/#showFailuresLink
> > > > > > > > > >
> > > > > > > > > > Ismael
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> >
> > --
> > Manyanda Chitimbo.
> >
>

Re: Test failures

Posted by Sagar <sa...@gmail.com>.
Hi,

FYI the pr which fixes the flaky test for connect is merged to trunk

Sagar.

On Wed, 26 Apr 2023 at 2:45 AM, Manyanda Chitimbo <
manyanda.chitimbo@gmail.com> wrote:

> HI Ismail,
>
> There is a PR to fix the failure in
> https://github.com/apache/kafka/pull/13634
>
> On Tue, Apr 25, 2023 at 9:40 PM Ismael Juma <is...@juma.me.uk> wrote:
>
> > I was looking at the CI builds and I came across the following test
> failure
> > that seems to be clear and consistent:
> >
> > org.mockito.exceptions.verification.TooFewActualInvocations:
> > kafkaBasedLog.send(<any>, <any>, <any>);
> > Wanted 2 times:
> > -> at
> > org.apache.kafka.connect.util.KafkaBasedLog.send(KafkaBasedLog.java:376)
> > But was 1 time:
> > -> at
> >
> org.apache.kafka.connect.storage.KafkaStatusBackingStore.sendTopicStatus(KafkaStatusBackingStore.java:315)
> >
> >
> >
> https://ci-builds.apache.org/job/Kafka/job/kafka/job/trunk/1795/testReport/junit/org.apache.kafka.connect.storage/KafkaStatusBackingStoreFormatTest/Build___JDK_11_and_Scala_2_13___putTopicStateRetriableFailure/
> >
> > :(
> >
> > Ismael
> >
> > On Fri, Apr 21, 2023 at 5:39 PM Sagar <sa...@gmail.com> wrote:
> >
> > > Hi Greg,
> > >
> > > The fix for 14929 has already been included as part of this pr :
> > > https://github.com/apache/kafka/pull/13594
> > >
> > > I can create a separate pr just for that flaky test if needed. Let me
> > know
> > > .
> > >
> > > Sagar.
> > >
> > > On Sat, 22 Apr 2023 at 3:20 AM, Greg Harris
> <greg.harris@aiven.io.invalid
> > >
> > > wrote:
> > >
> > > > Hey all,
> > > >
> > > > We just landed a fix for
> > > https://issues.apache.org/jira/browse/KAFKA-14905
> > > > which was causing all of
> > > > those MirrorConnectorsWithCustomForwardingAdminIntegrationTest
> > failures,
> > > > and we will monitor the build for any re-occurrances.
> > > > Unfortunately we discovered another test flake that was introduced
> > > recently
> > > > but that should have a straightforward resolution:
> > > > https://issues.apache.org/jira/browse/KAFKA-14929
> > > > Thanks Ismael for merging a fix for
> > > > https://issues.apache.org/jira/browse/KAFKA-8115 but it appears that
> > > there
> > > > is still more investigation needed there, as the test is still
> failing
> > > > occasionally.
> > > >
> > > > Thanks,
> > > > Greg
> > > >
> > > > On Fri, Apr 14, 2023 at 12:18 PM Ismael Juma <is...@juma.me.uk>
> > wrote:
> > > >
> > > > > Thanks Greg! I really appreciate the help.
> > > > >
> > > > > Ismael
> > > > >
> > > > > On Fri, Apr 14, 2023 at 12:08 PM Greg Harris
> > > > <greg.harris@aiven.io.invalid
> > > > > >
> > > > > wrote:
> > > > >
> > > > > > Hey Ismael,
> > > > > >
> > > > > > We're working to stabilize the Connect/MM2 tests with the
> following
> > > > > issues:
> > > > > >
> > > > > > * https://issues.apache.org/jira/browse/KAFKA-14905 to address
> > > > > > MirrorConectorsWithCustomForwardingAdminIntegrationTest with
> > > tentative
> > > > > open
> > > > > > PR
> > > > > > * https://issues.apache.org/jira/browse/KAFKA-14901 to address
> > > > > > ExactlyOnceSourceIntegrationTest caused by an (apparent) kafka
> bug
> > > > > >
> > > > > > Looking at the other failures in Connect/MM2 for that build in
> > > > > particular,
> > > > > > it appears that most of them include Embedded Kafka not coming
> > > > > up/shutting
> > > > > > down cleanly:
> > > > > > * MirrorConnectorsIntegrationBaseTest
> > > > > > * MirrorConnectorsIntegrationExactlyOnceTest
> > > > > > * MirrorConnectorsIntegrationSSLTest
> > > > > > * ConnectorClientPolicyIntegrationTest
> > > > > > * ConnectorTopicsIntegrationTest
> > > > > > * ExactlyOnceSourceIntegrationTest
> > > > > > * OffsetsApiIntegrationTest
> > > > > > I'll start investigating these failures to learn more.
> > > > > >
> > > > > > I also have a few older flaky test improvements that have not
> been
> > > > > reviewed
> > > > > > or merged yet:
> > > > > > * https://issues.apache.org/jira/browse/KAFKA-8115 to
> > > > > > address CoordinatorTest (reappeared in the linked build)
> > > > > > * https://issues.apache.org/jira/browse/KAFKA-14345 to address
> > > > > > (Dynamic)ConnectionQuotaTest
> > > > > >
> > > > > > It also appears that the flakey AuthorizerTest should be
> addressed
> > by
> > > > > > https://github.com/apache/kafka/pull/13543 which is on trunk now
> > but
> > > > > > wasn't
> > > > > > at the time of the above run.
> > > > > >
> > > > > > Thanks,
> > > > > > Greg
> > > > > >
> > > > > > On Fri, Apr 14, 2023 at 10:25 AM Ismael Juma <is...@juma.me.uk>
> > > > wrote:
> > > > > >
> > > > > > > Thanks Justine!
> > > > > > >
> > > > > > > Ismael
> > > > > > >
> > > > > > > On Fri, Apr 14, 2023 at 9:53 AM Justine Olshan
> > > > > > > <jo...@confluent.io.invalid>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hey Ismael -- thanks for bringing this up.
> > > > > > > > I've filed https://issues.apache.org/jira/browse/KAFKA-14904
> > and
> > > > am
> > > > > > > > working
> > > > > > > > on it now.
> > > > > > > >
> > > > > > > > I hope the other tests get fixed soon.
> > > > > > > >
> > > > > > > > On Fri, Apr 14, 2023 at 6:47 AM Ismael Juma <
> ismael@juma.me.uk
> > >
> > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi team,
> > > > > > > > >
> > > > > > > > > It looks like there are a lot of test failures in the
> master
> > > > > branch.
> > > > > > I
> > > > > > > > > don't know which commits introduced them, but can you
> please
> > > > check
> > > > > if
> > > > > > > > > commit(s) you merged or contributed are the reason and fix
> it
> > > > asap?
> > > > > > If
> > > > > > > > it's
> > > > > > > > > easy to fix the tests, let's do that - otherwise we should
> > > revert
> > > > > the
> > > > > > > > > faulty commit. And let's please be more careful going
> forward
> > > > when
> > > > > it
> > > > > > > > comes
> > > > > > > > > to the PRs we merge.
> > > > > > > > >
> > > > > > > > > An example from one of the builds, but there are many like
> > > this:
> > > > > > > > >
> > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > kafka.api.TransactionsBounceTest.testWithGroupMetadata()
> > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > > kafka.api.TransactionsBounceTest.testWithGroupId()
> > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.admin.TopicCommandIntegrationTest.testDescribeUnderMinIsrPartitionsMixed(String).quorum=zk
> > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAclOnPrefixedResource(String).quorum=kraft
> > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testNoAclFoundOverride(String).quorum=kraft
> > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testGetAclsPrincipal(String).quorum=kraft
> > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testSyncTopicConfigs()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testSyncTopicConfigs()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTransactionsTest.testReplicateSourceDefault()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testConnectorReconfiguration
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsetsCustomOffsetsTopic
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > >
> > > kafka.api.TransactionsTest.testFailureToFenceEpoch(String).quorum=kraft
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAllAclOnWildcardResource(String).quorum=kraft
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.server.metadata.BrokerMetadataPublisherTest.testReloadUpdatedFilesWithoutConfigChange()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > kafka.zk.ZkMigrationIntegrationTest.[1]
> > > > > > > > > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0,
> > > > > > > Security=PLAINTEXT
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.streams.integration.AdjustStreamThreadCountTest.shouldAddAndRemoveThreadsMultipleTimes()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.trogdor.coordinator.CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testRestartReplication()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForPrincipalPolicy
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExampleConnectIntegrationTest.testSinkConnector
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> integration.kafka.server.FetchFromFollowerIntegrationTest.testRackAwareRangeAssignor()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSuperUserHasAccess(String).quorum=kraft
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > > kafka.network.SocketServerTest.testConnectionRateLimit()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > kafka.zk.ZkMigrationIntegrationTest.[1]
> > > > > > > > > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0,
> > > > > > > Security=PLAINTEXT
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.controller.QuorumControllerTest.testDelayedConfigurationOperations()
> > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://ci-builds.apache.org/job/Kafka/job/kafka/job/trunk/1764/#showFailuresLink
> > > > > > > > >
> > > > > > > > > Ismael
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>
>
> --
> Manyanda Chitimbo.
>

Re: Test failures

Posted by Manyanda Chitimbo <ma...@gmail.com>.
HI Ismail,

There is a PR to fix the failure in
https://github.com/apache/kafka/pull/13634

On Tue, Apr 25, 2023 at 9:40 PM Ismael Juma <is...@juma.me.uk> wrote:

> I was looking at the CI builds and I came across the following test failure
> that seems to be clear and consistent:
>
> org.mockito.exceptions.verification.TooFewActualInvocations:
> kafkaBasedLog.send(<any>, <any>, <any>);
> Wanted 2 times:
> -> at
> org.apache.kafka.connect.util.KafkaBasedLog.send(KafkaBasedLog.java:376)
> But was 1 time:
> -> at
> org.apache.kafka.connect.storage.KafkaStatusBackingStore.sendTopicStatus(KafkaStatusBackingStore.java:315)
>
>
> https://ci-builds.apache.org/job/Kafka/job/kafka/job/trunk/1795/testReport/junit/org.apache.kafka.connect.storage/KafkaStatusBackingStoreFormatTest/Build___JDK_11_and_Scala_2_13___putTopicStateRetriableFailure/
>
> :(
>
> Ismael
>
> On Fri, Apr 21, 2023 at 5:39 PM Sagar <sa...@gmail.com> wrote:
>
> > Hi Greg,
> >
> > The fix for 14929 has already been included as part of this pr :
> > https://github.com/apache/kafka/pull/13594
> >
> > I can create a separate pr just for that flaky test if needed. Let me
> know
> > .
> >
> > Sagar.
> >
> > On Sat, 22 Apr 2023 at 3:20 AM, Greg Harris <greg.harris@aiven.io.invalid
> >
> > wrote:
> >
> > > Hey all,
> > >
> > > We just landed a fix for
> > https://issues.apache.org/jira/browse/KAFKA-14905
> > > which was causing all of
> > > those MirrorConnectorsWithCustomForwardingAdminIntegrationTest
> failures,
> > > and we will monitor the build for any re-occurrances.
> > > Unfortunately we discovered another test flake that was introduced
> > recently
> > > but that should have a straightforward resolution:
> > > https://issues.apache.org/jira/browse/KAFKA-14929
> > > Thanks Ismael for merging a fix for
> > > https://issues.apache.org/jira/browse/KAFKA-8115 but it appears that
> > there
> > > is still more investigation needed there, as the test is still failing
> > > occasionally.
> > >
> > > Thanks,
> > > Greg
> > >
> > > On Fri, Apr 14, 2023 at 12:18 PM Ismael Juma <is...@juma.me.uk>
> wrote:
> > >
> > > > Thanks Greg! I really appreciate the help.
> > > >
> > > > Ismael
> > > >
> > > > On Fri, Apr 14, 2023 at 12:08 PM Greg Harris
> > > <greg.harris@aiven.io.invalid
> > > > >
> > > > wrote:
> > > >
> > > > > Hey Ismael,
> > > > >
> > > > > We're working to stabilize the Connect/MM2 tests with the following
> > > > issues:
> > > > >
> > > > > * https://issues.apache.org/jira/browse/KAFKA-14905 to address
> > > > > MirrorConectorsWithCustomForwardingAdminIntegrationTest with
> > tentative
> > > > open
> > > > > PR
> > > > > * https://issues.apache.org/jira/browse/KAFKA-14901 to address
> > > > > ExactlyOnceSourceIntegrationTest caused by an (apparent) kafka bug
> > > > >
> > > > > Looking at the other failures in Connect/MM2 for that build in
> > > > particular,
> > > > > it appears that most of them include Embedded Kafka not coming
> > > > up/shutting
> > > > > down cleanly:
> > > > > * MirrorConnectorsIntegrationBaseTest
> > > > > * MirrorConnectorsIntegrationExactlyOnceTest
> > > > > * MirrorConnectorsIntegrationSSLTest
> > > > > * ConnectorClientPolicyIntegrationTest
> > > > > * ConnectorTopicsIntegrationTest
> > > > > * ExactlyOnceSourceIntegrationTest
> > > > > * OffsetsApiIntegrationTest
> > > > > I'll start investigating these failures to learn more.
> > > > >
> > > > > I also have a few older flaky test improvements that have not been
> > > > reviewed
> > > > > or merged yet:
> > > > > * https://issues.apache.org/jira/browse/KAFKA-8115 to
> > > > > address CoordinatorTest (reappeared in the linked build)
> > > > > * https://issues.apache.org/jira/browse/KAFKA-14345 to address
> > > > > (Dynamic)ConnectionQuotaTest
> > > > >
> > > > > It also appears that the flakey AuthorizerTest should be addressed
> by
> > > > > https://github.com/apache/kafka/pull/13543 which is on trunk now
> but
> > > > > wasn't
> > > > > at the time of the above run.
> > > > >
> > > > > Thanks,
> > > > > Greg
> > > > >
> > > > > On Fri, Apr 14, 2023 at 10:25 AM Ismael Juma <is...@juma.me.uk>
> > > wrote:
> > > > >
> > > > > > Thanks Justine!
> > > > > >
> > > > > > Ismael
> > > > > >
> > > > > > On Fri, Apr 14, 2023 at 9:53 AM Justine Olshan
> > > > > > <jo...@confluent.io.invalid>
> > > > > > wrote:
> > > > > >
> > > > > > > Hey Ismael -- thanks for bringing this up.
> > > > > > > I've filed https://issues.apache.org/jira/browse/KAFKA-14904
> and
> > > am
> > > > > > > working
> > > > > > > on it now.
> > > > > > >
> > > > > > > I hope the other tests get fixed soon.
> > > > > > >
> > > > > > > On Fri, Apr 14, 2023 at 6:47 AM Ismael Juma <ismael@juma.me.uk
> >
> > > > wrote:
> > > > > > >
> > > > > > > > Hi team,
> > > > > > > >
> > > > > > > > It looks like there are a lot of test failures in the master
> > > > branch.
> > > > > I
> > > > > > > > don't know which commits introduced them, but can you please
> > > check
> > > > if
> > > > > > > > commit(s) you merged or contributed are the reason and fix it
> > > asap?
> > > > > If
> > > > > > > it's
> > > > > > > > easy to fix the tests, let's do that - otherwise we should
> > revert
> > > > the
> > > > > > > > faulty commit. And let's please be more careful going forward
> > > when
> > > > it
> > > > > > > comes
> > > > > > > > to the PRs we merge.
> > > > > > > >
> > > > > > > > An example from one of the builds, but there are many like
> > this:
> > > > > > > >
> > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > kafka.api.TransactionsBounceTest.testWithGroupMetadata()
> > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > > kafka.api.TransactionsBounceTest.testWithGroupId()
> > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.admin.TopicCommandIntegrationTest.testDescribeUnderMinIsrPartitionsMixed(String).quorum=zk
> > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAclOnPrefixedResource(String).quorum=kraft
> > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testNoAclFoundOverride(String).quorum=kraft
> > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testGetAclsPrincipal(String).quorum=kraft
> > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testSyncTopicConfigs()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testSyncTopicConfigs()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTransactionsTest.testReplicateSourceDefault()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testConnectorReconfiguration
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsetsCustomOffsetsTopic
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > >
> > kafka.api.TransactionsTest.testFailureToFenceEpoch(String).quorum=kraft
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAllAclOnWildcardResource(String).quorum=kraft
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > > kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.server.metadata.BrokerMetadataPublisherTest.testReloadUpdatedFilesWithoutConfigChange()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > kafka.zk.ZkMigrationIntegrationTest.[1]
> > > > > > > > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0,
> > > > > > Security=PLAINTEXT
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.streams.integration.AdjustStreamThreadCountTest.shouldAddAndRemoveThreadsMultipleTimes()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.trogdor.coordinator.CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testRestartReplication()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForPrincipalPolicy
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExampleConnectIntegrationTest.testSinkConnector
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> integration.kafka.server.FetchFromFollowerIntegrationTest.testRackAwareRangeAssignor()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSuperUserHasAccess(String).quorum=kraft
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > kafka.network.SocketServerTest.testConnectionRateLimit()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > > kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > kafka.zk.ZkMigrationIntegrationTest.[1]
> > > > > > > > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0,
> > > > > > Security=PLAINTEXT
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.controller.QuorumControllerTest.testDelayedConfigurationOperations()
> > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://ci-builds.apache.org/job/Kafka/job/kafka/job/trunk/1764/#showFailuresLink
> > > > > > > >
> > > > > > > > Ismael
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>


-- 
Manyanda Chitimbo.

Re: Test failures

Posted by Ismael Juma <is...@juma.me.uk>.
I was looking at the CI builds and I came across the following test failure
that seems to be clear and consistent:

org.mockito.exceptions.verification.TooFewActualInvocations:
kafkaBasedLog.send(<any>, <any>, <any>);
Wanted 2 times:
-> at org.apache.kafka.connect.util.KafkaBasedLog.send(KafkaBasedLog.java:376)
But was 1 time:
-> at org.apache.kafka.connect.storage.KafkaStatusBackingStore.sendTopicStatus(KafkaStatusBackingStore.java:315)

https://ci-builds.apache.org/job/Kafka/job/kafka/job/trunk/1795/testReport/junit/org.apache.kafka.connect.storage/KafkaStatusBackingStoreFormatTest/Build___JDK_11_and_Scala_2_13___putTopicStateRetriableFailure/

:(

Ismael

On Fri, Apr 21, 2023 at 5:39 PM Sagar <sa...@gmail.com> wrote:

> Hi Greg,
>
> The fix for 14929 has already been included as part of this pr :
> https://github.com/apache/kafka/pull/13594
>
> I can create a separate pr just for that flaky test if needed. Let me know
> .
>
> Sagar.
>
> On Sat, 22 Apr 2023 at 3:20 AM, Greg Harris <gr...@aiven.io.invalid>
> wrote:
>
> > Hey all,
> >
> > We just landed a fix for
> https://issues.apache.org/jira/browse/KAFKA-14905
> > which was causing all of
> > those MirrorConnectorsWithCustomForwardingAdminIntegrationTest failures,
> > and we will monitor the build for any re-occurrances.
> > Unfortunately we discovered another test flake that was introduced
> recently
> > but that should have a straightforward resolution:
> > https://issues.apache.org/jira/browse/KAFKA-14929
> > Thanks Ismael for merging a fix for
> > https://issues.apache.org/jira/browse/KAFKA-8115 but it appears that
> there
> > is still more investigation needed there, as the test is still failing
> > occasionally.
> >
> > Thanks,
> > Greg
> >
> > On Fri, Apr 14, 2023 at 12:18 PM Ismael Juma <is...@juma.me.uk> wrote:
> >
> > > Thanks Greg! I really appreciate the help.
> > >
> > > Ismael
> > >
> > > On Fri, Apr 14, 2023 at 12:08 PM Greg Harris
> > <greg.harris@aiven.io.invalid
> > > >
> > > wrote:
> > >
> > > > Hey Ismael,
> > > >
> > > > We're working to stabilize the Connect/MM2 tests with the following
> > > issues:
> > > >
> > > > * https://issues.apache.org/jira/browse/KAFKA-14905 to address
> > > > MirrorConectorsWithCustomForwardingAdminIntegrationTest with
> tentative
> > > open
> > > > PR
> > > > * https://issues.apache.org/jira/browse/KAFKA-14901 to address
> > > > ExactlyOnceSourceIntegrationTest caused by an (apparent) kafka bug
> > > >
> > > > Looking at the other failures in Connect/MM2 for that build in
> > > particular,
> > > > it appears that most of them include Embedded Kafka not coming
> > > up/shutting
> > > > down cleanly:
> > > > * MirrorConnectorsIntegrationBaseTest
> > > > * MirrorConnectorsIntegrationExactlyOnceTest
> > > > * MirrorConnectorsIntegrationSSLTest
> > > > * ConnectorClientPolicyIntegrationTest
> > > > * ConnectorTopicsIntegrationTest
> > > > * ExactlyOnceSourceIntegrationTest
> > > > * OffsetsApiIntegrationTest
> > > > I'll start investigating these failures to learn more.
> > > >
> > > > I also have a few older flaky test improvements that have not been
> > > reviewed
> > > > or merged yet:
> > > > * https://issues.apache.org/jira/browse/KAFKA-8115 to
> > > > address CoordinatorTest (reappeared in the linked build)
> > > > * https://issues.apache.org/jira/browse/KAFKA-14345 to address
> > > > (Dynamic)ConnectionQuotaTest
> > > >
> > > > It also appears that the flakey AuthorizerTest should be addressed by
> > > > https://github.com/apache/kafka/pull/13543 which is on trunk now but
> > > > wasn't
> > > > at the time of the above run.
> > > >
> > > > Thanks,
> > > > Greg
> > > >
> > > > On Fri, Apr 14, 2023 at 10:25 AM Ismael Juma <is...@juma.me.uk>
> > wrote:
> > > >
> > > > > Thanks Justine!
> > > > >
> > > > > Ismael
> > > > >
> > > > > On Fri, Apr 14, 2023 at 9:53 AM Justine Olshan
> > > > > <jo...@confluent.io.invalid>
> > > > > wrote:
> > > > >
> > > > > > Hey Ismael -- thanks for bringing this up.
> > > > > > I've filed https://issues.apache.org/jira/browse/KAFKA-14904 and
> > am
> > > > > > working
> > > > > > on it now.
> > > > > >
> > > > > > I hope the other tests get fixed soon.
> > > > > >
> > > > > > On Fri, Apr 14, 2023 at 6:47 AM Ismael Juma <is...@juma.me.uk>
> > > wrote:
> > > > > >
> > > > > > > Hi team,
> > > > > > >
> > > > > > > It looks like there are a lot of test failures in the master
> > > branch.
> > > > I
> > > > > > > don't know which commits introduced them, but can you please
> > check
> > > if
> > > > > > > commit(s) you merged or contributed are the reason and fix it
> > asap?
> > > > If
> > > > > > it's
> > > > > > > easy to fix the tests, let's do that - otherwise we should
> revert
> > > the
> > > > > > > faulty commit. And let's please be more careful going forward
> > when
> > > it
> > > > > > comes
> > > > > > > to the PRs we merge.
> > > > > > >
> > > > > > > An example from one of the builds, but there are many like
> this:
> > > > > > >
> > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > kafka.api.TransactionsBounceTest.testWithGroupMetadata()
> > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > > kafka.api.TransactionsBounceTest.testWithGroupId()
> > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.admin.TopicCommandIntegrationTest.testDescribeUnderMinIsrPartitionsMixed(String).quorum=zk
> > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAclOnPrefixedResource(String).quorum=kraft
> > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testNoAclFoundOverride(String).quorum=kraft
> > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testGetAclsPrincipal(String).quorum=kraft
> > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testSyncTopicConfigs()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testSyncTopicConfigs()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTransactionsTest.testReplicateSourceDefault()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testConnectorReconfiguration
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsetsCustomOffsetsTopic
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > >
> kafka.api.TransactionsTest.testFailureToFenceEpoch(String).quorum=kraft
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAllAclOnWildcardResource(String).quorum=kraft
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > > kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.server.metadata.BrokerMetadataPublisherTest.testReloadUpdatedFilesWithoutConfigChange()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > kafka.zk.ZkMigrationIntegrationTest.[1]
> > > > > > > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0,
> > > > > Security=PLAINTEXT
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.streams.integration.AdjustStreamThreadCountTest.shouldAddAndRemoveThreadsMultipleTimes()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.trogdor.coordinator.CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testRestartReplication()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForPrincipalPolicy
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExampleConnectIntegrationTest.testSinkConnector
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> integration.kafka.server.FetchFromFollowerIntegrationTest.testRackAwareRangeAssignor()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSuperUserHasAccess(String).quorum=kraft
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > kafka.network.SocketServerTest.testConnectionRateLimit()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > > kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > kafka.zk.ZkMigrationIntegrationTest.[1]
> > > > > > > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0,
> > > > > Security=PLAINTEXT
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.controller.QuorumControllerTest.testDelayedConfigurationOperations()
> > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://ci-builds.apache.org/job/Kafka/job/kafka/job/trunk/1764/#showFailuresLink
> > > > > > >
> > > > > > > Ismael
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Test failures

Posted by Sagar <sa...@gmail.com>.
Hi Greg,

The fix for 14929 has already been included as part of this pr :
https://github.com/apache/kafka/pull/13594

I can create a separate pr just for that flaky test if needed. Let me know .

Sagar.

On Sat, 22 Apr 2023 at 3:20 AM, Greg Harris <gr...@aiven.io.invalid>
wrote:

> Hey all,
>
> We just landed a fix for https://issues.apache.org/jira/browse/KAFKA-14905
> which was causing all of
> those MirrorConnectorsWithCustomForwardingAdminIntegrationTest failures,
> and we will monitor the build for any re-occurrances.
> Unfortunately we discovered another test flake that was introduced recently
> but that should have a straightforward resolution:
> https://issues.apache.org/jira/browse/KAFKA-14929
> Thanks Ismael for merging a fix for
> https://issues.apache.org/jira/browse/KAFKA-8115 but it appears that there
> is still more investigation needed there, as the test is still failing
> occasionally.
>
> Thanks,
> Greg
>
> On Fri, Apr 14, 2023 at 12:18 PM Ismael Juma <is...@juma.me.uk> wrote:
>
> > Thanks Greg! I really appreciate the help.
> >
> > Ismael
> >
> > On Fri, Apr 14, 2023 at 12:08 PM Greg Harris
> <greg.harris@aiven.io.invalid
> > >
> > wrote:
> >
> > > Hey Ismael,
> > >
> > > We're working to stabilize the Connect/MM2 tests with the following
> > issues:
> > >
> > > * https://issues.apache.org/jira/browse/KAFKA-14905 to address
> > > MirrorConectorsWithCustomForwardingAdminIntegrationTest with tentative
> > open
> > > PR
> > > * https://issues.apache.org/jira/browse/KAFKA-14901 to address
> > > ExactlyOnceSourceIntegrationTest caused by an (apparent) kafka bug
> > >
> > > Looking at the other failures in Connect/MM2 for that build in
> > particular,
> > > it appears that most of them include Embedded Kafka not coming
> > up/shutting
> > > down cleanly:
> > > * MirrorConnectorsIntegrationBaseTest
> > > * MirrorConnectorsIntegrationExactlyOnceTest
> > > * MirrorConnectorsIntegrationSSLTest
> > > * ConnectorClientPolicyIntegrationTest
> > > * ConnectorTopicsIntegrationTest
> > > * ExactlyOnceSourceIntegrationTest
> > > * OffsetsApiIntegrationTest
> > > I'll start investigating these failures to learn more.
> > >
> > > I also have a few older flaky test improvements that have not been
> > reviewed
> > > or merged yet:
> > > * https://issues.apache.org/jira/browse/KAFKA-8115 to
> > > address CoordinatorTest (reappeared in the linked build)
> > > * https://issues.apache.org/jira/browse/KAFKA-14345 to address
> > > (Dynamic)ConnectionQuotaTest
> > >
> > > It also appears that the flakey AuthorizerTest should be addressed by
> > > https://github.com/apache/kafka/pull/13543 which is on trunk now but
> > > wasn't
> > > at the time of the above run.
> > >
> > > Thanks,
> > > Greg
> > >
> > > On Fri, Apr 14, 2023 at 10:25 AM Ismael Juma <is...@juma.me.uk>
> wrote:
> > >
> > > > Thanks Justine!
> > > >
> > > > Ismael
> > > >
> > > > On Fri, Apr 14, 2023 at 9:53 AM Justine Olshan
> > > > <jo...@confluent.io.invalid>
> > > > wrote:
> > > >
> > > > > Hey Ismael -- thanks for bringing this up.
> > > > > I've filed https://issues.apache.org/jira/browse/KAFKA-14904 and
> am
> > > > > working
> > > > > on it now.
> > > > >
> > > > > I hope the other tests get fixed soon.
> > > > >
> > > > > On Fri, Apr 14, 2023 at 6:47 AM Ismael Juma <is...@juma.me.uk>
> > wrote:
> > > > >
> > > > > > Hi team,
> > > > > >
> > > > > > It looks like there are a lot of test failures in the master
> > branch.
> > > I
> > > > > > don't know which commits introduced them, but can you please
> check
> > if
> > > > > > commit(s) you merged or contributed are the reason and fix it
> asap?
> > > If
> > > > > it's
> > > > > > easy to fix the tests, let's do that - otherwise we should revert
> > the
> > > > > > faulty commit. And let's please be more careful going forward
> when
> > it
> > > > > comes
> > > > > > to the PRs we merge.
> > > > > >
> > > > > > An example from one of the builds, but there are many like this:
> > > > > >
> > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > kafka.api.TransactionsBounceTest.testWithGroupMetadata()
> > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > > kafka.api.TransactionsBounceTest.testWithGroupId()
> > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.admin.TopicCommandIntegrationTest.testDescribeUnderMinIsrPartitionsMixed(String).quorum=zk
> > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAclOnPrefixedResource(String).quorum=kraft
> > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testNoAclFoundOverride(String).quorum=kraft
> > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testGetAclsPrincipal(String).quorum=kraft
> > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testSyncTopicConfigs()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testSyncTopicConfigs()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTransactionsTest.testReplicateSourceDefault()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testConnectorReconfiguration
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsetsCustomOffsetsTopic
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > kafka.api.TransactionsTest.testFailureToFenceEpoch(String).quorum=kraft
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAllAclOnWildcardResource(String).quorum=kraft
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > > kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.server.metadata.BrokerMetadataPublisherTest.testReloadUpdatedFilesWithoutConfigChange()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > kafka.zk.ZkMigrationIntegrationTest.[1]
> > > > > > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0,
> > > > Security=PLAINTEXT
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.streams.integration.AdjustStreamThreadCountTest.shouldAddAndRemoveThreadsMultipleTimes()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.trogdor.coordinator.CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testRestartReplication()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForPrincipalPolicy
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExampleConnectIntegrationTest.testSinkConnector
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> integration.kafka.server.FetchFromFollowerIntegrationTest.testRackAwareRangeAssignor()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSuperUserHasAccess(String).quorum=kraft
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > kafka.network.SocketServerTest.testConnectionRateLimit()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > > kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > kafka.zk.ZkMigrationIntegrationTest.[1]
> > > > > > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0,
> > > > Security=PLAINTEXT
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.controller.QuorumControllerTest.testDelayedConfigurationOperations()
> > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > Build / JDK 17 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > Build / JDK 8 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > Build / JDK 11 and Scala 2.13 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > > Build / JDK 11 and Scala 2.12 /
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://ci-builds.apache.org/job/Kafka/job/kafka/job/trunk/1764/#showFailuresLink
> > > > > >
> > > > > > Ismael
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Test failures

Posted by Greg Harris <gr...@aiven.io.INVALID>.
Hey all,

We just landed a fix for https://issues.apache.org/jira/browse/KAFKA-14905
which was causing all of
those MirrorConnectorsWithCustomForwardingAdminIntegrationTest failures,
and we will monitor the build for any re-occurrances.
Unfortunately we discovered another test flake that was introduced recently
but that should have a straightforward resolution:
https://issues.apache.org/jira/browse/KAFKA-14929
Thanks Ismael for merging a fix for
https://issues.apache.org/jira/browse/KAFKA-8115 but it appears that there
is still more investigation needed there, as the test is still failing
occasionally.

Thanks,
Greg

On Fri, Apr 14, 2023 at 12:18 PM Ismael Juma <is...@juma.me.uk> wrote:

> Thanks Greg! I really appreciate the help.
>
> Ismael
>
> On Fri, Apr 14, 2023 at 12:08 PM Greg Harris <greg.harris@aiven.io.invalid
> >
> wrote:
>
> > Hey Ismael,
> >
> > We're working to stabilize the Connect/MM2 tests with the following
> issues:
> >
> > * https://issues.apache.org/jira/browse/KAFKA-14905 to address
> > MirrorConectorsWithCustomForwardingAdminIntegrationTest with tentative
> open
> > PR
> > * https://issues.apache.org/jira/browse/KAFKA-14901 to address
> > ExactlyOnceSourceIntegrationTest caused by an (apparent) kafka bug
> >
> > Looking at the other failures in Connect/MM2 for that build in
> particular,
> > it appears that most of them include Embedded Kafka not coming
> up/shutting
> > down cleanly:
> > * MirrorConnectorsIntegrationBaseTest
> > * MirrorConnectorsIntegrationExactlyOnceTest
> > * MirrorConnectorsIntegrationSSLTest
> > * ConnectorClientPolicyIntegrationTest
> > * ConnectorTopicsIntegrationTest
> > * ExactlyOnceSourceIntegrationTest
> > * OffsetsApiIntegrationTest
> > I'll start investigating these failures to learn more.
> >
> > I also have a few older flaky test improvements that have not been
> reviewed
> > or merged yet:
> > * https://issues.apache.org/jira/browse/KAFKA-8115 to
> > address CoordinatorTest (reappeared in the linked build)
> > * https://issues.apache.org/jira/browse/KAFKA-14345 to address
> > (Dynamic)ConnectionQuotaTest
> >
> > It also appears that the flakey AuthorizerTest should be addressed by
> > https://github.com/apache/kafka/pull/13543 which is on trunk now but
> > wasn't
> > at the time of the above run.
> >
> > Thanks,
> > Greg
> >
> > On Fri, Apr 14, 2023 at 10:25 AM Ismael Juma <is...@juma.me.uk> wrote:
> >
> > > Thanks Justine!
> > >
> > > Ismael
> > >
> > > On Fri, Apr 14, 2023 at 9:53 AM Justine Olshan
> > > <jo...@confluent.io.invalid>
> > > wrote:
> > >
> > > > Hey Ismael -- thanks for bringing this up.
> > > > I've filed https://issues.apache.org/jira/browse/KAFKA-14904 and am
> > > > working
> > > > on it now.
> > > >
> > > > I hope the other tests get fixed soon.
> > > >
> > > > On Fri, Apr 14, 2023 at 6:47 AM Ismael Juma <is...@juma.me.uk>
> wrote:
> > > >
> > > > > Hi team,
> > > > >
> > > > > It looks like there are a lot of test failures in the master
> branch.
> > I
> > > > > don't know which commits introduced them, but can you please check
> if
> > > > > commit(s) you merged or contributed are the reason and fix it asap?
> > If
> > > > it's
> > > > > easy to fix the tests, let's do that - otherwise we should revert
> the
> > > > > faulty commit. And let's please be more careful going forward when
> it
> > > > comes
> > > > > to the PRs we merge.
> > > > >
> > > > > An example from one of the builds, but there are many like this:
> > > > >
> > > > > Build / JDK 17 and Scala 2.13 /
> > > > > kafka.api.TransactionsBounceTest.testWithGroupMetadata()
> > > > > Build / JDK 17 and Scala 2.13 /
> > > > > kafka.api.TransactionsBounceTest.testWithGroupId()
> > > > > Build / JDK 17 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > Build / JDK 17 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > Build / JDK 8 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.admin.TopicCommandIntegrationTest.testDescribeUnderMinIsrPartitionsMixed(String).quorum=zk
> > > > > Build / JDK 8 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > Build / JDK 8 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAclOnPrefixedResource(String).quorum=kraft
> > > > > Build / JDK 8 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testNoAclFoundOverride(String).quorum=kraft
> > > > > Build / JDK 8 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > Build / JDK 8 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testGetAclsPrincipal(String).quorum=kraft
> > > > > Build / JDK 8 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > > Build / JDK 8 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testSyncTopicConfigs()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testSyncTopicConfigs()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTransactionsTest.testReplicateSourceDefault()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testConnectorReconfiguration
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsetsCustomOffsetsTopic
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > kafka.api.TransactionsTest.testFailureToFenceEpoch(String).quorum=kraft
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAllAclOnWildcardResource(String).quorum=kraft
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > > kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.server.metadata.BrokerMetadataPublisherTest.testReloadUpdatedFilesWithoutConfigChange()
> > > > > Build / JDK 11 and Scala 2.13 /
> > kafka.zk.ZkMigrationIntegrationTest.[1]
> > > > > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0,
> > > Security=PLAINTEXT
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.streams.integration.AdjustStreamThreadCountTest.shouldAddAndRemoveThreadsMultipleTimes()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.trogdor.coordinator.CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testRestartReplication()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForPrincipalPolicy
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExampleConnectIntegrationTest.testSinkConnector
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> integration.kafka.server.FetchFromFollowerIntegrationTest.testRackAwareRangeAssignor()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSuperUserHasAccess(String).quorum=kraft
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > > kafka.network.SocketServerTest.testConnectionRateLimit()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > > kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > > > > Build / JDK 11 and Scala 2.12 /
> > kafka.zk.ZkMigrationIntegrationTest.[1]
> > > > > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0,
> > > Security=PLAINTEXT
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.controller.QuorumControllerTest.testDelayedConfigurationOperations()
> > > > > Build / JDK 17 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > Build / JDK 17 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > Build / JDK 17 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > Build / JDK 17 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > Build / JDK 17 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > Build / JDK 17 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > Build / JDK 8 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > Build / JDK 8 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > Build / JDK 8 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > Build / JDK 8 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > Build / JDK 8 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > Build / JDK 8 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > Build / JDK 11 and Scala 2.13 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > > Build / JDK 11 and Scala 2.12 /
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > >
> > > > >
> > > > >
> > > >
> > >
> >
> https://ci-builds.apache.org/job/Kafka/job/kafka/job/trunk/1764/#showFailuresLink
> > > > >
> > > > > Ismael
> > > > >
> > > >
> > >
> >
>

Re: Test failures

Posted by Ismael Juma <is...@juma.me.uk>.
Thanks Greg! I really appreciate the help.

Ismael

On Fri, Apr 14, 2023 at 12:08 PM Greg Harris <gr...@aiven.io.invalid>
wrote:

> Hey Ismael,
>
> We're working to stabilize the Connect/MM2 tests with the following issues:
>
> * https://issues.apache.org/jira/browse/KAFKA-14905 to address
> MirrorConectorsWithCustomForwardingAdminIntegrationTest with tentative open
> PR
> * https://issues.apache.org/jira/browse/KAFKA-14901 to address
> ExactlyOnceSourceIntegrationTest caused by an (apparent) kafka bug
>
> Looking at the other failures in Connect/MM2 for that build in particular,
> it appears that most of them include Embedded Kafka not coming up/shutting
> down cleanly:
> * MirrorConnectorsIntegrationBaseTest
> * MirrorConnectorsIntegrationExactlyOnceTest
> * MirrorConnectorsIntegrationSSLTest
> * ConnectorClientPolicyIntegrationTest
> * ConnectorTopicsIntegrationTest
> * ExactlyOnceSourceIntegrationTest
> * OffsetsApiIntegrationTest
> I'll start investigating these failures to learn more.
>
> I also have a few older flaky test improvements that have not been reviewed
> or merged yet:
> * https://issues.apache.org/jira/browse/KAFKA-8115 to
> address CoordinatorTest (reappeared in the linked build)
> * https://issues.apache.org/jira/browse/KAFKA-14345 to address
> (Dynamic)ConnectionQuotaTest
>
> It also appears that the flakey AuthorizerTest should be addressed by
> https://github.com/apache/kafka/pull/13543 which is on trunk now but
> wasn't
> at the time of the above run.
>
> Thanks,
> Greg
>
> On Fri, Apr 14, 2023 at 10:25 AM Ismael Juma <is...@juma.me.uk> wrote:
>
> > Thanks Justine!
> >
> > Ismael
> >
> > On Fri, Apr 14, 2023 at 9:53 AM Justine Olshan
> > <jo...@confluent.io.invalid>
> > wrote:
> >
> > > Hey Ismael -- thanks for bringing this up.
> > > I've filed https://issues.apache.org/jira/browse/KAFKA-14904 and am
> > > working
> > > on it now.
> > >
> > > I hope the other tests get fixed soon.
> > >
> > > On Fri, Apr 14, 2023 at 6:47 AM Ismael Juma <is...@juma.me.uk> wrote:
> > >
> > > > Hi team,
> > > >
> > > > It looks like there are a lot of test failures in the master branch.
> I
> > > > don't know which commits introduced them, but can you please check if
> > > > commit(s) you merged or contributed are the reason and fix it asap?
> If
> > > it's
> > > > easy to fix the tests, let's do that - otherwise we should revert the
> > > > faulty commit. And let's please be more careful going forward when it
> > > comes
> > > > to the PRs we merge.
> > > >
> > > > An example from one of the builds, but there are many like this:
> > > >
> > > > Build / JDK 17 and Scala 2.13 /
> > > > kafka.api.TransactionsBounceTest.testWithGroupMetadata()
> > > > Build / JDK 17 and Scala 2.13 /
> > > > kafka.api.TransactionsBounceTest.testWithGroupId()
> > > > Build / JDK 17 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > Build / JDK 17 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > Build / JDK 8 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> kafka.admin.TopicCommandIntegrationTest.testDescribeUnderMinIsrPartitionsMixed(String).quorum=zk
> > > > Build / JDK 8 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > Build / JDK 8 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAclOnPrefixedResource(String).quorum=kraft
> > > > Build / JDK 8 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testNoAclFoundOverride(String).quorum=kraft
> > > > Build / JDK 8 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > Build / JDK 8 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testGetAclsPrincipal(String).quorum=kraft
> > > > Build / JDK 8 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > > Build / JDK 8 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testSyncTopicConfigs()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testSyncTopicConfigs()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTransactionsTest.testReplicateSourceDefault()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testConnectorReconfiguration
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsetsCustomOffsetsTopic
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> kafka.api.TransactionsTest.testFailureToFenceEpoch(String).quorum=kraft
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAllAclOnWildcardResource(String).quorum=kraft
> > > > Build / JDK 11 and Scala 2.13 /
> > > > kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> kafka.server.metadata.BrokerMetadataPublisherTest.testReloadUpdatedFilesWithoutConfigChange()
> > > > Build / JDK 11 and Scala 2.13 /
> kafka.zk.ZkMigrationIntegrationTest.[1]
> > > > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0,
> > Security=PLAINTEXT
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.streams.integration.AdjustStreamThreadCountTest.shouldAddAndRemoveThreadsMultipleTimes()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.trogdor.coordinator.CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testRestartReplication()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForPrincipalPolicy
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.ExampleConnectIntegrationTest.testSinkConnector
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> integration.kafka.server.FetchFromFollowerIntegrationTest.testRackAwareRangeAssignor()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSuperUserHasAccess(String).quorum=kraft
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > > > Build / JDK 11 and Scala 2.12 /
> > > > kafka.network.SocketServerTest.testConnectionRateLimit()
> > > > Build / JDK 11 and Scala 2.12 /
> > > > kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > > > Build / JDK 11 and Scala 2.12 /
> kafka.zk.ZkMigrationIntegrationTest.[1]
> > > > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0,
> > Security=PLAINTEXT
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.controller.QuorumControllerTest.testDelayedConfigurationOperations()
> > > > Build / JDK 17 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > Build / JDK 17 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > Build / JDK 17 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > Build / JDK 17 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > Build / JDK 17 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > Build / JDK 17 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > Build / JDK 8 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > Build / JDK 8 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > Build / JDK 8 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > Build / JDK 8 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > Build / JDK 8 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > Build / JDK 8 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > Build / JDK 11 and Scala 2.13 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > > Build / JDK 11 and Scala 2.12 /
> > > >
> > > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > >
> > > >
> > > >
> > >
> >
> https://ci-builds.apache.org/job/Kafka/job/kafka/job/trunk/1764/#showFailuresLink
> > > >
> > > > Ismael
> > > >
> > >
> >
>

Re: Test failures

Posted by Greg Harris <gr...@aiven.io.INVALID>.
Hey Ismael,

We're working to stabilize the Connect/MM2 tests with the following issues:

* https://issues.apache.org/jira/browse/KAFKA-14905 to address
MirrorConectorsWithCustomForwardingAdminIntegrationTest with tentative open
PR
* https://issues.apache.org/jira/browse/KAFKA-14901 to address
ExactlyOnceSourceIntegrationTest caused by an (apparent) kafka bug

Looking at the other failures in Connect/MM2 for that build in particular,
it appears that most of them include Embedded Kafka not coming up/shutting
down cleanly:
* MirrorConnectorsIntegrationBaseTest
* MirrorConnectorsIntegrationExactlyOnceTest
* MirrorConnectorsIntegrationSSLTest
* ConnectorClientPolicyIntegrationTest
* ConnectorTopicsIntegrationTest
* ExactlyOnceSourceIntegrationTest
* OffsetsApiIntegrationTest
I'll start investigating these failures to learn more.

I also have a few older flaky test improvements that have not been reviewed
or merged yet:
* https://issues.apache.org/jira/browse/KAFKA-8115 to
address CoordinatorTest (reappeared in the linked build)
* https://issues.apache.org/jira/browse/KAFKA-14345 to address
(Dynamic)ConnectionQuotaTest

It also appears that the flakey AuthorizerTest should be addressed by
https://github.com/apache/kafka/pull/13543 which is on trunk now but wasn't
at the time of the above run.

Thanks,
Greg

On Fri, Apr 14, 2023 at 10:25 AM Ismael Juma <is...@juma.me.uk> wrote:

> Thanks Justine!
>
> Ismael
>
> On Fri, Apr 14, 2023 at 9:53 AM Justine Olshan
> <jo...@confluent.io.invalid>
> wrote:
>
> > Hey Ismael -- thanks for bringing this up.
> > I've filed https://issues.apache.org/jira/browse/KAFKA-14904 and am
> > working
> > on it now.
> >
> > I hope the other tests get fixed soon.
> >
> > On Fri, Apr 14, 2023 at 6:47 AM Ismael Juma <is...@juma.me.uk> wrote:
> >
> > > Hi team,
> > >
> > > It looks like there are a lot of test failures in the master branch. I
> > > don't know which commits introduced them, but can you please check if
> > > commit(s) you merged or contributed are the reason and fix it asap? If
> > it's
> > > easy to fix the tests, let's do that - otherwise we should revert the
> > > faulty commit. And let's please be more careful going forward when it
> > comes
> > > to the PRs we merge.
> > >
> > > An example from one of the builds, but there are many like this:
> > >
> > > Build / JDK 17 and Scala 2.13 /
> > > kafka.api.TransactionsBounceTest.testWithGroupMetadata()
> > > Build / JDK 17 and Scala 2.13 /
> > > kafka.api.TransactionsBounceTest.testWithGroupId()
> > > Build / JDK 17 and Scala 2.13 /
> > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > Build / JDK 17 and Scala 2.13 /
> > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > Build / JDK 8 and Scala 2.12 /
> > >
> > >
> >
> kafka.admin.TopicCommandIntegrationTest.testDescribeUnderMinIsrPartitionsMixed(String).quorum=zk
> > > Build / JDK 8 and Scala 2.12 /
> > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > Build / JDK 8 and Scala 2.12 /
> > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAclOnPrefixedResource(String).quorum=kraft
> > > Build / JDK 8 and Scala 2.12 /
> > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testNoAclFoundOverride(String).quorum=kraft
> > > Build / JDK 8 and Scala 2.12 /
> > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > Build / JDK 8 and Scala 2.12 /
> > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testGetAclsPrincipal(String).quorum=kraft
> > > Build / JDK 8 and Scala 2.12 /
> > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > > Build / JDK 8 and Scala 2.12 /
> > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testSyncTopicConfigs()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testSyncTopicConfigs()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTransactionsTest.testReplicateSourceDefault()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testConnectorReconfiguration
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsetsCustomOffsetsTopic
> > > Build / JDK 11 and Scala 2.13 /
> > > kafka.api.TransactionsTest.testFailureToFenceEpoch(String).quorum=kraft
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAllAclOnWildcardResource(String).quorum=kraft
> > > Build / JDK 11 and Scala 2.13 /
> > > kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> kafka.server.metadata.BrokerMetadataPublisherTest.testReloadUpdatedFilesWithoutConfigChange()
> > > Build / JDK 11 and Scala 2.13 / kafka.zk.ZkMigrationIntegrationTest.[1]
> > > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0,
> Security=PLAINTEXT
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.streams.integration.AdjustStreamThreadCountTest.shouldAddAndRemoveThreadsMultipleTimes()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.trogdor.coordinator.CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testRestartReplication()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForPrincipalPolicy
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.ExampleConnectIntegrationTest.testSinkConnector
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> integration.kafka.server.FetchFromFollowerIntegrationTest.testRackAwareRangeAssignor()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSuperUserHasAccess(String).quorum=kraft
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > > Build / JDK 11 and Scala 2.12 /
> > > kafka.network.SocketServerTest.testConnectionRateLimit()
> > > Build / JDK 11 and Scala 2.12 /
> > > kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > > Build / JDK 11 and Scala 2.12 / kafka.zk.ZkMigrationIntegrationTest.[1]
> > > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0,
> Security=PLAINTEXT
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.controller.QuorumControllerTest.testDelayedConfigurationOperations()
> > > Build / JDK 17 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > Build / JDK 17 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > Build / JDK 17 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > Build / JDK 17 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > Build / JDK 17 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > Build / JDK 17 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > Build / JDK 8 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > Build / JDK 8 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > Build / JDK 8 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > Build / JDK 8 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > Build / JDK 8 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > Build / JDK 8 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > Build / JDK 11 and Scala 2.13 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > > Build / JDK 11 and Scala 2.12 /
> > >
> > >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > >
> > >
> > >
> >
> https://ci-builds.apache.org/job/Kafka/job/kafka/job/trunk/1764/#showFailuresLink
> > >
> > > Ismael
> > >
> >
>

Re: Test failures

Posted by Ismael Juma <is...@juma.me.uk>.
Thanks Justine!

Ismael

On Fri, Apr 14, 2023 at 9:53 AM Justine Olshan <jo...@confluent.io.invalid>
wrote:

> Hey Ismael -- thanks for bringing this up.
> I've filed https://issues.apache.org/jira/browse/KAFKA-14904 and am
> working
> on it now.
>
> I hope the other tests get fixed soon.
>
> On Fri, Apr 14, 2023 at 6:47 AM Ismael Juma <is...@juma.me.uk> wrote:
>
> > Hi team,
> >
> > It looks like there are a lot of test failures in the master branch. I
> > don't know which commits introduced them, but can you please check if
> > commit(s) you merged or contributed are the reason and fix it asap? If
> it's
> > easy to fix the tests, let's do that - otherwise we should revert the
> > faulty commit. And let's please be more careful going forward when it
> comes
> > to the PRs we merge.
> >
> > An example from one of the builds, but there are many like this:
> >
> > Build / JDK 17 and Scala 2.13 /
> > kafka.api.TransactionsBounceTest.testWithGroupMetadata()
> > Build / JDK 17 and Scala 2.13 /
> > kafka.api.TransactionsBounceTest.testWithGroupId()
> > Build / JDK 17 and Scala 2.13 /
> >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > Build / JDK 17 and Scala 2.13 /
> >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > Build / JDK 8 and Scala 2.12 /
> >
> >
> kafka.admin.TopicCommandIntegrationTest.testDescribeUnderMinIsrPartitionsMixed(String).quorum=zk
> > Build / JDK 8 and Scala 2.12 /
> >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > Build / JDK 8 and Scala 2.12 /
> >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAclOnPrefixedResource(String).quorum=kraft
> > Build / JDK 8 and Scala 2.12 /
> >
> >
> kafka.security.authorizer.AuthorizerTest.testNoAclFoundOverride(String).quorum=kraft
> > Build / JDK 8 and Scala 2.12 /
> >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > Build / JDK 8 and Scala 2.12 /
> >
> >
> kafka.security.authorizer.AuthorizerTest.testGetAclsPrincipal(String).quorum=kraft
> > Build / JDK 8 and Scala 2.12 /
> >
> >
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> > Build / JDK 8 and Scala 2.12 /
> >
> >
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testSyncTopicConfigs()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testSyncTopicConfigs()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTransactionsTest.testReplicateSourceDefault()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testConnectorReconfiguration
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsetsCustomOffsetsTopic
> > Build / JDK 11 and Scala 2.13 /
> > kafka.api.TransactionsTest.testFailureToFenceEpoch(String).quorum=kraft
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> kafka.security.authorizer.AuthorizerTest.testDeleteAllAclOnWildcardResource(String).quorum=kraft
> > Build / JDK 11 and Scala 2.13 /
> > kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> kafka.server.metadata.BrokerMetadataPublisherTest.testReloadUpdatedFilesWithoutConfigChange()
> > Build / JDK 11 and Scala 2.13 / kafka.zk.ZkMigrationIntegrationTest.[1]
> > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0, Security=PLAINTEXT
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.streams.integration.AdjustStreamThreadCountTest.shouldAddAndRemoveThreadsMultipleTimes()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.trogdor.coordinator.CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testRestartReplication()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithFrequentOffsetSyncs()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForPrincipalPolicy
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.integration.ExampleConnectIntegrationTest.testSinkConnector
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> integration.kafka.server.FetchFromFollowerIntegrationTest.testRackAwareRangeAssignor()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> kafka.security.authorizer.AuthorizerTest.testSuperUserHasAccess(String).quorum=kraft
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> > Build / JDK 11 and Scala 2.12 /
> > kafka.network.SocketServerTest.testConnectionRateLimit()
> > Build / JDK 11 and Scala 2.12 /
> > kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> > Build / JDK 11 and Scala 2.12 / kafka.zk.ZkMigrationIntegrationTest.[1]
> > Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0, Security=PLAINTEXT
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.controller.QuorumControllerTest.testDelayedConfigurationOperations()
> > Build / JDK 17 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > Build / JDK 17 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > Build / JDK 17 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > Build / JDK 17 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > Build / JDK 17 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > Build / JDK 17 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > Build / JDK 8 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > Build / JDK 8 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > Build / JDK 8 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > Build / JDK 8 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > Build / JDK 8 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > Build / JDK 8 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > Build / JDK 11 and Scala 2.13 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> > Build / JDK 11 and Scala 2.12 /
> >
> >
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> >
> >
> >
> https://ci-builds.apache.org/job/Kafka/job/kafka/job/trunk/1764/#showFailuresLink
> >
> > Ismael
> >
>

Re: Test failures

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Hey Ismael -- thanks for bringing this up.
I've filed https://issues.apache.org/jira/browse/KAFKA-14904 and am working
on it now.

I hope the other tests get fixed soon.

On Fri, Apr 14, 2023 at 6:47 AM Ismael Juma <is...@juma.me.uk> wrote:

> Hi team,
>
> It looks like there are a lot of test failures in the master branch. I
> don't know which commits introduced them, but can you please check if
> commit(s) you merged or contributed are the reason and fix it asap? If it's
> easy to fix the tests, let's do that - otherwise we should revert the
> faulty commit. And let's please be more careful going forward when it comes
> to the PRs we merge.
>
> An example from one of the builds, but there are many like this:
>
> Build / JDK 17 and Scala 2.13 /
> kafka.api.TransactionsBounceTest.testWithGroupMetadata()
> Build / JDK 17 and Scala 2.13 /
> kafka.api.TransactionsBounceTest.testWithGroupId()
> Build / JDK 17 and Scala 2.13 /
>
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> Build / JDK 17 and Scala 2.13 /
>
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> Build / JDK 8 and Scala 2.12 /
>
> kafka.admin.TopicCommandIntegrationTest.testDescribeUnderMinIsrPartitionsMixed(String).quorum=zk
> Build / JDK 8 and Scala 2.12 /
>
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> Build / JDK 8 and Scala 2.12 /
>
> kafka.security.authorizer.AuthorizerTest.testDeleteAclOnPrefixedResource(String).quorum=kraft
> Build / JDK 8 and Scala 2.12 /
>
> kafka.security.authorizer.AuthorizerTest.testNoAclFoundOverride(String).quorum=kraft
> Build / JDK 8 and Scala 2.12 /
>
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> Build / JDK 8 and Scala 2.12 /
>
> kafka.security.authorizer.AuthorizerTest.testGetAclsPrincipal(String).quorum=kraft
> Build / JDK 8 and Scala 2.12 /
>
> kafka.security.authorizer.AuthorizerTest.testAllowAllAccess(String).quorum=kraft
> Build / JDK 8 and Scala 2.12 /
>
> kafka.security.authorizer.AuthorizerTest.testAuthorizeThrowsOnNonLiteralResource(String).quorum=kraft
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testSyncTopicConfigs()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testSyncTopicConfigs()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTransactionsTest.testReplicateSourceDefault()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testConnectorReconfiguration
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsetsCustomOffsetsTopic
> Build / JDK 11 and Scala 2.13 /
> kafka.api.TransactionsTest.testFailureToFenceEpoch(String).quorum=kraft
> Build / JDK 11 and Scala 2.13 /
>
> kafka.security.authorizer.AuthorizerTest.testDeleteAllAclOnWildcardResource(String).quorum=kraft
> Build / JDK 11 and Scala 2.13 /
> kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> Build / JDK 11 and Scala 2.13 /
>
> kafka.server.metadata.BrokerMetadataPublisherTest.testReloadUpdatedFilesWithoutConfigChange()
> Build / JDK 11 and Scala 2.13 / kafka.zk.ZkMigrationIntegrationTest.[1]
> Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0, Security=PLAINTEXT
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.streams.integration.AdjustStreamThreadCountTest.shouldAddAndRemoveThreadsMultipleTimes()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.trogdor.coordinator.CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testRestartReplication()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest.testOneWayReplicationWithFrequentOffsetSyncs()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplication()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateSourceDefault()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOffsetSyncsTopicsOnTarget()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testReplicateTargetDefault()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testNoCheckpointsIfNoRecordsAreMirrored()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithAutoOffsetSync()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testSyncTopicConfigs()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testRestartReplication()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithFrequentOffsetSyncs()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplication()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateSourceDefault()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOffsetSyncsTopicsOnTarget()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicateTargetDefault()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testNoCheckpointsIfNoRecordsAreMirrored()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForPrincipalPolicy
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithOverridesForNonePolicy
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithAllowedOverridesForDefaultPolicy
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.integration.ConnectorClientPolicyIntegrationTest.testCreateWithNoAllowedOverridesForNonePolicy
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testGetActiveTopics
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingResetIsDisabled
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.integration.ConnectorTopicsIntegrationTest.testTopicTrackingIsDisabled
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.integration.ExampleConnectIntegrationTest.testSinkConnector
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsets
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetNonExistentConnectorOffsets
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsOverriddenConsumerGroupId
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSourceConnectorOffsets
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted
> Build / JDK 11 and Scala 2.12 /
>
> integration.kafka.server.FetchFromFollowerIntegrationTest.testRackAwareRangeAssignor()
> Build / JDK 11 and Scala 2.12 /
>
> kafka.security.authorizer.AuthorizerTest.testSuperUserHasAccess(String).quorum=kraft
> Build / JDK 11 and Scala 2.12 /
>
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> Build / JDK 11 and Scala 2.12 /
>
> kafka.security.authorizer.AuthorizerTest.testSingleCharacterResourceAcls(String).quorum=kraft
> Build / JDK 11 and Scala 2.12 /
> kafka.network.SocketServerTest.testConnectionRateLimit()
> Build / JDK 11 and Scala 2.12 /
> kafka.server.RaftClusterSnapshotTest.testSnapshotsGenerated()
> Build / JDK 11 and Scala 2.12 / kafka.zk.ZkMigrationIntegrationTest.[1]
> Type=ZK, Name=testDualWrite, MetadataVersion=3.4-IV0, Security=PLAINTEXT
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.controller.QuorumControllerTest.testDelayedConfigurationOperations()
> Build / JDK 17 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> Build / JDK 17 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> Build / JDK 17 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> Build / JDK 17 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> Build / JDK 17 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> Build / JDK 17 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> Build / JDK 8 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> Build / JDK 8 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> Build / JDK 8 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> Build / JDK 8 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> Build / JDK 8 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> Build / JDK 8 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> Build / JDK 11 and Scala 2.13 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testCreatePartitionsUseProvidedForwardingAdmin()
> Build / JDK 11 and Scala 2.12 /
>
> org.apache.kafka.connect.mirror.integration.MirrorConnectorsWithCustomForwardingAdminIntegrationTest.testSyncTopicConfigUseProvidedForwardingAdmin()
>
>
> https://ci-builds.apache.org/job/Kafka/job/kafka/job/trunk/1764/#showFailuresLink
>
> Ismael
>