├── .idea └── vcs.xml ├── src └── main │ ├── java │ └── com │ │ └── shapira │ │ └── examples │ │ └── streams │ │ └── clickstreamenrich │ │ ├── model │ │ ├── UserWindow.java │ │ ├── PageView.java │ │ ├── Search.java │ │ ├── UserActivity.java │ │ └── UserProfile.java │ │ ├── Constants.java │ │ ├── serde │ │ ├── JsonSerializer.java │ │ ├── JsonDeserializer.java │ │ └── WrapperSerde.java │ │ ├── ClickstreamEnrichment.java │ │ └── GenerateData.java │ └── resources │ └── log4j.properties ├── README.md └── pom.xml /.idea/vcs.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | -------------------------------------------------------------------------------- /src/main/java/com/shapira/examples/streams/clickstreamenrich/model/UserWindow.java: -------------------------------------------------------------------------------- 1 | package com.shapira.examples.streams.clickstreamenrich.model; 2 | 3 | /** 4 | * Created by gwen on 1/28/17. 5 | */ 6 | public class UserWindow { 7 | int userID; 8 | long timestamp; 9 | } 10 | -------------------------------------------------------------------------------- /src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | log4j.rootLogger=INFO, stderr 2 | 3 | log4j.appender.stderr=org.apache.log4j.ConsoleAppender 4 | log4j.appender.stderr.layout=org.apache.log4j.PatternLayout 5 | log4j.appender.stderr.layout.ConversionPattern=[%d] %p %m (%c)%n 6 | log4j.appender.stderr.Target=System.err -------------------------------------------------------------------------------- /src/main/java/com/shapira/examples/streams/clickstreamenrich/model/PageView.java: -------------------------------------------------------------------------------- 1 | package com.shapira.examples.streams.clickstreamenrich.model; 2 | 3 | /** 4 | * Created by gwen on 1/28/17. 5 | */ 6 | public class PageView { 7 | int userID; 8 | String page; 9 | 10 | public PageView(int userID, String page) { 11 | this.userID = userID; 12 | this.page = page; 13 | } 14 | 15 | public int getUserID() { 16 | return userID; 17 | } 18 | 19 | public String getPage() { 20 | return page; 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /src/main/java/com/shapira/examples/streams/clickstreamenrich/model/Search.java: -------------------------------------------------------------------------------- 1 | package com.shapira.examples.streams.clickstreamenrich.model; 2 | 3 | /** 4 | * Created by gwen on 1/28/17. 5 | */ 6 | public class Search { 7 | int userID; 8 | String searchTerms; 9 | 10 | public Search(int userID, String searchTerms) { 11 | this.userID = userID; 12 | this.searchTerms = searchTerms; 13 | } 14 | 15 | public int getUserID() { 16 | return userID; 17 | } 18 | 19 | public String getSearchTerms() { 20 | return searchTerms; 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /src/main/java/com/shapira/examples/streams/clickstreamenrich/Constants.java: -------------------------------------------------------------------------------- 1 | package com.shapira.examples.streams.clickstreamenrich; 2 | 3 | /** 4 | * Created by gwen on 1/28/17. 5 | */ 6 | public class Constants { 7 | 8 | public static final String BROKER = "localhost:9092"; 9 | 10 | public static final String USER_PROFILE_TOPIC = "clicks.user.profile"; 11 | public static final String PAGE_VIEW_TOPIC = "clicks.pages.views"; 12 | public static final String SEARCH_TOPIC = "clicks.search"; 13 | public static final String USER_ACTIVITY_TOPIC = "clicks.user.activity"; 14 | 15 | } 16 | -------------------------------------------------------------------------------- /src/main/java/com/shapira/examples/streams/clickstreamenrich/serde/JsonSerializer.java: -------------------------------------------------------------------------------- 1 | package com.shapira.examples.streams.clickstreamenrich.serde; 2 | 3 | import com.google.gson.Gson; 4 | import org.apache.kafka.common.serialization.Serializer; 5 | 6 | import java.nio.charset.Charset; 7 | import java.util.Map; 8 | 9 | public class JsonSerializer implements Serializer { 10 | 11 | private Gson gson = new Gson(); 12 | 13 | @Override 14 | public void configure(Map map, boolean b) { 15 | 16 | } 17 | 18 | @Override 19 | public byte[] serialize(String topic, T t) { 20 | return gson.toJson(t).getBytes(Charset.forName("UTF-8")); 21 | } 22 | 23 | @Override 24 | public void close() { 25 | 26 | } 27 | } -------------------------------------------------------------------------------- /src/main/java/com/shapira/examples/streams/clickstreamenrich/model/UserActivity.java: -------------------------------------------------------------------------------- 1 | package com.shapira.examples.streams.clickstreamenrich.model; 2 | 3 | /** 4 | * Created by gwen on 1/28/17. 5 | */ 6 | public class UserActivity { 7 | int userId; 8 | String userName; 9 | String zipcode; 10 | String[] interests; 11 | String searchTerm; 12 | String page; 13 | 14 | public UserActivity(int userId, String userName, String zipcode, String[] interests, String searchTerm, String page) { 15 | this.userId = userId; 16 | this.userName = userName; 17 | this.zipcode = zipcode; 18 | this.interests = interests; 19 | this.searchTerm = searchTerm; 20 | this.page = page; 21 | } 22 | 23 | public UserActivity updateSearch(String searchTerm) { 24 | this.searchTerm = searchTerm; 25 | return this; 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /src/main/java/com/shapira/examples/streams/clickstreamenrich/model/UserProfile.java: -------------------------------------------------------------------------------- 1 | package com.shapira.examples.streams.clickstreamenrich.model; 2 | 3 | /** 4 | * Created by gwen on 1/28/17. 5 | */ 6 | public class UserProfile { 7 | int userID; 8 | String userName; 9 | String zipcode; 10 | String[] interests; 11 | 12 | 13 | public UserProfile(int userID, String userName, String zipcode, String[] interests) { 14 | this.userID = userID; 15 | this.userName = userName; 16 | this.zipcode = zipcode; 17 | this.interests = interests; 18 | } 19 | 20 | public int getUserID() { 21 | return userID; 22 | } 23 | 24 | public UserProfile update(String zipcode, String[] interests) { 25 | this.zipcode = zipcode; 26 | this.interests = interests; 27 | return this; 28 | } 29 | 30 | public String getUserName() { 31 | return userName; 32 | } 33 | 34 | public String getZipcode() { 35 | return zipcode; 36 | } 37 | 38 | public String[] getInterests() { 39 | return interests; 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /src/main/java/com/shapira/examples/streams/clickstreamenrich/serde/JsonDeserializer.java: -------------------------------------------------------------------------------- 1 | package com.shapira.examples.streams.clickstreamenrich.serde; 2 | 3 | 4 | import com.google.gson.Gson; 5 | import org.apache.kafka.common.serialization.Deserializer; 6 | 7 | import java.util.Map; 8 | 9 | public class JsonDeserializer implements Deserializer { 10 | 11 | private Gson gson = new Gson(); 12 | private Class deserializedClass; 13 | 14 | public JsonDeserializer(Class deserializedClass) { 15 | this.deserializedClass = deserializedClass; 16 | } 17 | 18 | public JsonDeserializer() { 19 | } 20 | 21 | @Override 22 | @SuppressWarnings("unchecked") 23 | public void configure(Map map, boolean b) { 24 | if(deserializedClass == null) { 25 | deserializedClass = (Class) map.get("serializedClass"); 26 | } 27 | } 28 | 29 | @Override 30 | public T deserialize(String s, byte[] bytes) { 31 | if(bytes == null){ 32 | return null; 33 | } 34 | 35 | return gson.fromJson(new String(bytes),deserializedClass); 36 | 37 | } 38 | 39 | @Override 40 | public void close() { 41 | 42 | } 43 | } -------------------------------------------------------------------------------- /src/main/java/com/shapira/examples/streams/clickstreamenrich/serde/WrapperSerde.java: -------------------------------------------------------------------------------- 1 | package com.shapira.examples.streams.clickstreamenrich.serde; 2 | 3 | import org.apache.kafka.common.serialization.Deserializer; 4 | import org.apache.kafka.common.serialization.Serde; 5 | import org.apache.kafka.common.serialization.Serializer; 6 | 7 | import java.util.Map; 8 | 9 | 10 | public class WrapperSerde implements Serde { 11 | 12 | final private Serializer serializer; 13 | final private Deserializer deserializer; 14 | 15 | public WrapperSerde(Serializer serializer, Deserializer deserializer) { 16 | this.serializer = serializer; 17 | this.deserializer = deserializer; 18 | } 19 | 20 | @Override 21 | public void configure(Map configs, boolean isKey) { 22 | serializer.configure(configs, isKey); 23 | deserializer.configure(configs, isKey); 24 | } 25 | 26 | @Override 27 | public void close() { 28 | serializer.close(); 29 | deserializer.close(); 30 | } 31 | 32 | @Override 33 | public Serializer serializer() { 34 | return serializer; 35 | } 36 | 37 | @Override 38 | public Deserializer deserializer() { 39 | return deserializer; 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # kafka-clickstream-enrich 2 | Kafka Streams Example - Joining streams to generate rich clickstream analysis data 3 | 4 | Overview 5 | -------- 6 | This example takes 2 streams of data: Stream of searches and stream of user clicks 7 | It also takes a stream of profile-updates, representing changes to a profiles table (assume we are getting those changes from MySQL using KafkaConnect connector) 8 | 9 | It joins those activity streams together, to generate an holistic view of user activity. The results show you, in one record the user's location, interests, what they searched for and what they ended up clicking. Providing rich source of data for analysis - which products are more desirable for which audience? "users who searched for this also looked at..." and other data products. 10 | 11 | This example makes use of the unique windowed-join, allowing us to match clicks with the search that happened in the same time window - in order to generate relevant results. 12 | 13 | To run: 14 | -------- 15 | 16 | 0. Build the project with `mvn package`, this will generate an uber-jar with the streams app and all its dependencies. 17 | 1. Next, we need to generate some clicks, searches and profiles. Run the generator. It should take about 5 seconds to run. Don't worry about complete lack of output... 18 | `$ java -cp target/uber-kafka-clickstream-enrich-1.0-SNAPSHOT.jar com.shapira.examples.streams.clickstreamenrich.GenerateData` 19 | 2. Run the streams app: 20 | `java -cp target/uber-kafka-clickstream-enrich-1.0-SNAPSHOT.jar com.shapira.examples.streams.clickstreamenrich.ClickstreamEnrichment` 21 | Streams apps typically run forever, but this one will just run for a minute and exit 22 | 3. Check the results: 23 | `bin/kafka-console-consumer.sh --topic clicks.user.activity --from-beginning --bootstrap-server localhost:9092 --property print.key=true` 24 | 25 | If you want to reset state and re-run the application (maybe with some changes?) on existing input topic, you can: 26 | 27 | 1. Reset internal topics (used for shuffle and state-stores): 28 | 29 | `bin/kafka-streams-application-reset.sh --application-id clicks --bootstrap-servers localhost:9092 --input-topics clicks.user.profile,clicks.pages.views,clicks.search ` 30 | 31 | 2. (optional) Delete the output topic: 32 | 33 | `bin/kafka-topics.sh --zookeeper localhost:2181 --delete --topic clicks.user.activity` -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 4.0.0 6 | 7 | kafka-clickstream-enrich 8 | kafka-clickstream-enrich 9 | 1.0-SNAPSHOT 10 | 11 | 12 | com.google.code.gson 13 | gson 14 | 2.2.2 15 | 16 | 17 | org.apache.kafka 18 | kafka-clients 19 | 2.8.0 20 | 21 | 22 | org.apache.kafka 23 | kafka-streams 24 | 2.8.0 25 | 26 | 28 | 29 | org.slf4j 30 | slf4j-log4j12 31 | 1.7.21 32 | 33 | 34 | org.apache.logging.log4j 35 | log4j-api 36 | 2.14.1 37 | 38 | 39 | org.apache.logging.log4j 40 | log4j-core 41 | 2.14.1 42 | 43 | 44 | 45 | 46 | 47 | 48 | org.apache.maven.plugins 49 | maven-compiler-plugin 50 | 3.1 51 | 52 | 1.8 53 | 1.8 54 | 55 | 56 | 57 | org.apache.maven.plugins 58 | maven-shade-plugin 59 | 3.2.4 60 | 61 | 62 | package 63 | 64 | shade 65 | 66 | 67 | 68 | 69 | uber-${project.artifactId}-${project.version} 70 | 71 | 72 | 73 | 74 | -------------------------------------------------------------------------------- /src/main/java/com/shapira/examples/streams/clickstreamenrich/ClickstreamEnrichment.java: -------------------------------------------------------------------------------- 1 | package com.shapira.examples.streams.clickstreamenrich; 2 | 3 | import com.shapira.examples.streams.clickstreamenrich.model.PageView; 4 | import com.shapira.examples.streams.clickstreamenrich.model.Search; 5 | import com.shapira.examples.streams.clickstreamenrich.model.UserActivity; 6 | import com.shapira.examples.streams.clickstreamenrich.model.UserProfile; 7 | import com.shapira.examples.streams.clickstreamenrich.serde.JsonDeserializer; 8 | import com.shapira.examples.streams.clickstreamenrich.serde.JsonSerializer; 9 | import com.shapira.examples.streams.clickstreamenrich.serde.WrapperSerde; 10 | import org.apache.kafka.clients.consumer.ConsumerConfig; 11 | import org.apache.kafka.common.serialization.Serdes; 12 | import org.apache.kafka.streams.KafkaStreams; 13 | import org.apache.kafka.streams.StreamsConfig; 14 | import org.apache.kafka.streams.kstream.Consumed; 15 | import org.apache.kafka.streams.kstream.JoinWindows; 16 | import org.apache.kafka.streams.kstream.KStream; 17 | import org.apache.kafka.streams.StreamsBuilder; 18 | import org.apache.kafka.streams.kstream.KTable; 19 | import org.apache.kafka.streams.kstream.Produced; 20 | import org.apache.kafka.streams.kstream.StreamJoined; 21 | 22 | import java.time.Duration; 23 | import java.util.Properties; 24 | 25 | public class ClickstreamEnrichment { 26 | 27 | public static void main(String[] args) throws Exception { 28 | 29 | Properties props = new Properties(); 30 | props.put(StreamsConfig.APPLICATION_ID_CONFIG, "clicks"); 31 | props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, Constants.BROKER); 32 | // Since each step in the stream will involve different objects, we can't use default Serde 33 | 34 | // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data 35 | // Note: To re-run the demo, you need to use the offset reset tool: 36 | // https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+Application+Reset+Tool 37 | props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); 38 | 39 | StreamsBuilder builder = new StreamsBuilder(); 40 | 41 | KStream views = builder.stream(Constants.PAGE_VIEW_TOPIC, Consumed.with(Serdes.Integer(), new PageViewSerde())); 42 | KTable profiles = builder.table(Constants.USER_PROFILE_TOPIC, Consumed.with(Serdes.Integer(), new ProfileSerde())); 43 | KStream searches = builder.stream(Constants.SEARCH_TOPIC, Consumed.with( Serdes.Integer(), new SearchSerde())); 44 | 45 | KStream viewsWithProfile = views.leftJoin(profiles, 46 | (page, profile) -> { 47 | if (profile != null) 48 | return new UserActivity(profile.getUserID(), profile.getUserName(), profile.getZipcode(), profile.getInterests(), "", page.getPage()); 49 | else 50 | return new UserActivity(-1, "", "", null, "", page.getPage()); 51 | 52 | }); 53 | 54 | KStream userActivityKStream = viewsWithProfile.leftJoin(searches, 55 | (userActivity, search) -> { 56 | if (search != null) 57 | userActivity.updateSearch(search.getSearchTerms()); 58 | else 59 | userActivity.updateSearch(""); 60 | return userActivity; 61 | }, 62 | JoinWindows.of(Duration.ofSeconds(1)), 63 | StreamJoined.with(Serdes.Integer(), new UserActivitySerde(), new SearchSerde())); 64 | 65 | userActivityKStream.to(Constants.USER_ACTIVITY_TOPIC, Produced.with(Serdes.Integer(), new UserActivitySerde())); 66 | 67 | KafkaStreams streams = new KafkaStreams(builder.build(), props); 68 | 69 | streams.cleanUp(); 70 | 71 | streams.start(); 72 | 73 | // usually the stream application would be running forever, 74 | // in this example we just let it run for some time and stop since the input data is finite. 75 | Thread.sleep(60000L); 76 | 77 | streams.close(); 78 | 79 | 80 | } 81 | 82 | static public final class PageViewSerde extends WrapperSerde { 83 | public PageViewSerde() { 84 | super(new JsonSerializer<>(), new JsonDeserializer<>(PageView.class)); 85 | } 86 | } 87 | 88 | static public final class ProfileSerde extends WrapperSerde { 89 | public ProfileSerde() { 90 | super(new JsonSerializer<>(), new JsonDeserializer<>(UserProfile.class)); 91 | } 92 | } 93 | 94 | static public final class SearchSerde extends WrapperSerde { 95 | public SearchSerde() { 96 | super(new JsonSerializer<>(), new JsonDeserializer<>(Search.class)); 97 | } 98 | } 99 | 100 | static public final class UserActivitySerde extends WrapperSerde { 101 | public UserActivitySerde() { 102 | super(new JsonSerializer<>(), new JsonDeserializer<>(UserActivity.class)); 103 | } 104 | } 105 | } 106 | -------------------------------------------------------------------------------- /src/main/java/com/shapira/examples/streams/clickstreamenrich/GenerateData.java: -------------------------------------------------------------------------------- 1 | package com.shapira.examples.streams.clickstreamenrich; 2 | 3 | import com.google.gson.Gson; 4 | import com.shapira.examples.streams.clickstreamenrich.model.PageView; 5 | import com.shapira.examples.streams.clickstreamenrich.model.Search; 6 | import com.shapira.examples.streams.clickstreamenrich.model.UserProfile; 7 | import org.apache.kafka.clients.producer.KafkaProducer; 8 | import org.apache.kafka.clients.producer.ProducerRecord; 9 | import org.apache.kafka.clients.producer.RecordMetadata; 10 | 11 | import java.util.ArrayList; 12 | import java.util.List; 13 | import java.util.Properties; 14 | 15 | /** 16 | * This class will generate fake clicks, fake searches and fake profile updates 17 | * For simplicity, we will actually generate very few events - 2 profiles, update to one profile, 3 searches, 5 clicks 18 | */ 19 | public class GenerateData { 20 | 21 | public static KafkaProducer producer = null; 22 | 23 | public static void main(String[] args) throws Exception { 24 | 25 | System.out.println("Press CTRL-C to stop generating data"); 26 | 27 | List> records = new ArrayList<>(); 28 | Gson gson = new Gson(); 29 | 30 | 31 | // add shutdown hook 32 | Runtime.getRuntime().addShutdownHook(new Thread( () -> { 33 | System.out.println("Shutting Down"); 34 | if (producer != null) 35 | producer.close(); 36 | })); 37 | 38 | 39 | 40 | // Two users 41 | String[] interests1 = {"Surfing", "Hiking"}; 42 | UserProfile user1 = new UserProfile(1, "Matthias", "94301", interests1 ); 43 | 44 | String[] interests2 = {"Ski", "Dancing"}; 45 | UserProfile user2 = new UserProfile(2, "Anna", "94302", interests2); 46 | 47 | records.add(new ProducerRecord<>(Constants.USER_PROFILE_TOPIC, user1.getUserID(), gson.toJson(user1))); 48 | records.add(new ProducerRecord<>(Constants.USER_PROFILE_TOPIC, user2.getUserID(), gson.toJson(user2))); 49 | 50 | // profile update 51 | 52 | String[] newInterests = {"Ski", "stream processing"}; 53 | 54 | records.add(new ProducerRecord<>(Constants.USER_PROFILE_TOPIC, user2.getUserID(), gson.toJson(user2.update("94303", newInterests)))); 55 | 56 | 57 | // Two searches 58 | 59 | Search search1 = new Search(1, "retro wetsuit"); 60 | Search search2 = new Search(2, "light jacket"); 61 | 62 | 63 | records.add(new ProducerRecord<>(Constants.SEARCH_TOPIC, search1.getUserID(), gson.toJson(search1))); 64 | records.add(new ProducerRecord<>(Constants.SEARCH_TOPIC, search2.getUserID(), gson.toJson(search2))); 65 | 66 | // three clicks 67 | 68 | PageView view1 = new PageView(1, "collections/mens-wetsuits/products/w3-worlds-warmest-wetsuit"); 69 | PageView view2 = new PageView(2, "product/womens-dirt-craft-bike-mountain-biking-jacket"); 70 | PageView view3 = new PageView(2, "/product/womens-ultralight-down-jacket"); 71 | 72 | records.add(new ProducerRecord<>(Constants.PAGE_VIEW_TOPIC, view1.getUserID(), gson.toJson(view1))); 73 | records.add(new ProducerRecord<>(Constants.PAGE_VIEW_TOPIC, view2.getUserID(), gson.toJson(view2))); 74 | records.add(new ProducerRecord<>(Constants.PAGE_VIEW_TOPIC, view3.getUserID(), gson.toJson(view3))); 75 | 76 | 77 | // Configure a producer. 78 | // We'll use User ID as the key for all events - since joins require a common key 79 | // Since we are going to write objects of different types as values, we'll serialize all of them to JSON strings ourselves 80 | // So the producer type and serializer are just for strings 81 | 82 | 83 | 84 | Properties props = new Properties(); 85 | 86 | props.put("bootstrap.servers", Constants.BROKER); 87 | props.put("key.serializer", "org.apache.kafka.common.serialization.IntegerSerializer"); 88 | props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); 89 | 90 | // Starting producer 91 | producer = new KafkaProducer<>(props); 92 | 93 | // Send existing events 94 | 95 | for (ProducerRecord record: records) 96 | producer.send(record, (RecordMetadata r, Exception e) -> { 97 | if (e != null) { 98 | System.out.println("Error producing to topic " + r.topic()); 99 | e.printStackTrace(); 100 | } 101 | }); 102 | 103 | 104 | // Sleep 5 seconds, to make sure we recognize the new events as a separate session 105 | records.clear(); 106 | Thread.sleep(5000); 107 | 108 | // One more search 109 | 110 | Search search3 = new Search(2, "carbon ski boots"); 111 | 112 | records.add(new ProducerRecord<>(Constants.SEARCH_TOPIC, search3.getUserID(), gson.toJson(search3))); 113 | 114 | // Two clicks 115 | PageView view4 = new PageView(2, "product/salomon-quest-access-custom-heat-ski-boots-womens"); 116 | PageView view5 = new PageView(2, "product/nordica-nxt-75-ski-boots-womens"); 117 | 118 | records.add(new ProducerRecord<>(Constants.PAGE_VIEW_TOPIC, view4.getUserID(), gson.toJson(view4))); 119 | records.add(new ProducerRecord<>(Constants.PAGE_VIEW_TOPIC, view5.getUserID(), gson.toJson(view5))); 120 | 121 | // Click for an unknown user without searches - we want to make sure we have results for those too. 122 | 123 | PageView view6 = new PageView(-1, "product/osprey-atmos-65-ag-pack"); 124 | records.add(new ProducerRecord<>(Constants.PAGE_VIEW_TOPIC, view6.getUserID(), gson.toJson(view6))); 125 | 126 | 127 | // Send additional events 128 | for (ProducerRecord record: records) 129 | producer.send(record, (RecordMetadata r, Exception e) -> { 130 | if (e != null) { 131 | System.out.println("Error producing to topic " + r.topic()); 132 | e.printStackTrace(); 133 | } 134 | }); 135 | 136 | 137 | // and done... 138 | 139 | producer.close(); 140 | 141 | 142 | } 143 | 144 | 145 | } 146 | --------------------------------------------------------------------------------