You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 40 Next »

Status

Current state: Under Discussion

Discussion thread: here

JIRA: KAFKA-3705

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

We want to close the gap between the semantics of KTables in streams and tables in relational databases. It is common practice to capture changes as they are made to tables in some RDMS into kafka topics (JDBC-connect, debezium, Maxwell). It is also common practice to use some sort of normalization inside the RDBMS so that entities end up in a one-to-many relationship. Usually RDBMSs offer good support to resolve this relationship with a join. Streams falls short here and the workaround (group by - join - lateral view) is not well supported as well and is not in line with the idea of record based processing.

 

Public Interfaces

Less intrusive

We would introduce a new Method into KTable and KTableImpl

KTable.java
   /**
     * 
     * Joins one record of this KTable to n records of the other KTable,
     * an update in this KTable will update all n matching records, an update
     * in other table will update only the one matching record.
     * 
     * @param the table containing n records for each K of this table
     * @param keyExtractor a {@code ValueMapper} returning the key of this table from the others value
     * @param joinPrefixFaker a {@code ValueMapper} returning an outputkey that when serialized only produces the
     *                             prefix of the output key which is the same as serializing K 
     * @param leftKeyExtractor a {@code ValueMapper} extracting the Key of this table from the resulting Key 
     * @param <KO> the resultings tables Key
     * @param <VO> the resultings tables Value
     * @param joiner
     * @return
     */
    <KO, VO, K1, V1> KTable<KO, VO> oneToManyJoin(KTable<K1, V1> other,
            ValueMapper<V1, K> keyExtractor,
            ValueMapper<K, KO> joinPrefixFaker,
            ValueMapper<KO, K> leftKeyExtractor, 
            ValueJoiner<V, V1, VO> joiner,
            Serde<K1> keyOtherSerde, Serde<V1> valueOtherSerde,
            Serde<KO> joinKeySerde, Serde<VO> joinValueSerde);

More intrusive

We would Introduce a new Complex type: Combined Key

package org.apache.kafka.streams;

public class CombinedKey<P,S> {
    public P prefix;
    public S suffix;
}

The method could be rewritten

KTable.java
   /**
     * 
     * Joins one record of this KTable to n records of the other KTable,
     * an update in this KTable will update all n matching records, an update
     * in other table will update only the one matching record.
     * 
     * @param the table containing n records for each K of this table
     * @param keyExtractor a {@code ValueMapper} returning the key of this table from the others value
     * @param leftKeyExtractor a {@code ValueMapper} extracting the Key of this table from the resulting Key 
     * @param <VO> the resultings tables Value
     * @param joiner
     * @return
     */
    <VO, K1, V1> KTable<CombinedKey<K,K1>,VO> oneToManyJoin(KTable<K1, V1> other,
            ValueMapper<V1, K> keyExtractor, 
            ValueJoiner<V, V1, VO> joiner,
            Serde<K1> keyOtherSerde, Serde<V1> valueOtherSerde,
            Serde<VO> joinValueSerde);

Tradeoffs

The more intrusive version gives the user better clarity that his resulting KTable is not only keyed by the other table's key but its also keyed by this table's key. So he will be less surprised that in a theoretical later aggregation he might find the same key from the other ktable twice. On the other hand the less intrusive method doesn't need to introduce this wrapper class but let the user handle the need of having both tables keys present in the output key himself. This might lead to a deeper understanding for the user and serdes might be able to pack the data denser. An additional benefit is that the user can stick with his default serde or his standard way of serializing when sinking the data into another topic using for example to() while the CombinedKey would require an additional mapping to what the less intrusive method has.

Back and forth mapper

This is a proposal to get rid of the Type CombinedKey in the return type. We would internally use a Combined key and a Combined Key Serde and apply the mappers only at the processing boundaries (ValueGetterSupplier, context.forward). The data will still be serialized for repartitioning in a way that is specific to Kafka and might prevent users from using their default tooling with these topics.

KTable.java
     /**
	 * 
     * Joins one record of this KTable to n records of the other KTable,
     * an update in this KTable will update all n matching records, an update
     * in other table will update only the one matching record.
     * 
     * @param the table containing n records for each K of this table
     * @param keyExtractor a {@code ValueMapper} returning the key of this table from the others value
     * @param customCombinedKey a {@code ValueMapper} allowing the CombinedKey to be wrapped in a custom object
	 * @param combinedKey a {@code ValueMapper} allowing to unwrap the custom object again.
     * @param <VO> the resultings tables Value
     * @param joiner
     * @return
     */
    <KO VO, K1, V1> KTable<KO,VO> oneToManyJoin(KTable<K1, V1> other,
            ValueMapper<V1, K> keyExtractor, 
			ValueMapper<CombinedKey<K1,K>,KO> outputKeyCombiner,
			ValueMapper<KO,CombinedKey<K1,K>> outputKeySpliter,
			ValueJoiner<V, V1, VO> joiner,
            Serde<K1> keyOtherSerde, 
	        Serde<V1> valueOtherSerde,
            Serde<VO> joinValueSerde);

Custom Serde

Introducing an additional new Serde. This is the approach is the counterpart to having a back and forth mapper. With this approach it is possible to keep any Custom serialization mechanism off the wire. How to serialize is completely with the user.

package org.apache.kafka.streams;

public class CombinedKeySerde<K,K1> extends Serde<CombinedKey<K,K1>> {
	
	public Serializer<K> getPartialKeySerializer();
 }
KTable.java
   /**
     * 
     * Joins one record of this KTable to n records of the other KTable,
     * an update in this KTable will update all n matching records, an update
     * in other table will update only the one matching record.
     * 
     * @param the table containing n records for each K of this table
     * @param keyExtractor a {@code ValueMapper} returning the key of this table from the others value
     * @param leftKeyExtractor a {@code ValueMapper} extracting the Key of this table from the resulting Key 
     * @param <VO> the resultings tables Value
     * @param joiner
     * @return
     */
    <VO, K1, V1> KTable<CombinedKey<K,K1>,VO> oneToManyJoin(KTable<K1, V1> other,
            ValueMapper<V1, K> keyExtractor, 
            ValueJoiner<V, V1, VO> joiner,
            Serde<K1> keyOtherSerde, Serde<V1> valueOtherSerde,
            Serde<VO> joinValueSerde,
			CombinedKeySerde<K,K1> combinedKeySerde);

Streams

We will implement a default CombinedKeySerde that will use a regular length encoding for both fields. So calls to the "intrusive approach" would constuct a default CombinedKeySerde and invoke the Serde Overload. This would work with all serde frameworks if the user is not interested in how the data is serialized in the topics.

Protobuf / Avro / thrift / Hadoop-Writeable / Custom

Users of these frameworks should have a very easy time implementing a CombinedKeySerde. Essentially they define an object that wraps K and K1 as usual keeping K1 as an optional field. The serializer returned from getPartialKeySerializer() would do the following:

  1. create such a wrapping object
  2. set the value for the K field
  3. serialize the wrapped object as usual.

This should work straight forward and users might implement a CombinedKeySerde that is specific to their framework and reuse the logic without implementing a new Serde for each key-pair.

JSON

Implementing a CombinedKeySerde depends on the specific framework with json. A full key would look like this "{  "a" :{ "key":"a1" }, "b": {"key":"b5" }   }" to generate a true prefix one had to generate "{ "a" :{ "key":"a1"", which is not valid json. This invalid Json will not leave the jvm but it might be more or less tricky to implement a serializer generating it. Maybe we could provide users with a utility method to make sure their serde statisfies our invariants.

 

 

Proposed Changes

Goal

With the two relations A,B and there is one A for each B and there may be many B's for each A. A is represented by the KTable the method described above gets invoked on, while B is represented by that methods first argument. We want to implement a Set of processors that allows a user to create a new KTable where A and B are joined based on the reference to A in B. A and B are represented as KTable B being partitioned by B's key and A being partitioned by A's key.

Algorithm


ascii_raw_KIP-213

 

  • Call enable sendOldValues() on sources with "*"
  • Register a child of B
    • extract A's key and B's key as key and B as value.
      • forward(key, null) for old
      • forward(key, b) for new
      • skip old if A's key didn't change (ends up in same partition)
  • Register sink for internal repartition topic (number of partitions equal to A, if a is internal prefer B over A for deciding number of partitions)
    • in the sink, only use A's key to determine partition
  • Register source for intermediate topic
    • co-partition with A's sources
    • materialize
    • serde for rocks needs to serialize A before B. ideally we use the same serde also for the topic
  • Register processor after above source.
    • On event extract A's key from the key
    • look up A by it's key
    • perform the join (as usual)
  • Register processor after A's processor
    • On event uses A's key to perform a Range scan on B's materialization
    • For every row retrieved perform join as usual
  • Register merger
    • Forward join Results
    • On lookup use full key to lookup B and extract A's key from the key and lookup A. Then perform join.
  • Merger wrapped into KTable and returned to the user.

Step by Step

TOPOLOGY INPUT ATOPOLOGY INPUT BSTATE A MATERIALZEDSTATE B MATERIALIZEINTERMEDIATE RECORDS PRODUCEDSTATE B OTHER TASKOutput A Source / Input Range ProccesorOUTPUT RANGE PROCESSOROUTPUT LOOKUP PROCESSOR
key: A0 value: [A0 ...] key: A0 value: [A0 ...]   Change<null,[A0 ...]>

invoked but nothing found.

Nothing forwarded

 
key: A1 value: [A1 ...] 

key: A0 value: [A0 ...]

key: A1 value: [A1 ...]

   Change<null,[A1 ...]>invoked but nothing found. Nothing forwarded 
 key: B0 : value [A2,B0 ...]

key: A0 value: [A0 ...]

key: A1 value: [A1 ...]

key: B0 : value [A2,B0 ...]partition key: A2 key: A2B0 value: [A2,B0 ...]key: A2B0 : value [A2,B0 ...]  

invoked but nothing found

Nothing forwarded

 key: B1 : value [A2,B1 ...]

key: A0 value: [A0 ...]

key: A1 value: [A1 ...]

key: B0 : value [A2,B0 ...]

key: B1 : value [A2,B1 ...]

partition key: A2 key: A2B1 value [A2,B1 ...]

key: A2B0 : value [A2,B0 ...]

key: A2B1 : value [A2,B1 ...]

  

invoked but nothing found

Nothing forwarded

key: A2 value: [A2 ...] 

key: A0 value: [A0 ...]

key: A1 value: [A1 ...]

key: A2 value: [A2 ...]

key: B0 : value [A2,B0 ...]

key: B1 : value [A2,B1 ...]

 

key: A2B0 : value [A2,B0 ...]

key: A2B1 : value [A2,B1 ...]

Change<null,[A2 ...]>

key A2B0 value: Change<null,join([A2 ...],[A2,B0 ...])

key A2B1 value: Change<null,join([A2 ...],[A2,B1...])

 
 key: B1 : value null key: B0 : value [A2,B0 ...]partition key: A2 key: A2B1 value:nullkey: A2B0 : value [A2,B0 ...]  key A2B1 value: Change<join([A2 ...],[A2,B1...],null)
 key: B3 : value [A0,B3 ...] 

key: B0 : value [A2,B0 ...]

key: B3 : value [A0,B3 ...]

partition key: A0 key: A0B3 value:[A0,B3 ...]

key: A2B0 : value [A2,B0 ...]

key: A0B3 : value [A0,B3 ...]

  key A0B3 value: Change<join(null,[A0 ...],[A0,B3...])
key: A2 value: null 

key: A0 value: [A0 ...]

key: A1 value: [A1 ...]

key: B0 : value [A2,B0 ...]

key: B3 : value [A0,B3 ...]

 

key: A2B0 : value [A2,B0 ...]

key: A0B3 : value [A0,B3 ...]

Change<[A2 ...],null>key A2B0 value: Change<join([A2 ...],[A2,B0 ...],null) 

 

Range lookup

It is pretty straight forward to completely flush all changes that happened before the range lookup into rocksb and let it handle a the range scan. Merging rocksdb's result iterator with current in-heap caches might be not in scope of this initial KIP. Currently we at trivago can not identify the rocksDb flushes to be a performance problem. Usually the amount of emitted records is the harder problem to deal with in the first place.

Missing reference to A

B records with a 'null' A-key value would be silently dropped.

Compatibility, Deprecation, and Migration Plan

  • There is no impact to existing users.

Rejected Alternatives

If there are alternative ways of accomplishing the same thing, what were they? The purpose of this section is to motivate why the design is the way it is and not some other way.



Alternative Design - Adam Bellemare July 2018

Features:

  • Performs updates from both sides of the join.
  • Uses user-specified serdes to perform the rangeScans.
  • Resolves out-of-order processing due to switching the foreignKey on the event side.

Current Limitations:

  • Requires two full materialized State Stores at a minimum
    • One for the rangeScannable events from the left table.
    • One for the highwater mark for resolving out-of-order processing of events.

Motivation

Same as above.

Public Interfaces

<V0, KL, VL, KR, VR> KTable<KL, V0> foreignKeyJoin(KTable<KR, VR> other,
                                                    ValueMapper<VL, KR> foreignKeyExtractor,
                                                    final ValueJoiner<VR, VL, V0> joiner,
                                                    final Materialized<KL, V0, KeyValueStore<Bytes, byte[]>> materialized,
                                                    Serde<KL> thisKeySerde,
                                                    Serde<KR> otherKeySerde,
                                                    Serde<VR> otherValueSerde,
                                                    Serde<V0> joinedValueSerde);


Workflows

For the purposes of this KIP, keep these concepts in mind as you read through the document.

Left KTable - This contains the many. Think of this table as a large number of events with their own distinct keys, each of which contains a foreign key that we wish to join on. This is the table of events which we will be joining from. (ie: KTableLeft.foreignKeyJoin ... )
Example: Users clicking on a product, where an event contains a key on UserId+Timestamp and a value containing the foreign key, productId.

Left Sample: 
(key,value)  =  (UserA-TimestampA, Event(type=Click, productid=900560) )


Right KTable - This contains the one. This is the table that contains the data keyed on the foreign key.

Example: The Product Data:

Right Sample:
(key, value)  =  (900560, ProductDetails(size = small, colour = blue, fabric = wool)) 


Overview

The overall process is fairly simple. This process is outlined below.

BellemareOverview213


Repartitioning using a CombinedKey Wrapper

CombinedKey is a simple tuple wrapper that simply stores the primary key and the foreign key together. The CombinedKey serde requires the usage of both the primary key and foreign key serdes, and that is left entirely up to the user to specify. It is assumed that both the complete foreign key and the extracted foreign key will serialize in exactly the same way. I do not know of the json complexities that Jan speaks of above, but as long as the extracted foreign key from the left table is identical to the primary key in the right table, the serialization should be identical.

The CombinedKey is serialized as follows:

{4-bytes foreign key serialized length}{n-bytes serialized foreign key}{4-bytes primary key serialized length}{n-bytes serialized primary key}

This can support up to MAXINT size of data in bytes per key, which currently far exceeds the realistic sizing of messages in Kafka. If we so wish, we could expand this to be much larger, but I don't see a need to do so at this time. A more clever or compact serialization solution may be available, this is just the simplest one I came up with.

When performing the prefix scan, we simply drop the primary key component, such that the serialized combined key looks like:

{4-bytes foreign key serialized length}{n-bytes serialized foreign key}

This allows for the RocksDB prefix scan to work correctly, since all data is stored in the same format in the key-value store.


Custom Partitioner Usage

A custom partitioner is used to ensure that the CombinedKey left table data is correctly copartitioned with the right table data. This is a simple operation, as it simply extracts the foreign key and applies the partitioner logic. It is important that the same partitioner that is used to partition the right table is used to partition the rekeyed left-table data.


CombinedKeys Usage by Left Processor

CombinedKeys are used simply to ensure that the left data is correctly copartitioned, and such that the primary key is preserved for future downstream rekeying.

LeftProcessingWithCombinedKey

CombinedKeys Usage by PrefixScan Processor

The left's CombinedKey is extremely important for allowing the prefixScan to occur. The serialized byte payload is used to determine prefix equivalence during the prefixScan.

RangeScanCombinedKeyUsage



Problem: Out-of-order processing of Rekeyed data

There is an issue that arises when updating a foreign key value in an event in the left table. We first must send a delete on the previous CombinedKey, and send the new value on the new CombinedKey. This results in a race condition, as illustrated below.

RaceCondition-Simple

This race condition is especially visible when multiple threads are being used to process the Kafka topic partitions. A given thread on a given node may process its records much sooner or much later than the other threads, due to load, network, polling cycles, and a variety of other causes. It is expected that there is no guarantee on the order in which the messages arrive. All things equal, it is equally likely that you would see the "null" message as the final result as it would be the correct updated message. This issue is only further compounded if the foreign key were changed several times in short succession, with multiple additional partitions. The only remedy to this that I can currently find is to propagate the offset of the original update along with the payload of the message. This introduces my next section.


Solution: The PropagationWrapper - Resolving out-of-order data processing

Purpose

This class stores two properties for handling out-of-order resolution.

  • long offset - Contains the offset, in a given partition, for an event keyed on KR in the original source topic. This is used in the highwater mark computation at the resolution stage.
  • boolean propagate - This is used to halt the propagation of the "null" value for foreignKey changes, but NOT the propagation of "null" values for deletions. This comes into play during resolution.

Deletion

Since a deleted record will never be just a plain "null" value anymore (but instead a wrapped null), the log cleaner will not automatically clean it up. This can still be compacted, but it will result in higher data usage than if we were able to use pure null values. A suggestion is that we can add the concept of a "weak reference" to Kafka, where the log cleaner will still treat it as a null for log cleaner purposes, but where it could also still contain data. This is beyond the scope of this ticket.

Usage - Example of PropagationWrapper


PropagationWrapperIntro

Multiple Rapid Foreign-Key Value Changes - Example of PropagationWrapper

In this case, there are multiple, rapid changes made to the foreign-key of a single record. We will need to resolve which one is the correct final value.

HighwaterMarkUsage


Current Issues to Work out

1) Work with DefaultPartitioner class and with custom partitioners. Currently hardwired using a copy of the default Partitioner code for key != null logic.


2) Testing Framework. I have done all of my testing outside of the kafka streams unit testing framework, using my own framework that brings up a kafka cluster with user-specified partition counts. I was able to test the copartitioning and


3) Ensuring covereage for restoring from a failed state. I am unsure how logging state stores back data up to the cluster with regards to internal topic propagation, which has influenced the decision to include a "propagate" boolean in the PropagationWrapper, versus letting the high-water mark resolve it.







  • No labels