You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "gharris1727 (via GitHub)" <gi...@apache.org> on 2023/04/12 19:42:24 UTC

[GitHub] [kafka] gharris1727 opened a new pull request, #13548: MINOR: Fix regression in MM2 task forwarding introduced by KAFKA-14783

gharris1727 opened a new pull request, #13548:
URL: https://github.com/apache/kafka/pull/13548

   The DistributedHerder was computing the forwarded URL for publishing task configs incorrectly leading to 404s in MM2 distributed mode.
   
   This regression appears in #13424 and presently only exists on trunk. This manifests as a return to pre-KIP-710 behavior, and causes the DedicatedMirrorIntegrationTest to fail whenever forwarding happens, making the test flake in >50% of runs.
   
   It appears to be just a typo and not an intended change, and was hidden by the github diff when this function was split into two parts.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [kafka] ijuma commented on pull request #13548: MINOR: Fix regression in MM2 task forwarding introduced by KAFKA-14783

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on PR #13548:
URL: https://github.com/apache/kafka/pull/13548#issuecomment-1508828513

   @gharris1727 I think what you did was fine. :) My concern is more on how we decided that it was ok to merge. I looked a bit more and I think the issues potentially started around here for a bunch of the tests:
   
   https://github.com/apache/kafka/pull/13373#issuecomment-1508823431
   
   And it looks like a similar pattern happened there where we merged without a detailed analysis of the failures.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [kafka] ijuma commented on pull request #13548: MINOR: Fix regression in MM2 task forwarding introduced by KAFKA-14783

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on PR #13548:
URL: https://github.com/apache/kafka/pull/13548#issuecomment-1508774105

   But how do we even know that this is helping vs making it worse with so many failures?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [kafka] C0urante merged pull request #13548: MINOR: Fix regression in MM2 task forwarding introduced by KAFKA-14783

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante merged PR #13548:
URL: https://github.com/apache/kafka/pull/13548


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [kafka] gharris1727 commented on pull request #13548: MINOR: Fix regression in MM2 task forwarding introduced by KAFKA-14783

Posted by "gharris1727 (via GitHub)" <gi...@apache.org>.
gharris1727 commented on PR #13548:
URL: https://github.com/apache/kafka/pull/13548#issuecomment-1508814690

   @ijuma I proposed this PR in the form I did because it restored a piece of code that was unintentionally changed. From an end user-perspective, the unintentional change was certainly bad, and reverting that change (in this PR) is certainly good. I'm sorry if the PR description did not make that clear, and I'll try to improve in the future.
   
   I have looked into the other changes and identified their causes to be something distinct from this regression:
   
   * DedicatedMirrorIntegrationTest has been flakey since before this regression was committed, and affects both testSingleNodeCluster and testMultiNodeCluster (as seen in your list of test failures). The regression that this PR addresses only ever caused a failure in the MultiNode case, as this code path is only active in multi node clusters.
   
   * ExactlyOnceSourceIntegrationTest appears to be failing due to an error from the kafka TransactionManager, and surfaces with a different error than this regression would cause.
   * I have not yet completed my investigation on the ForwardingAdmin test failures, but those are consistently failing (not flakey like this regression), started failing on a different PR than the one which introduced this regression, and only affect the forwarding admin tests. Note: 'forwarding admin' is a feature unrelated to 'connect herder request forwarding', despite sharing the word 'forwarding'.
   * AuthorizerTest is failing due to an NPE in a completely unrelated part of the code base.
   * The initializationError failures are due to 3 leaked threads, which would surprise me if it was related to this URL formatting logic.
   
   All in all, I saw an obvious typo in a previous PR, and opened a follow-up to fix it. I did not want to issue a "Fix every Connect test failure" as I think that would be harder to review and less modular if reverts needed to be made in the future.
   How would you approach this differently?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [kafka] ijuma commented on pull request #13548: MINOR: Fix regression in MM2 task forwarding introduced by KAFKA-14783

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on PR #13548:
URL: https://github.com/apache/kafka/pull/13548#issuecomment-1508573002

   The PR build has 95 failures:
   
   > Build / JDK 17 and Scala 2.13 / org.apache.kafka.common.network.SslTransportLayerTest.[2] tlsProtocol=TLSv1.2, useInlinePem=true
   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 17 and Scala 2.13 / kafka.security.authorizer.AuthorizerTest.testAuthorizeWithPrefixedResource(String).quorum=kraft
   Build / JDK 17 and Scala 2.13 / org.apache.kafka.streams.integration.PurgeRepartitionTopicIntegrationTest.shouldRestoreState()
   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 8 and Scala 2.12 / org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testConnectorReconfiguration
   Build / JDK 8 and Scala 2.12 / org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest.testConnectorReconfiguration
   Build / JDK 8 and Scala 2.12 / integration.kafka.server.FetchFromFollowerIntegrationTest.testRackAwareRangeAssignor()
   Build / JDK 8 and Scala 2.12 / kafka.api.ConsumerBounceTest.testConsumptionWithBrokerFailures()
   Build / JDK 8 and Scala 2.12 / kafka.api.ConsumerBounceTest.executionError
   Build / JDK 8 and Scala 2.12 / kafka.api.ConsumerWithLegacyMessageFormatIntegrationTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.api.DelegationTokenEndToEndAuthorizationTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.api.EndToEndClusterIdTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.api.GroupCoordinatorIntegrationTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.api.LogAppendTimeTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.api.PlaintextConsumerTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.api.PlaintextEndToEndAuthorizationTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.api.ProducerFailureHandlingTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.api.ProducerSendWhileDeletionTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.api.SaslClientsWithInvalidCredentialsTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.api.SaslGssapiSslEndToEndAuthorizationTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.api.SaslOAuthBearerSslEndToEndAuthorizationTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.api.SaslPlaintextConsumerTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.api.SaslSslConsumerTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.api.SslConsumerTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.api.SslProducerSendTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.api.TransactionsBounceTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.api.TransactionsTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.api.UserClientIdQuotaTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.api.test.ProducerCompressionTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.integration.MinIsrConfigTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.metrics.MetricsTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.network.DynamicConnectionQuotaTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.security.authorizer.AclAuthorizerWithZkSaslTest.initializationError
   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.testAuthorizeByResourceTypeNoAclFoundOverride(String).quorum=kraft
   Build / JDK 8 and Scala 2.12 / kafka.server.AdvertiseBrokerTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.AlterReplicaLogDirsRequestTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.AlterUserScramCredentialsRequestTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.CreateTopicsRequestTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.DelegationTokenRequestsOnPlainTextTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.DelegationTokenRequestsWithDisableTokenFeatureTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.DeleteTopicsRequestTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.DescribeClusterRequestTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.DescribeLogDirsRequestTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.DescribeUserScramCredentialsRequestNotAuthorizedTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.DynamicBrokerReconfigurationTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.EdgeCaseRequestTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.FetchRequestDownConversionConfigTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.FetchRequestTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.FetchRequestWithLegacyMessageFormatTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.FinalizedFeatureChangeListenerTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.GssapiAuthenticationTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.KafkaMetricReporterClusterIdTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.KafkaMetricReporterExceptionHandlingTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.KafkaMetricsReporterTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.LeaderElectionTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.ListOffsetsRequestWithRemoteStoreTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.LogOffsetTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.MetadataRequestBetweenDifferentIbpTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.MultipleListenersWithDefaultJaasContextTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.OffsetsForLeaderEpochRequestTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.ProduceRequestTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.ReplicaFetchTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.ReplicationQuotasTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.RequestQuotaTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.ServerGenerateBrokerIdTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.ServerStartupTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.StopReplicaRequestTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.TopicIdWithOldInterBrokerProtocolTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.epoch.EpochDrivenReplicationProtocolAcceptanceTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.server.epoch.LeaderEpochIntegrationTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.tools.GetOffsetShellTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.tools.MirrorMakerIntegrationTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.utils.ReplicationUtilsTest.initializationError
   Build / JDK 8 and Scala 2.12 / kafka.zookeeper.ZooKeeperClientTest.initializationError
   Build / JDK 11 and Scala 2.13 / org.apache.kafka.connect.mirror.integration.DedicatedMirrorIntegrationTest.testSingleNodeCluster()
   Build / JDK 11 and Scala 2.13 / org.apache.kafka.connect.mirror.integration.DedicatedMirrorIntegrationTest.testMultiNodeCluster()
   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.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.13 / integration.kafka.server.FetchFromFollowerIntegrationTest.testRackAwareRangeAssignor()
   
   How come we merged this without mentioning why it's ok to do so?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [kafka] gharris1727 commented on pull request #13548: MINOR: Fix regression in MM2 task forwarding introduced by KAFKA-14783

Posted by "gharris1727 (via GitHub)" <gi...@apache.org>.
gharris1727 commented on PR #13548:
URL: https://github.com/apache/kafka/pull/13548#issuecomment-1508726959

   @ijuma The scope of this PR was small, addressing just one issue. We certainly have several flakey and failing tests that need to be addressed during this test stabilization period. Here are the relevant tickets i've found:
   
   * `ExactlyOnceSourceIntegrationTest` failures are covered under https://issues.apache.org/jira/browse/KAFKA-14901
   * `DedicatedMirrorIntegrationTest` failures are covered under https://issues.apache.org/jira/browse/KAFKA-14718
   * `MirrorConnectorsWithCustomForwardingAdminIntegrationTest` failures are covered under https://issues.apache.org/jira/browse/KAFKA-14905
   
   And outside of the connect/mirrormaker area:
   * `AuthorizerTest` failures are covered under https://issues.apache.org/jira/browse/KAFKA-14900
   * the `initializationError` tests all appear to be failing due to a leaked thread, I could not find a ticket for these.
   The other tests appear unrelated as this is a connect-specific change, and could be flakey tests that have not been identified yet.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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