You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Randall Hauch (JIRA)" <ji...@apache.org> on 2017/05/01 22:48:04 UTC

[jira] [Comment Edited] (KAFKA-4667) Connect should create internal topics

    [ https://issues.apache.org/jira/browse/KAFKA-4667?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15991697#comment-15991697 ] 

Randall Hauch edited comment on KAFKA-4667 at 5/1/17 10:47 PM:
---------------------------------------------------------------

The AdminClient pull request is being added as part of [PR 2472|https://github.com/apache/kafka/pull/2472], though it looks like it's very close to being merged.

As mentioned above, the most strongly consistent and durable topics require per-topic configuration with:

* replication factor of {{min(3, nodeCount)}}
* number of partitions to 1
* {{min.insync.replicas=min(round(replicationFactor/2),nodeCount)}}
* {{unclean.leader.election.enable=false}}
* {{cleanup.policy=compact}}

Note that since we can't specify a replication factor or {{min.insync.replicas}} greater than the number of nodes in a cluster, we need to first determine the cluster size, {{nodeCount}}, and calculate the desired replication factor and {{min.insync.replicas}} value.

Kafka Connect currently creates 3 internal topics for each connector: one for the connector configuration(s), one for the offsets, and one for status. Should all 3 of these internal topics (per connector) use these same settings, or might the status topic not need to be so conservatively defined? Secondly, should a connector configuration be allowed to override these or any of the other [topic configuration properties| https://kafka.apache.org/documentation/#topic-config]?

Finally, the AdminClient in [PR 2472|https://github.com/apache/kafka/pull/2472] will allow us to get the standard metadata for each topic partition, meaning we can get the partition count, ISRs, and replica nodes. So we could output a warning if the replication factor is smaller than the preferred value as calculated above, but can we do much else? For example, can we even read the topic's cleanup policy, whether unclean leader election is enabled, or other topic properties, and if so it doesn't appear we can alter a topic configuration.


was (Author: rhauch):
The AdminClient pull request is being added as part of [PR 2472|https://github.com/apache/kafka/pull/2472], though it looks like it's very close to being merged.

As mentioned above, the most strongly consistent and durable topics require per-topic configuration with:

* replication factor of {{min(3, nodeCount)}}
* number of partitions to 1
* {{min.insync.replicas=min(round(replicationFactor/2),nodeCount)}}
* {{unclean.leader.election.enable=false}}
* {{cleanup.policy=compact}}

Note that since we can't specify a replication factor or {{min.insync.replicas}} greater than the number of nodes in a cluster, we need to first determine the cluster size, {{nodeCount}}.

Kafka Connect currently creates 3 internal topics for each connector: one for the connector configuration(s), one for the offsets, and one for status. Should all 3 of these internal topics (per connector) use these same settings, or might the status topic not need to be so conservatively defined? Secondly, should a connector configuration be allowed to override these or any of the other [topic configuration properties| https://kafka.apache.org/documentation/#topic-config]?

Finally, the AdminClient in [PR 2472|https://github.com/apache/kafka/pull/2472] will allow us to get the standard metadata for each topic partition, meaning we can get the partition count, ISRs, and replica nodes. So we could output a warning if the replication factor is smaller than the preferred value as calculated above, but can we do much else? For example, can we even read the topic's cleanup policy, whether unclean leader election is enabled, or other topic properties, and if so it doesn't appear we can alter a topic configuration.

> Connect should create internal topics
> -------------------------------------
>
>                 Key: KAFKA-4667
>                 URL: https://issues.apache.org/jira/browse/KAFKA-4667
>             Project: Kafka
>          Issue Type: Bug
>          Components: KafkaConnect
>            Reporter: Emanuele Cesena
>            Priority: Critical
>             Fix For: 0.11.0.0
>
>
> I'm reporting this as an issue but in fact it requires more investigation (which unfortunately I'm not able to perform at this time).
> Repro steps:
> - configure Kafka for consistency, for example:
> default.replication.factor=3
> min.insync.replicas=2
> unclean.leader.election.enable=false
> - run Connect for the first time, which should create its internal topics
> I believe these topics are created with the broker's default, in particular:
> min.insync.replicas=2
> unclean.leader.election.enable=false
> but connect doesn't produce with acks=all, which in turn may cause the cluster to go in a bad state (see, e.g., https://issues.apache.org/jira/browse/KAFKA-4666).
> Solution would be to force availability mode, i.e. force:
> unclean.leader.election.enable=true
> when creating the connect topics, or viceversa detect availability vs consistency mode and turn acks=all if needed.
> I assume the same happens with other kafka-based services such as streams.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)