{"payload":{"allShortcutsEnabled":false,"fileTree":{"src/main/java/io/confluent/examples/streams/interactivequeries/kafkamusic":{"items":[{"name":"KafkaMusicExample.java","path":"src/main/java/io/confluent/examples/streams/interactivequeries/kafkamusic/KafkaMusicExample.java","contentType":"file"},{"name":"KafkaMusicExampleDriver.java","path":"src/main/java/io/confluent/examples/streams/interactivequeries/kafkamusic/KafkaMusicExampleDriver.java","contentType":"file"},{"name":"MusicPlaysRestService.java","path":"src/main/java/io/confluent/examples/streams/interactivequeries/kafkamusic/MusicPlaysRestService.java","contentType":"file"},{"name":"SongBean.java","path":"src/main/java/io/confluent/examples/streams/interactivequeries/kafkamusic/SongBean.java","contentType":"file"},{"name":"SongPlayCountBean.java","path":"src/main/java/io/confluent/examples/streams/interactivequeries/kafkamusic/SongPlayCountBean.java","contentType":"file"}],"totalCount":5},"src/main/java/io/confluent/examples/streams/interactivequeries":{"items":[{"name":"kafkamusic","path":"src/main/java/io/confluent/examples/streams/interactivequeries/kafkamusic","contentType":"directory"},{"name":"HostStoreInfo.java","path":"src/main/java/io/confluent/examples/streams/interactivequeries/HostStoreInfo.java","contentType":"file"},{"name":"KeyValueBean.java","path":"src/main/java/io/confluent/examples/streams/interactivequeries/KeyValueBean.java","contentType":"file"},{"name":"MetadataService.java","path":"src/main/java/io/confluent/examples/streams/interactivequeries/MetadataService.java","contentType":"file"},{"name":"WordCountInteractiveQueriesDriver.java","path":"src/main/java/io/confluent/examples/streams/interactivequeries/WordCountInteractiveQueriesDriver.java","contentType":"file"},{"name":"WordCountInteractiveQueriesExample.java","path":"src/main/java/io/confluent/examples/streams/interactivequeries/WordCountInteractiveQueriesExample.java","contentType":"file"},{"name":"WordCountInteractiveQueriesRestService.java","path":"src/main/java/io/confluent/examples/streams/interactivequeries/WordCountInteractiveQueriesRestService.java","contentType":"file"}],"totalCount":7},"src/main/java/io/confluent/examples/streams":{"items":[{"name":"interactivequeries","path":"src/main/java/io/confluent/examples/streams/interactivequeries","contentType":"directory"},{"name":"microservices","path":"src/main/java/io/confluent/examples/streams/microservices","contentType":"directory"},{"name":"utils","path":"src/main/java/io/confluent/examples/streams/utils","contentType":"directory"},{"name":"AnomalyDetectionLambdaExample.java","path":"src/main/java/io/confluent/examples/streams/AnomalyDetectionLambdaExample.java","contentType":"file"},{"name":"ApplicationResetExample.java","path":"src/main/java/io/confluent/examples/streams/ApplicationResetExample.java","contentType":"file"},{"name":"GlobalKTablesExample.java","path":"src/main/java/io/confluent/examples/streams/GlobalKTablesExample.java","contentType":"file"},{"name":"GlobalKTablesExampleDriver.java","path":"src/main/java/io/confluent/examples/streams/GlobalKTablesExampleDriver.java","contentType":"file"},{"name":"MapFunctionLambdaExample.java","path":"src/main/java/io/confluent/examples/streams/MapFunctionLambdaExample.java","contentType":"file"},{"name":"PageViewRegionExample.java","path":"src/main/java/io/confluent/examples/streams/PageViewRegionExample.java","contentType":"file"},{"name":"PageViewRegionExampleDriver.java","path":"src/main/java/io/confluent/examples/streams/PageViewRegionExampleDriver.java","contentType":"file"},{"name":"PageViewRegionLambdaExample.java","path":"src/main/java/io/confluent/examples/streams/PageViewRegionLambdaExample.java","contentType":"file"},{"name":"SecureKafkaStreamsExample.java","path":"src/main/java/io/confluent/examples/streams/SecureKafkaStreamsExample.java","contentType":"file"},{"name":"SessionWindowsExample.java","path":"src/main/java/io/confluent/examples/streams/SessionWindowsExample.java","contentType":"file"},{"name":"SessionWindowsExampleDriver.java","path":"src/main/java/io/confluent/examples/streams/SessionWindowsExampleDriver.java","contentType":"file"},{"name":"SumLambdaExample.java","path":"src/main/java/io/confluent/examples/streams/SumLambdaExample.java","contentType":"file"},{"name":"SumLambdaExampleDriver.java","path":"src/main/java/io/confluent/examples/streams/SumLambdaExampleDriver.java","contentType":"file"},{"name":"TopArticlesExampleDriver.java","path":"src/main/java/io/confluent/examples/streams/TopArticlesExampleDriver.java","contentType":"file"},{"name":"TopArticlesLambdaExample.java","path":"src/main/java/io/confluent/examples/streams/TopArticlesLambdaExample.java","contentType":"file"},{"name":"UserRegionLambdaExample.java","path":"src/main/java/io/confluent/examples/streams/UserRegionLambdaExample.java","contentType":"file"},{"name":"WikipediaFeedAvroExample.java","path":"src/main/java/io/confluent/examples/streams/WikipediaFeedAvroExample.java","contentType":"file"},{"name":"WikipediaFeedAvroExampleDriver.java","path":"src/main/java/io/confluent/examples/streams/WikipediaFeedAvroExampleDriver.java","contentType":"file"},{"name":"WikipediaFeedAvroLambdaExample.java","path":"src/main/java/io/confluent/examples/streams/WikipediaFeedAvroLambdaExample.java","contentType":"file"},{"name":"WordCountLambdaExample.java","path":"src/main/java/io/confluent/examples/streams/WordCountLambdaExample.java","contentType":"file"}],"totalCount":23},"src/main/java/io/confluent/examples":{"items":[{"name":"streams","path":"src/main/java/io/confluent/examples/streams","contentType":"directory"}],"totalCount":1},"src/main/java/io/confluent":{"items":[{"name":"examples","path":"src/main/java/io/confluent/examples","contentType":"directory"}],"totalCount":1},"src/main/java/io":{"items":[{"name":"confluent","path":"src/main/java/io/confluent","contentType":"directory"}],"totalCount":1},"src/main/java":{"items":[{"name":"io","path":"src/main/java/io","contentType":"directory"}],"totalCount":1},"src/main":{"items":[{"name":"java","path":"src/main/java","contentType":"directory"},{"name":"resources","path":"src/main/resources","contentType":"directory"},{"name":"scala","path":"src/main/scala","contentType":"directory"}],"totalCount":3},"src":{"items":[{"name":"assembly","path":"src/assembly","contentType":"directory"},{"name":"main","path":"src/main","contentType":"directory"},{"name":"test","path":"src/test","contentType":"directory"}],"totalCount":3},"":{"items":[{"name":"docs","path":"docs","contentType":"directory"},{"name":"include","path":"include","contentType":"directory"},{"name":"src","path":"src","contentType":"directory"},{"name":"test","path":"test","contentType":"directory"},{"name":".gitignore","path":".gitignore","contentType":"file"},{"name":"Dockerfile","path":"Dockerfile","contentType":"file"},{"name":"Jenkinsfile","path":"Jenkinsfile","contentType":"file"},{"name":"LICENSE","path":"LICENSE","contentType":"file"},{"name":"NOTICE","path":"NOTICE","contentType":"file"},{"name":"README.md","path":"README.md","contentType":"file"},{"name":"checkstyle.xml","path":"checkstyle.xml","contentType":"file"},{"name":"docker-compose.yml","path":"docker-compose.yml","contentType":"file"},{"name":"pom.xml","path":"pom.xml","contentType":"file"},{"name":"requirements.txt","path":"requirements.txt","contentType":"file"},{"name":"setup.py","path":"setup.py","contentType":"file"},{"name":"tox.ini","path":"tox.ini","contentType":"file"}],"totalCount":16}},"fileTreeProcessingTime":41.001915,"foldersToFetch":[],"repo":{"id":100753606,"defaultBranch":"master","name":"kafka-streams-examples","ownerLogin":"confluentinc","currentUserCanPush":false,"isFork":false,"isEmpty":false,"createdAt":"2017-08-18T22:07:36.000Z","ownerAvatar":"https://avatars.githubusercontent.com/u/9439498?v=4","public":true,"private":false,"isOrgOwned":true},"symbolsExpanded":false,"treeExpanded":true,"refInfo":{"name":"5.0.0-post","listCacheKey":"v0:1710810393.0","canEdit":false,"refType":"branch","currentOid":"014e9bd70661614b1dc640e0b985aaad9f3059ca"},"path":"src/main/java/io/confluent/examples/streams/interactivequeries/kafkamusic/KafkaMusicExample.java","currentUser":null,"blob":{"rawLines":["/*"," * Copyright Confluent Inc."," *"," * Licensed under the Apache License, Version 2.0 (the \"License\");"," * you may not use this file except in compliance with the License."," * You may obtain a copy of the License at"," *"," * http://www.apache.org/licenses/LICENSE-2.0"," *"," * Unless required by applicable law or agreed to in writing, software"," * distributed under the License is distributed on an \"AS IS\" BASIS,"," * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied."," * See the License for the specific language governing permissions and"," * limitations under the License."," */","package io.confluent.examples.streams.interactivequeries.kafkamusic;","","import io.confluent.examples.streams.avro.PlayEvent;","import io.confluent.examples.streams.avro.Song;","import io.confluent.examples.streams.avro.SongPlayCount;","import io.confluent.kafka.serializers.AbstractKafkaAvroSerDeConfig;","import io.confluent.kafka.streams.serdes.avro.SpecificAvroSerde;","import org.apache.kafka.clients.consumer.ConsumerConfig;","import org.apache.kafka.common.serialization.Deserializer;","import org.apache.kafka.common.serialization.Serde;","import org.apache.kafka.common.serialization.Serdes;","import org.apache.kafka.common.serialization.Serializer;","import org.apache.kafka.common.utils.Bytes;","import org.apache.kafka.streams.KafkaStreams;","import org.apache.kafka.streams.KeyValue;","import org.apache.kafka.streams.StreamsBuilder;","import org.apache.kafka.streams.StreamsConfig;","import org.apache.kafka.streams.kstream.Consumed;","import org.apache.kafka.streams.kstream.Joined;","import org.apache.kafka.streams.kstream.KStream;","import org.apache.kafka.streams.kstream.KTable;","import org.apache.kafka.streams.kstream.Materialized;","import org.apache.kafka.streams.kstream.Serialized;","import org.apache.kafka.streams.state.HostInfo;","import org.apache.kafka.streams.state.KeyValueStore;","","import java.io.ByteArrayInputStream;","import java.io.ByteArrayOutputStream;","import java.io.DataInputStream;","import java.io.DataOutputStream;","import java.io.IOException;","import java.util.Collections;","import java.util.HashMap;","import java.util.Iterator;","import java.util.Map;","import java.util.Properties;","import java.util.TreeSet;","","/**"," * Demonstrates how to locate and query state stores (Interactive Queries)."," *"," * This application continuously computes the latest Top 5 music charts based on song play events"," * collected in real-time in a Kafka topic. This charts data is maintained in a continuously updated"," * state store that can be queried interactively via a REST API."," *"," * Note: This example uses Java 8 functionality and thus works with Java 8+ only. But of course you"," * can use the Interactive Queries feature of Kafka Streams also with Java 7."," *"," * The topology in this example is modelled on a (very) simple streaming music service. It has 2"," * input topics: song-feed and play-events."," *"," * The song-feed topic contains all of the songs available in the streaming service and is read"," * as a KTable with all songs being stored in the all-songs state store."," *"," * The play-events topic is a feed of song plays. We filter the play events to only accept events"," * where the duration is >= 30 seconds. We then map the stream so that it is keyed by songId."," *"," * Now that both streams are keyed the same we can join the play events with the songs, group by"," * the song and count them into a KTable, songPlayCounts, and a state store, song-play-count,"," * to keep track of the number of times each song has been played."," *"," * Next, we group the songPlayCounts KTable by genre and aggregate into another KTable with the"," * state store, top-five-songs-by-genre, to track the top five songs by genre. Subsequently, we"," * group the same songPlayCounts KTable such that all song plays end up in the same partition. We"," * use this to aggregate the overall top five songs played into the state store, top-five."," *"," * HOW TO RUN THIS EXAMPLE"," *"," * 1) Start Zookeeper, Kafka, and Confluent Schema Registry. Please refer to QuickStart."," *"," * 2) Create the input and output topics used by this example."," *"," *
"," * {@code"," * $ bin/kafka-topics --create --topic play-events \\"," * --zookeeper localhost:2181 --partitions 4 --replication-factor 1"," * $ bin/kafka-topics --create --topic song-feed \\"," * --zookeeper localhost:2181 --partitions 4 --replication-factor 1"," *"," * }"," *"," *"," * Note: The above commands are for the Confluent Platform. For Apache Kafka it should be"," * `bin/kafka-topics.sh ...`."," *"," *"," * 3) Start two instances of this example application either in your IDE or on the command"," * line."," *"," * If via the command line please refer to Packaging."," *"," * Once packaged you can then start the first instance of the application (on port 7070):"," *"," *
"," * {@code"," * $ java -cp target/kafka-streams-examples-5.0.0-SNAPSHOT-standalone.jar \\"," * io.confluent.examples.streams.interactivequeries.kafkamusic.KafkaMusicExample 7070"," * }"," *"," *"," * Here, `7070` sets the port for the REST endpoint that will be used by this application instance."," *"," * Then, in a separate terminal, run the second instance of this application (on port 7071):"," *"," *
"," * {@code"," * $ java -cp target/kafka-streams-examples-5.0.0-SNAPSHOT-standalone.jar \\"," * io.confluent.examples.streams.interactivequeries.kafkamusic.KafkaMusicExample 7071"," * }"," *"," *"," *"," * 4) Write some input data to the source topics (e.g. via {@link KafkaMusicExampleDriver}). The"," * already running example application (step 3) will automatically process this input data"," *"," *"," * 5) Use your browser to hit the REST endpoint of the app instance you started in step 3 to query"," * the state managed by this application. Note: If you are running multiple app instances, you can"," * query them arbitrarily -- if an app instance cannot satisfy a query itself, it will fetch the"," * results from the other instances."," *"," * For example:"," *"," *
"," * {@code"," * # List all running instances of this application"," * http://localhost:7070/kafka-music/instances"," *"," * # List app instances that currently manage (parts of) state store \"song-play-count\""," * http://localhost:7070/kafka-music/instances/song-play-count"," *"," * # Get the latest top five for the genre \"punk\""," * http://localhost:7070/kafka-music/charts/genre/punk"," *"," * # Get the latest top five across all genres"," * http://localhost:7070/kafka-music/charts/top-five"," * }"," *"," *"," * Note: that the REST functionality is NOT part of Kafka Streams or its API. For demonstration"," * purposes of this example application, we decided to go with a simple, custom-built REST API that"," * uses the Interactive Queries API of Kafka Streams behind the scenes to expose the state stores of"," * this application via REST."," *"," * 6) Once you're done with your experiments, you can stop this example via `Ctrl-C`. If needed,"," * also stop the Schema Registry (`Ctrl-C`), the Kafka broker (`Ctrl-C`), and only then stop the ZooKeeper instance"," * (`Ctrl-C`)."," *"," * If you like you can run multiple instances of this example by passing in a different port. You"," * can then experiment with seeing how keys map to different instances etc."," */","","public class KafkaMusicExample {",""," private static final Long MIN_CHARTABLE_DURATION = 30 * 1000L;"," private static final String SONG_PLAY_COUNT_STORE = \"song-play-count\";"," static final String PLAY_EVENTS = \"play-events\";"," static final String ALL_SONGS = \"all-songs\";"," static final String SONG_FEED = \"song-feed\";"," static final String TOP_FIVE_SONGS_BY_GENRE_STORE = \"top-five-songs-by-genre\";"," static final String TOP_FIVE_SONGS_STORE = \"top-five-songs\";"," static final String TOP_FIVE_KEY = \"all\";",""," private static final String DEFAULT_REST_ENDPOINT_HOSTNAME = \"localhost\";"," private static final String DEFAULT_BOOTSTRAP_SERVERS = \"localhost:9092\";"," private static final String DEFAULT_SCHEMA_REGISTRY_URL = \"http://localhost:8081\";",""," public static void main(final String[] args) throws Exception {"," if (args.length == 0 || args.length > 4) {"," throw new IllegalArgumentException(\"usage: ...