Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-16 Thread Frank Grimes
Ah, it definitely seems like KIP-710 will address the issue we've been bitten by most.We'll eagerly await the kafka-3.5.0 release and then see if enabling 'dedicated.mode.enable.internal.rest' is possible with Strimzi. Thanks for the help and patience! :-)

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-15 Thread Greg Harris
Frank, > I don't think forcing the API users to introduce the nonce is desirable. I agree. That is why the nonce is a workaround, and not a proper solution. It's something to alleviate the symptoms in the short-term until a bugfix & upgrade can fix it. > Have you had any ideas on how this can be

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-15 Thread Frank Grimes
So we've just hit this issue again just with the MM2 connector and trying to add a new mirrored topic.We're running MirrorMaker 2 in Strimzi. i.e. "connector.class": "org.apache.kafka.connect.mirror.MirrorSourceConnector"We have 6 worker nodes.We changed the config to add a new mirror topic. i.e

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-14 Thread Frank Grimes
I don't think forcing the API users to introduce the nonce is desirable.For us, it would mean reaching out to the Strimzi project to try to get that implemented on their side, which I would imagine would be a proposal which would meet some resistance. Have you had any ideas on how this can be im

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-09 Thread Greg Harris
Frank, The configs are being compared after ConfigProviders have been resolved. This is happening both as a Connector config (by ClusterConfigState::connectorConfig) and as task configs (by ClusterConfigState::taskConfig). This means that two configurations that have different direct contents (the

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-09 Thread Frank Grimes
I'm still having trouble understanding how the configs could match in the code you highlighted when we change connector and/or task config values when no keys are being pruned by the connector implementations in question.Would capturing a new generation value within the config itself on every s

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-08 Thread Greg Harris
Frank, > I'm operating on the assumption that the connectors in question get stuck in an inconsistent state > Another thought... if an API exists to list all connectors in such a state, then at least some monitoring/alerting could be put in place, right? There is two different inconsistencies rel

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-08 Thread Frank Grimes
Another thought... if an API exists to list all connectors in such a state, then at least some monitoring/alerting could be put in place, right?

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-08 Thread Frank Grimes
So I've been looking into the codebase to familiarize myself with it.I'm operating on the assumption that the connectors in question get stuck in an inconsistent state which causes them to prune the new task configs from those which are "broadcast" to the workers.I see on KafkaConfigBackingSto

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-06 Thread Greg Harris
Frank, I don't think that the fix needs to necessarily follow the #12450 PR, we can choose to start from scratch now that we know more about the issue. If that PR is useful as a starting point, we can also include it, that is up to you. Greg On Mon, Feb 6, 2023 at 10:21 AM Frank Grimes wrote:

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-06 Thread Frank Grimes
Hi Greg, I actually just found the following comment on this PR for  https://issues.apache.org/jira/browse/KAFKA-13809:  https://github.com/apache/kafka/pull/12450 > we get the same behavior (KAFKA-9228 notwithstanding) by passing the original >properties through to tasks transparently It seems

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-06 Thread Greg Harris
Frank, I think you're right that the KAFKA-9228 ticket doesn't capture every possible reconfiguration that might result in a dropped restart. The ticket calls out the FileStream connectors, which generate their configurations by dropping unknown config values, which is relatively uncommon. This me

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-06 Thread Frank Grimes
Hi Greg, The "long-term inconsistency" we have observed is not with no tasks at all, but instead with all the previously running tasks remaining in a running state but with a previous config. If I'm understanding the original bug report correctly, the scope of the problem was thought to only af

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-03 Thread Greg Harris
Frank, I realized I didn't respond to the title directly, sorry about that. The reason that `ClusterConfigState::inconsistentConnectors` is not used, is that the effect of an inconsistent connector is applied via `ClusterConfigState::tasks`. If a connector is inconsistent, then the tasks method wi

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-03 Thread Greg Harris
Frank, The inconsistentConnectors method is related to an extremely specific inconsistency that can happen when a worker writes some task configurations, and then disconnects without writing a following "commit tasks record" to the config topic. This is a hold-over from the early days of connect f

Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-03 Thread Frank Grimes
Hi, we're investigating an issue where occasionally config changes don't propagate to connectors/tasks. When this occurs, the only way to ensure that the configuration takes effect is to resize the number of tasks back down to 1 and then resize back up to the original number of tasks. In search