Kafka Streams est la bibliothèque de stream processing proposée par Apache Kafka. Elle permet la transformation à la volé de flux continus. Avec cette bibliothèque il est possible d’atteindre de très faibles latences. Parmi toutes les transformations classiques réalisables (filtres, aggrégations etc) on retrouve les jointures. Il y a un concept essentiel en Kafka Streams qui va permettre ces jointures : le co-partitionnement ! C’est une idée qui est assez bien décrite et documentée, l’objet de ce post est donc de la mettre en avant, très simplement et en image ce concept pour en obtenir quelque chose de concret et visuel.
Co-partitioning, une définition
On dit de deux streams qu’ils sont co-partitionnés si :
- Les clefs des messages ont le même schéma
- Les topics qui les matérialisent ont le même nombre de partitions
- Les producteurs de données utilisent le
DefaultPartitioner
ou un partitioner qui respecte certaines propriétés*
Ces trois points sont décrits comme les conditions nécessaires au co-partitioning (co-partitioning requirements). Plus concrètement, il s’agit de s’assurer que deux messages issus de flux différents, mais avec la même clef, se voient assigner le même numéro de partition. Et effectivement, dans notre définition tout est réuni pour que ce soit le cas !
Rappel : Les messages Kafka sont publiés dans des topics qui sont subdivisés en partitions. La partition de destination est déterminée par une implémentation du Partitioner. Le DefaultPartitioner applique la fonction suivante : partition = hash(key) % numPartitions
.
Si l’on traduit cette formule en français: on applique une fonction de hash à la clé de notre message et le modulo du résultat avec le nombre de partitions nous donne l’assignation de notre message. Suivre cette formule permet d’avoir une manière déterministe d’assigner les partitions de chaque message.
fig 1: détermination des partitions par le default partitioner
Mais dans le fond quel est l’intérêt et comment cela aide nos jointures en temps réel ?
Pour apprécier pleinement le fait que nos flux soient co-partitionnés il faut aussi comprendre comment ils sont consommés. Pour cela faisons un focus sur un deuxième concept présent au niveau des clients Kafka : le Partition Assignment.
Encore plus simple cette fois. Les partitions nous permettent de passer à l’échelle. Au déploiement de plusieurs instances de consumer pour assurer la prise en compte de la charge, elles se répartissent les messages. Chaque message n’est consommé que par une seule instance d’un groupe de consommation. Et pour se faire, nos instances s’organisent grâce aux partitions. De ce fait, pour une application qui doit réaliser une jointure entre deux flux, nous devons nous assurer que les messages censés matcher ne sont pas tirés par des instances différentes.
fig 2: assignation des partitions d’un flux non co-patitionné
L’image précédente montre un mauvais co-partitioning. Les messages du topic click avec une clef bleue sont tirés par l’instance numéro 1, mais le message du topic account (dont ils devraient être rapprochés) se retrouvent sur l’instance 2. Ce que nous voulons, c’est réunir ces messages au sein d’une seule JVM. Ce que nous voulons, c’est qu’ils soient co-partitionnés !
fig 3: assignation des partitions de deux topics co-partitionés
Il nous suffit simplement d’attribuer les même numéros de partitions aux messages qui doivent être joints ensemble et de son côté Kafka assigne les mêmes numéros de partition aux instances de notre déploiement Kafka Streams.
Note sur l’Assignment Strategy : c’est la règle qui décrit comment sont attribuées les partitions. Il en existe trois: Range (défault), RoundRobin et Sticky. Les images précédentes considèrent les clients Kafka en général et illustre donc le résultat d’une Range Strategy. Mais la stratégie Sticky (qui est introduite pour répondre aux spécificités du stream processing) est celle qui est utilisée par vos applications Kafka Streams. Pour la question du qui: cette assignation est la responsabilité partagé d’un de vos consommateurs (Group Leader) et d’un des brokers Kafka (Group Coordinateur). Enfin pour ce qui est du quand, cette assignation à lieu au rebalance, c’est à dire à l’ajout/retrait d’un consommateur ou d’un partition. Plus de détails sur l’assignation des partitions dans ce post.
Un mauvais exemple de partitionnement
Nous avons parlé du co-partitioning de manière théorique et dans un cas idéal. Voyons maintenant si on peut illustrer son rôle sur nos jointures en jouant sur des cas d’erreurs. Procédons par l’absurde : Que se passe-t-il si nos flux ne sont pas co-partitionnés?
La suite de cet article peut être lu de manière linéaire. Des schémas et enregistrements permettront de restituer les démonstrations évoquées, mais elle est également reproductible (Github : Les produits suivants sont nécessaires au lancement de la démo :
L’impact du nombre de partitions
Reprenons l’exemple de la figure n°1. Soit un flux de clics avec trois partitions. Un autre flux, avec une unique partition représente les comptes utilisateurs. Les messages de ces streams ont pour clef un identifiant utilisateur. A chaque clics on veut ajouter à la volé des informations sur les utilisateurs. C’est le cas le plus fréquent, il est décrit comme un exercice d’enrichissement et se traduit par une jointure KStream-KTable. Observons ce qu’il se passe lorsque nos flux ne sont pas co-partitionnées.
fig 4: flux de clics join avec succès aux informations utilisateur
git clone --branch blog/step0 cd ko-partitioning ./gradlew run
Les commandes précédentes récupèrent le projet d’exemple, le compile et build l’image docker d’un générateur de données. Une fois terminé, les services suivants sont démarrés via docker :
- Zookeeper ;
- Kafka ;
- Schema Registry ;
- Datagenerator ;
- Kafka Streams App (dans un JVM en dehors de compose) ;
A ce niveau, si vous avez bien checkout la branche blog/step0
, vous avez du tomber sur une petite surprise. Enregistrement de la démo n°0.
Encountered the following unexpected Kafka exception during processing, this usually indicate Streams internal errors: org.apache.kafka.streams.errors.TopologyException: Invalid topology stream-thread [<application.id>-...-StreamThread-1-consumer] Topics not co-partitioned: [{ACCOUNT-TOPIC=1, CLICK-TOPIC=3}]
Pour les raisons que nous avons abordées, Kafka Streams n’autorise pas notre jointure. Une TopologyException
est levée par CopartitionedTopicsEnforcer
. L’usage de KStream#groupByKey
, (sans modification des clefs) sous entend que l’on utilise directement les clefs des messages Kafka et que l’on considère que ces messages ont été partitionnés avec le DefaultPartitioner
. Quoiqu’il en soit le message est clair. Pour relancer notre application ces streams doivent avoir le même nombre partition. Après avoir changé de branche, on réalise alors l’arrêt/relance de nos services et notre application. Dans notre première fenêtre :
./gradlew dockerComposeDown cleanUpState git checkout -b blog/step1 origin/blog/step1 ./gradlew run
La seule modification observée est l’ajustement du nombre de partitions du topic de comptes (application.conf
) :
Et voilà ! Nos services, notre générateur de données et notre application fonctionnent. Enregistrement de la démo n°1
Note : Cette disposition de topics, avec un flux largement moins partitionné, est fréquente. Ce dernier représente souvent l’évolution d’un référentiel de taille réduite comme un agrégat ou un table de référence (stock ou utilisateur). Nous verrons donc plus tard dans l’article une meilleur façon de surmonter cette erreur sans toucher aux partitions (avec l’usage d’un GlobalKTable
).
Même si Kafka Streams pose une première sécurité, notre expérience ne tombe pas à l’eau pour autant. Le nombre de partition ne nous permet pas d’illustrer le co-partitioning, mais nous avons vu au paragraphe précédent qu’il dépendait de plusieurs facteurs. Voyons ce que l’on peut activer d’autre.
L’impact du partitionner
Nos topics ont bien 3 partitions chacun. Mais que se passerait-il si tous les comptes utilisateurs étaient publiés dans la même partition ? Ici c’est l’usage d’un custom Partitioner
qui va nous permettre de réaliser notre expérience. Puisqu’une sécurité nous empêche d’avoir des topics de partitions différentes pour notre jointure, nous allons introduire une anomalie à un autre niveau de la définition du co-partitioning. Pensez à un partitionner custom qui renvoie, quoi qu’il arrive, la même partition. On ajoute au code existant :
class ConstPartitioner extends DefaultPartitioner { override def partition(_: String, _: Array[Byte], //...): Int = 2 }
L’unique différence entre les producers des clics et des comptes utilisateur devient alors leur partitioner (via la Producer Config: partitioner.class
).
Note : Dans mon exemple, l’usage du constructeur ProducerRecord::new(String, Integer, K, V)
où l’entier est un numéro de partition aurait été suffisant. L’intérêt est de souligner l’impact que peut avoir des customs partitionner pour nos downstream. Dans notre première fenêtre :
# (Ctrl + C) ./gradlew dockerComposeDown cleanUpState git checkout -b blog/step2 origin/blog/step2 ./gradlew run
Il vous est alors possible, dans une autre fenêtre de terminal de consommer les messages avec les commandes suivantes :
./gradlew consumeAccounts --console=plain
Dans une troisième fenêtre de terminal les clics enrichis d’information utilisateur par notre application Kafka Streams
./gradlew consumeClicks --console=plain
Le résultat nous donne un référentiel de quatre utilisateurs avec des identifiants (AccountId
) allant de 0 à 3. Pourtant lorsqu’on y joint le flux de leurs clics avec une application Kafka Streams seul les identifiants 2 et 3 trouvent leurs éléments correspondant. Alors que se passe-t-il ? Ici trois StreamTasks
se chargent de traiter chacune des partitions au sein de la même StreamThread
. La table en figure 3 est en réalité partitionnée. Chaque StreamTasks
dispose d’une partition de la table des comptes. Le déséquilibre induit par le partitionner fait qu’à l’exception de task0_2
toutes les autres tasks ont un store vide. Les tâches 0_0 et 0_1 ne sont donc pas capable de rapprocher les informations de comptes lorsqu’il reçoivent un événement de clics (respectivement AccountID("ID-000")
et AccountID("ID-001")
). Pour mettre en évidence le tout, la jointure qui est réalisée est un LEFT_JOIN
. On voit les valeurs null provoquée par les clics non enrichis. Recording de la demo n°2
fig 5: flux de clics partiellement join aux informations utilisateurs
val builder = new StreamsBuilder val clickStreams: KStream[AccountId, Click] = builder.stream(config.application.inputClickTopic.name) val accountTable: KTable[AccountId, Account] = builder.table(config.application.inputAccountTopic.name) val newClicks: KStream[AccountId, UserEvent] = clickStreams.leftJoin(accountTable) { (click, maybeAccount) => Option(maybeAccount).map(account => UserEvent(account, click)).orNull }
Maintenant que le problème est illustré revenons à un situation correct en utilisant bien le DefautPartitionner
pour nos deux streams. On réalise alors l’arrêt/relance nos services et notre application.
Co-partitioning by design
Nous avons traité un cas d’erreur particulier dans le chapitre précédent. Voyons maintenant comment Kafka Streams nous permet de résoudre automatiquement ces soucis de partitioning.
La GlobalKTable, un référentiel partagé
Revenons d’abord sur le cas de l’enrichissement. Nous avons un topic avec une forte volumétrie qui représente un flux d’évènements (CLICK-TOPIC
). Nous avons également un topic de volumétrie plus faible et qui représente le changelog d’un référentiel (ACCOUNT-TOPIC
). Dans ce cas classique il est d’usage d’attribuer plus de partitions à notre CLICK-TOPIC
car ces consommateurs sont plus susceptibles de profiter de la scalabilité des groupes de consommations. Une structure au sein de Kafka Streams va nous permettre de réaliser une jointure entre ces deux topics indépendamment de leur nombre de partitions: La GlobalKTable
.
fig 6: Broadcast des informations utilisateurs
L’idée ici est simple. Plutôt que d’avoir une table partitionnée dont chaque partition est maintenue par une StreamTask
nous aurons un broadcast de la table entière sur chaque instance Kafka Streams. Ces instances d’application pourront donc rapprocher tous les clics de leur information dans la table. On coupe toute nos commande puis dans la première fenêtre :
# (Ctrl + C) ./gradlew dockerComposeDown cleanUpState git checkout -b blog/step3 origin/blog/step3 ./gradlew run
On a bien le fonctionnement attendu ici. Notre application se lance correctement, bien que les topics soient configurés de la même manière qu’à la première partie. Également, tous les clics sont enrichis. Plus d’information manquantes. Alors quelle sont les modifications de code? On représente juste notre référentiel par une GlobalKTable[K, V]
en utilisant la méthode : StreamsBuilder#globaTable
. Les changements de code principaux :
val builder = new StreamsBuilder implicit val materializedAccount: Materialized[AccountId, Account, ByteArrayKeyValueStore] = ??? // new val clickStreams: KStream[AccountId, Click] = builder.stream(config.application.inputClickTopic.name) val accountTable: GlobalKTable[AccountId, Account] = builder .globalTable(config.application.inputAccountTopic.name, materializedAccount) val newClicks: KStream[AccountId, UserEvent] = clickStreams.leftJoin(accountTable)( (id, _) => id, (click, maybeAccount) => Option(maybeAccount).map(account => UserEvent(account, click)).orNull )
Marquer des streams pour une répartition
Nous avons donc, dans notre arsenal, autre pattern classique de plus (KStream-GlobalKTable). Mais cela semble un peu contraignant. Si on se réfère uniquement aux parties 2.1 et 3.1 on pourrait penser que nos seuls choix pour réaliser une jointure sont :
- N’avoir que des topics de même nombre de partition ;
Ou
- Forwarder l’ensemble de nos tables sur toutes nos instances ;
Deux méthodes qui pourraient sur le long terme apporter des problématiques opérationnelles si elles sont utilisées en dehors de leur cas d’usage. Il est temps de passer à une situation plus générale et de regarder comment Kafka Streams re-partitionne nos flux. Parce que oui, notre librairie intègre cette notion de répartition. Mais oubliez les techniques de shuffle avec des échanges réseaux entre instances clientes comme avec Apache Spark. Ici toutes nos applications disposent d’une connection privilégiée avec les brokers Kafka, elles s’échangent donc la donnée par ce moyens, via des topics techniques de re-partition.
fig 7: Re-partitionnement d’un topic pour une jointure
Ce processus est déclenché lorsqu’une de nos transformations de données “marquent nos streams pour répartition”. Typiquement toutes les modifications des clefs (ex : avec un KStream#selectKey
) auront cette effet. C’est aussi la raison pour laquelle KStream#mapValues
est décrit comme plus optimal que KStream#map
. KStream#map
marquera nos streams pour une répartition, ce que vous voulez éviter si cela n’est pas nécessaire. Comme le terme l’indique, “marqué pour répartition” ne signifie pas un re-partitionnement immédiat. Votre topologie doit présenter une key based operation (jointure, aggregation), qui déclenchera le passage par un topic de répartition. Les flux seront re-partitionnés en fonction des opérations qui suivent.
La banche 4 propose donc une transformation triviale de la clef de nos deux flux (changement de type et de serializer). Observons ce qu’il se passe. Couper les processus de toute les fenêtres et dans la première réalisez les commandes :
# (Ctrl + C) ./gradlew dockerComposeDown cleanUpState git checkout -b blog/step4 origin/blog/step4 ./gradlew run
Tout fonctionne, sans modifications de topic, ni de table globale partagée. Néanmoins, notre topologie fait apparaître un nouveau topic intermédiaire : KSTREAM-TOTABLE-0000000003-repartition
.
Ce topic a été créé par Kafka Streams avec trois partitions pour matcher avec notre topic de clics utilisateurs. L’enregistrement de la demo n°4.
Co-partitionner ses streams
Peut-on aller plus loin ? Nous avons identifié les transformations susceptibles de repartitionner les streams, mais peut-on forcer cette action. Voyons une dernière méthode qui imposera un partitionnement à coup sûr.
The repartition operator.
Publier nos messages avec une clef différente peut nous aider à changer le partitionnement de nos messages. Mais cela peut aussi se faire en conservant les même clefs et en publiant les messages dans un topic de partition différente. Par exemple, pour matcher le partitionnement d’un autre flux avec lequel on va réaliser une jointure. La première approche qui nous viendrait en tête est d’utiliser un KStream#to
en passant en paramètre un "nom_de_topic_intermédiaire"
, puis de consommer les messages de ce dernier topic. C’est assez fastidieux et cela n’aide pas la lisibilité de notre code.
A la place KStream#through
, va nous permettre de produire et consommer avec une seule instruction. La méthode renvoie un KStream
. Il nous faut garder en tête que les éléments de ce KStream
sont repassés par Kafka. Le code pourrait se rapprocher de ce bloc :
val accountMateriazer: Materialized[String, Account, ByteArrayKeyValueStore] = ??? val accountTable: KTable[AccountId, Account] = builder .stream(config.application.inputAccountTopic.name) // potential key transformation # selectKey(... .through("nom_de_topic_intermédiaire") .toTable(accountMateriazer) clickStream.join(accountTable // etc ...
Plutôt concis et pratique. Mais il reste tout de même quelque manipulations préalables à réaliser pour que ce code fonctionne. Le topic nom_de_topic_intermédiaire
doit être créé avec le bon nombre de partition. En général l’auto création de topic ne sera pas activée. Et si nous changeons d’avis sur le nombre de partitions adéquat, d’autres opérations d’administration seront nécessaires (suppression, création, autorisation…). Pour répondre à cette dernière problématique KAFKA-8611 introduit l’opérateur repartition.
Spécifié dans le KIP-221 il s’agit d’ajouter une méthode KStream#repartition
qui forcera la répartition (comme KStream#through
) mais qui prendra aussi en charge la création du topic. Cette méthode prend un instance de Repartitioned
(à configurer avec : un nombre de partition, un nom de topic et des serializers). La méthode renvoie un KStream
. À l’écriture de cet article la fonctionnalité à été ajoutée à la release 2.6.0 qui devrait être disponible mi-juin.
val accountTable: KTable[AccountId, Account] = builder .stream(config.application.inputAccountTopic.name) .repartition(Repartitioned.numberOfPartitions(10).withName("nom_de_topic_intermédiaire")) .toTable(accountMateriazer) clickStream.join(accountTable // etc ...
Pour plus d’information sur la manière dont se comporte cet opérateur n’hésitez pas à vous reporter au KIP-221.
Conclusion
Nous y sommes, nous avons survolé quelques manières de jouer avec nos partitions, que ce soit avant une jointure ou après une agrégation. J’espère qu’avoir vu ces illustrations vous mettra sur la voix la prochaine fois que vous vous battrez avec vos partitions. En attend n’hésitez pas à visiter le site kafka-tutorials.confluent.io. La communauté Kafka y collecte tout un tas de transformations communes en stream processing et vous explique comment les réaliser avec Kafka Streams et ksqlDB. A voir absolument :
- How to: Rekey with KSQL function
- How to: Join a stream to a table in KSQL
- How to: Join a stream to a stream in KSQL
- Je vous ai préparé un tutoriel: Window Final Result
- Un jour j’ai dit des trucs sympas sur la gestion d’erreurs (Kafka Summit San Francisco)
- Avec les copains on donne aussi une formation sur ces sujets: Real-time apps with Kafka Streams
- Documentation: les co-partitioning requirements avec Kafka Streams
- Documentation: les co-partitioning requirements avec ksqlDB