├── .gitignore ├── README.md ├── gettingstarted.gif ├── pom.xml ├── run-job.sh ├── run-query-repl.sh └── src └── main ├── java └── com │ └── dataartisans │ └── queryablestatedemo │ ├── BumpEvent.java │ ├── BumpEventGeneratorSource.java │ ├── EventCountClient.java │ ├── EventCountJob.java │ └── QueryClientHelper.java └── resources └── log4j.properties /.gitignore: -------------------------------------------------------------------------------- 1 | .cache 2 | scalastyle-output.xml 3 | .classpath 4 | .idea 5 | .metadata 6 | .settings 7 | .project 8 | .version.properties 9 | filter.properties 10 | logs.zip 11 | target 12 | tmp 13 | *.class 14 | *.iml 15 | *.swp 16 | *.jar 17 | *.log 18 | .DS_Store 19 | build-target 20 | flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/generated/ 21 | flink-runtime-web/web-dashboard/assets/fonts/ 22 | flink-runtime-web/web-dashboard/node_modules/ 23 | flink-runtime-web/web-dashboard/bower_components/ 24 | atlassian-ide-plugin.xml 25 | out/ 26 | /docs/api 27 | /docs/content 28 | /docs/.bundle 29 | /docs/.rubydeps 30 | /docs/ruby2/.bundle 31 | /docs/ruby2/.rubydeps 32 | /docs/.jekyll-metadata 33 | *.ipr 34 | *.iws 35 | tools/flink 36 | tools/flink-* 37 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | ## Apache Flink Queryable State Demo 2 | 3 | This repository contains a short demo of Apache Flink's Queryable State feature. You can check out the [Flink documentation](https://ci.apache.org/projects/flink/flink-docs-release-1.2/dev/stream/queryable_state.html) for [more details about Queryable State](https://ci.apache.org/projects/flink/flink-docs-release-1.2/dev/stream/queryable_state.html). 4 | 5 | ### Getting Started 6 | 7 | 0. **Requirements** 8 | 9 | - Git 10 | - Java 8 11 | - Maven 3 12 | 13 | 1. **Clone this repository.** 14 | 15 | ```bash 16 | $ git clone https://github.com/dataArtisans/flink-queryable_state_demo.git 17 | ``` 18 | 19 | 2. **Run the demo job.** 20 | 21 | ```bash 22 | % ./run-job.sh 23 | [info] Building demo JAR... this can take a few seconds. 24 | [info] Done. Demo JAR created in target/queryablestatedemo-1.0-SNAPSHOT.jar. 25 | [info] Executing EventCountJob from queryablestatedemo-1.0-SNAPSHOT.jar (exit via Control+C) 26 | [info] Job ID: 2488a115d832013edbbd5a6599e49e45 27 | 28 | Generating 454435 elements per second 29 | Generating 880747 elements per second 30 | Generating 903855 elements per second 31 | ``` 32 | 33 | 3. **Run the demo query client.** 34 | 35 | ```bash 36 | ./run-query-repl.sh 2488a115d832013edbbd5a6599e49e45 37 | [info] Querying job with ID '2488a115d832013edbbd5a6599e49e45' 38 | [info] Executing EventCountClient from queryablestatedemo-1.0-SNAPSHOT.jar (exit via Control+C) 39 | 40 | Using JobManager localhost:6124 41 | Enter a key to query. 42 | 43 | The EventCountJob itemCounts state instance has String keys that are three characters long and alphanumeric, e.g. 'AP2' or 'LOL'. 44 | 45 | $ ABC 46 | [info] Querying key 'abc' 47 | 446 (query took 99 ms) 48 | $ ABC 49 | [info] Querying key 'abc' 50 | 604 (query took 2 ms) 51 | $ ABC 52 | [info] Querying key 'abc' 53 | 631 (query took 1 ms) 54 | $ ele 55 | [info] Querying key 'ele' 56 | 834 (query took 2 ms) 57 | $ notexistingkey 58 | [info] Querying key 'notexistingkey' 59 | Unknown key notexistingkey (query took 11 ms) 60 | ``` 61 | 62 | ![Getting Started](gettingstarted.gif) 63 | 64 | ### Code 65 | 66 | The code for this demo is found in the `src/main` directory under the package `com.dataartisans.queryablestatedemo`. 67 | 68 | The two main classes are `EventCountJob` and `EventCountClient`. 69 | 70 | #### EventCountJob 71 | 72 | The `EventCountJob` is a simple job that generates random `BumpEvent` instances and counts how many counts happen per bumped item. The bump events model users interacting with items on a website and *liking* them. 73 | 74 | ```java 75 | public class BumpEvent { 76 | 77 | // ID of the user bumping the item 78 | private final int userId; 79 | 80 | // Item being bumped 81 | private final String itemId; 82 | 83 | } 84 | ``` 85 | 86 | The item IDs in our demo are *three character alphanumeric Strings* like `ABC` or `1A3`. The stream of `BumpEvent` instances is keyed by the `itemId` and the count is increased for each item. 87 | 88 | ```java 89 | // Increment the count for each event (keyed on itemId) 90 | FoldingStateDescriptor countingState = new FoldingStateDescriptor<>( 91 | "itemCounts", 92 | 0L, // Initial value is 0 93 | (acc, event) -> acc + 1L, // Increment for each event 94 | Long.class); 95 | 96 | bumps.keyBy(BumpEvent::getItemId).asQueryableState("itemCounts", countingState); 97 | ``` 98 | 99 | The call to `asQueryableState(State, StateDescriptor)` exposes a queryable state stream that is identified via the name `itemCounts`. The `countingState` descriptor specifies a state instance that increments the count for each incoming event. Because we use a keyed stream here (the `keyBy`) each count is automatically scoped to the item ID. You can find more information about [Flink's state abstraction](https://ci.apache.org/projects/flink/flink-docs-release-1.2/dev/stream/state.html) in the documentation. 100 | 101 | As soon as the job is running, the `itemCounts` state instance is available for queries. As part of this demo, you can use the included **`run-job.sh` script** which takes care of building the job and executing it. Make sure to note down the displayed JobID as you will need it for the query client below. 102 | 103 | ##### Exposing State for External Queries 104 | 105 | As of Flink 1.2, we have two options for exposing state for queries. 106 | 107 | 1. **Queryable State Stream**: The above variant is a shorthand that we call a *queryable state stream*. The `asQueryableState` call is overloaded to allow different state variants like `ValueState` or `ReducingState`, allowing you to increment the state stream in different ways. 108 | 109 | 2. **StateDescripor**. The other variant is to call `setQueryable("itemCounts")` when creating a `StateDescriptor`. This is more flexible as you can expose any keyed state instance you are working with for queries. The following example is achieves the same result as the queryable state stream from above: 110 | 111 | ```java 112 | static class EventCounter extends RichFlatMapFunction { 113 | 114 | // Flink's managed state for keeping track of the counts. The state is 115 | // automatically scoped to the item key and updates happen by key. 116 | private ReducingState runningCount; 117 | 118 | @Override 119 | public void open(Configuration config) throws Exception { 120 | // Reducing state that keeps a sum 121 | ReducingStateDescriptor stateDescriptor = new ReducingStateDescriptor<>( 122 | "itemCounts", (a, b) -> a + b, Long.class); 123 | 124 | // Mark state as queryable. This is how we expose the state instance 125 | // for external queries. 126 | stateDescriptor.setQueryable("itemCounts"); 127 | 128 | this.runningCount = getRuntimeContext().getReducingState(stateDescriptor); 129 | } 130 | 131 | @Override 132 | public void flatMap(T event, Collector collector) throws Exception { 133 | // Increment the count for the current key 134 | runningCount.add(1L); 135 | } 136 | } 137 | ``` 138 | 139 | #### EventCountClient 140 | 141 | The `EventCountClient` of this demo is a very simple REPL that queries the `itemCounts` state instance. 142 | 143 | The `QueryClientHelper` takes care of setting up Flink's low level `QueryableStateClient` that was released with Flink 1.2.0. The `QueryableStateClient` is a fully asynchronous client that takes care of 144 | 145 | 1. **Location lookup**: communicate with the JobManager and look up the location of a queried key among the available TaskManager instances, and 146 | 2. **Network communication**: submitting the query via the network to a specific TaskManager. 147 | 148 | This means that the client takes care of all communication with the Flink application. As a user, you only have to provide the following information: 149 | 150 | - **JobID**. The ID of the job to query (displayed by the demo job or found in the web UI) 151 | - **Types**. The types of the queried keys and returned values (our keys are Strings and the returned counts are Longs, using the `StringSerializer` and `LongSerializer` respectively) 152 | 153 | As part of this demo, the `EventCountClient` takes care of setting up the types and submitting the queries. You only have to provide the JobID. 154 | 155 | You can use the included **`run-query-repl.sh` script** which takes care of building the client and executing it. It has the jobId of the running job as a required argument. 156 | 157 | If you want to have a look at how the `QueryableStateClient` is set up as part of this demo, check out the [QueryClientHelper](https://github.com/dataArtisans/flink-queryable_state_demo/blob/master/src/main/java/com/dataartisans/queryablestatedemo/QueryClientHelper.java) class. 158 | 159 | The main query method is `Optional queryState(String name, K key)`. The name for the demo is `itemCounts` and the keys are three character alphanumeric Strings like `AB1`. If a key is not available, the returned Optional will be empty. 160 | -------------------------------------------------------------------------------- /gettingstarted.gif: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dataArtisans/flink-queryable_state_demo/511e284528ed2bb9ce31343b07095fc93045d7d6/gettingstarted.gif -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 19 | 22 | 4.0.0 23 | 24 | com.dataartisans 25 | queryablestatedemo 26 | 1.0-SNAPSHOT 27 | jar 28 | 29 | Flink Quickstart Job 30 | http://www.myorganization.org 31 | 32 | 33 | UTF-8 34 | 1.2.0 35 | 1.7.7 36 | 1.2.17 37 | 38 | 39 | 40 | 41 | apache.snapshots 42 | Apache Development Snapshot Repository 43 | https://repository.apache.org/content/repositories/snapshots/ 44 | 45 | false 46 | 47 | 48 | true 49 | 50 | 51 | 52 | 53 | 76 | 77 | 78 | 79 | 80 | org.apache.flink 81 | flink-java 82 | ${flink.version} 83 | 84 | 85 | org.apache.flink 86 | flink-streaming-java_2.10 87 | ${flink.version} 88 | 89 | 90 | org.apache.flink 91 | flink-clients_2.10 92 | ${flink.version} 93 | 94 | 95 | jline 96 | jline 97 | 2.14.2 98 | 99 | 100 | org.slf4j 101 | slf4j-log4j12 102 | ${slf4j.version} 103 | 104 | 105 | log4j 106 | log4j 107 | ${log4j.version} 108 | 109 | 110 | 111 | 112 | 113 | 114 | build-jar 115 | 116 | false 117 | 118 | 119 | 120 | org.apache.flink 121 | flink-java 122 | ${flink.version} 123 | provided 124 | 125 | 126 | org.apache.flink 127 | flink-streaming-java_2.10 128 | ${flink.version} 129 | provided 130 | 131 | 132 | org.apache.flink 133 | flink-clients_2.10 134 | ${flink.version} 135 | provided 136 | 137 | 138 | org.slf4j 139 | slf4j-log4j12 140 | ${slf4j.version} 141 | provided 142 | 143 | 144 | log4j 145 | log4j 146 | ${log4j.version} 147 | provided 148 | 149 | 150 | 151 | 152 | 153 | 154 | 155 | org.apache.maven.plugins 156 | maven-shade-plugin 157 | 2.4.1 158 | 159 | 160 | package 161 | 162 | shade 163 | 164 | 165 | 166 | 167 | 168 | 169 | 170 | 171 | 172 | 173 | 174 | 175 | 176 | 177 | 178 | 179 | 182 | 183 | org.apache.maven.plugins 184 | maven-shade-plugin 185 | 2.4.1 186 | 187 | 188 | 189 | package 190 | 191 | shade 192 | 193 | 194 | 195 | 196 | 198 | *:* 199 | 200 | META-INF/*.SF 201 | META-INF/*.DSA 202 | META-INF/*.RSA 203 | 204 | 205 | 206 | 208 | 209 | 211 | reference.conf 212 | 213 | 214 | false 215 | 216 | 217 | 218 | 219 | 220 | 221 | org.apache.maven.plugins 222 | maven-compiler-plugin 223 | 3.1 224 | 225 | 1.8 226 | 1.8 227 | 228 | 229 | 230 | 231 | 232 | 233 | 234 | maven-compiler-plugin 235 | 236 | 1.8 237 | 1.8 238 | jdt 239 | 240 | 241 | 242 | org.eclipse.tycho 243 | tycho-compiler-jdt 244 | 0.21.0 245 | 246 | 247 | 248 | 249 | 250 | org.eclipse.m2e 251 | lifecycle-mapping 252 | 1.0.0 253 | 254 | 255 | 256 | 257 | 258 | org.apache.maven.plugins 259 | maven-assembly-plugin 260 | [2.4,) 261 | 262 | single 263 | 264 | 265 | 266 | 267 | 268 | 269 | 270 | 271 | org.apache.maven.plugins 272 | maven-compiler-plugin 273 | [3.1,) 274 | 275 | testCompile 276 | compile 277 | 278 | 279 | 280 | 281 | 282 | 283 | 284 | 285 | 286 | 287 | 288 | 289 | 290 | 291 | -------------------------------------------------------------------------------- /run-job.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | base="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" 4 | target="$base/target/queryablestatedemo-1.0-SNAPSHOT.jar" 5 | 6 | if [ ! -f $target ]; then 7 | echo "[info] Building demo JAR... this can take a few seconds." 8 | mvn clean package -DskipTests &> /dev/null 9 | echo "[info] Done. Demo JAR created in $target." 10 | fi 11 | 12 | echo "[info] Executing EventCountJob from queryablestatedemo-1.0-SNAPSHOT.jar (exit via Control+C)" 13 | java -cp $target com.dataartisans.queryablestatedemo.EventCountJob 14 | -------------------------------------------------------------------------------- /run-query-repl.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | base="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" 4 | target="$base/target/queryablestatedemo-1.0-SNAPSHOT.jar" 5 | 6 | if [ ! -f $target ]; then 7 | echo "[info] Building demo JAR... this can take a few seconds" 8 | mvn clean package -DskipTests &> /dev/null 9 | echo "[info] Done. Demo JAR created in $target" 10 | echo "" 11 | fi 12 | 13 | if [ -z $1 ]; then 14 | echo "Usage: run-query-repl.sh [jobManagerHost] [jobManagerPort]"; 15 | exit 1 16 | else 17 | jobId=$1 18 | shift 19 | echo "[info] Querying job with ID '$jobId'"; 20 | fi 21 | 22 | echo "[info] Executing EventCountClient from queryablestatedemo-1.0-SNAPSHOT.jar (exit via Control+C)" 23 | echo "" 24 | java -cp $target com.dataartisans.queryablestatedemo.EventCountClient $jobId 25 | -------------------------------------------------------------------------------- /src/main/java/com/dataartisans/queryablestatedemo/BumpEvent.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.dataartisans.queryablestatedemo; 20 | 21 | /** 22 | * A bump event. 23 | */ 24 | public class BumpEvent { 25 | 26 | /** 27 | * ID of the user that triggered the bump. 28 | */ 29 | private final int userId; 30 | 31 | /** 32 | * ID of the item that was bumped (article, post, etc.). 33 | */ 34 | private final String itemId; 35 | 36 | public BumpEvent(int userId, String itemId) { 37 | this.userId = userId; 38 | this.itemId = itemId; 39 | } 40 | 41 | /** 42 | * Returns the ID of the user that triggered the bump. 43 | * 44 | * @return ID of the user that triggered the bump. 45 | */ 46 | public int getUserId() { 47 | return userId; 48 | } 49 | 50 | /** 51 | * Returns the ID of the content that was bumped. 52 | * 53 | * @return ID of the content that was bumped. 54 | */ 55 | public String getItemId() { 56 | return itemId; 57 | } 58 | 59 | @Override 60 | public String toString() { 61 | return "BumpEvent{" + 62 | "userId=" + userId + 63 | ", itemId=" + itemId + 64 | '}'; 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /src/main/java/com/dataartisans/queryablestatedemo/BumpEventGeneratorSource.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.dataartisans.queryablestatedemo; 20 | 21 | import java.util.Random; 22 | import java.util.concurrent.atomic.AtomicLong; 23 | import org.apache.commons.lang3.RandomStringUtils; 24 | import org.apache.flink.streaming.api.functions.source.RichSourceFunction; 25 | 26 | /** 27 | * A (non-parallel) {@link BumpEvent} generator source. 28 | */ 29 | class BumpEventGeneratorSource extends RichSourceFunction { 30 | 31 | private static final long serialVersionUID = 244478060020939187L; 32 | 33 | /** 34 | * Number of alpha numeric characters of the items. 35 | */ 36 | private static final int ITEM_ID_NUM_CHARS = 3; 37 | 38 | /** 39 | * Flag indicating whether we should print the throughput. If true, the {@link ThroughputLogger} 40 | * thread is started. 41 | */ 42 | private final boolean printThroughput; 43 | 44 | /** 45 | * Flag indicating whether we are still running. 46 | */ 47 | private volatile boolean running = true; 48 | 49 | BumpEventGeneratorSource(boolean printThroughput) { 50 | this.printThroughput = printThroughput; 51 | } 52 | 53 | @Override 54 | public void run(SourceContext sourceContext) throws Exception { 55 | final Random rand = new Random(); 56 | final AtomicLong count = new AtomicLong(); 57 | 58 | Thread throughputLogger = null; 59 | if (printThroughput) { 60 | throughputLogger = new Thread(new ThroughputLogger(count), "ThroughputLogger"); 61 | throughputLogger.start(); 62 | } 63 | 64 | try { 65 | while (running) { 66 | // Generate random events 67 | final int userId = rand.nextInt(Integer.MAX_VALUE); 68 | 69 | final String itemCase = RandomStringUtils 70 | .randomAlphanumeric(ITEM_ID_NUM_CHARS).toLowerCase(); 71 | 72 | synchronized (sourceContext.getCheckpointLock()) { 73 | sourceContext.collect(new BumpEvent(userId, itemCase)); 74 | } 75 | 76 | // Increment count for throughput logger 77 | count.incrementAndGet(); 78 | 79 | Thread.yield(); 80 | } 81 | } finally { 82 | if (throughputLogger != null) { 83 | throughputLogger.interrupt(); 84 | throughputLogger.join(); 85 | } 86 | } 87 | } 88 | 89 | @Override 90 | public void cancel() { 91 | running = false; 92 | } 93 | 94 | // -------------------------------------------------------------------------- 95 | 96 | private static class ThroughputLogger implements Runnable { 97 | 98 | private final AtomicLong count; 99 | 100 | ThroughputLogger(AtomicLong count) { 101 | this.count = count; 102 | } 103 | 104 | @Override 105 | public void run() { 106 | long lastCount = 0L; 107 | long lastTimestamp = System.currentTimeMillis(); 108 | 109 | while (!Thread.interrupted()) { 110 | try { 111 | Thread.sleep(1000); 112 | } catch (InterruptedException e) { 113 | return; 114 | } 115 | 116 | final long ts = System.currentTimeMillis(); 117 | 118 | final long currCount = count.get(); 119 | 120 | final double factor = (ts - lastTimestamp) / 1000.0; 121 | final int perSec = (int) ((currCount - lastCount) / factor); 122 | 123 | lastTimestamp = ts; 124 | lastCount = currCount; 125 | 126 | System.out.println(String.format("Generating %d elements per second", perSec)); 127 | } 128 | } 129 | } 130 | 131 | } 132 | -------------------------------------------------------------------------------- /src/main/java/com/dataartisans/queryablestatedemo/EventCountClient.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.dataartisans.queryablestatedemo; 20 | 21 | import java.io.PrintWriter; 22 | import java.util.Optional; 23 | import jline.console.ConsoleReader; 24 | import org.apache.flink.api.common.JobID; 25 | import org.apache.flink.api.common.time.Time; 26 | import org.apache.flink.api.common.typeutils.base.LongSerializer; 27 | import org.apache.flink.api.common.typeutils.base.StringSerializer; 28 | 29 | public class EventCountClient { 30 | 31 | public static void main(String[] args) throws Exception { 32 | if (args.length == 0) { 33 | throw new IllegalArgumentException("Missing required job ID argument. " 34 | + "Usage: ./EventCountClient [jobManagerHost] [jobManagerPort]"); 35 | } 36 | String jobIdParam = args[0]; 37 | 38 | // Configuration 39 | final String jobManagerHost = args.length > 1 ? args[1] : "localhost"; 40 | final int jobManagerPort = args.length > 1 ? Integer.parseInt(args[1]) : 6124; 41 | 42 | System.out.println("Using JobManager " + jobManagerHost + ":" + jobManagerPort); 43 | 44 | final JobID jobId = JobID.fromHexString(jobIdParam); 45 | 46 | final StringSerializer keySerializer = StringSerializer.INSTANCE; 47 | final LongSerializer valueSerializer = LongSerializer.INSTANCE; 48 | 49 | final Time queryTimeout = Time.seconds(5); 50 | 51 | try ( 52 | // This helper is for convenience and not part of Flink 53 | QueryClientHelper client = new QueryClientHelper<>( 54 | jobManagerHost, 55 | jobManagerPort, 56 | jobId, 57 | keySerializer, 58 | valueSerializer, 59 | queryTimeout)) { 60 | 61 | printUsage(); 62 | 63 | ConsoleReader reader = new ConsoleReader(); 64 | reader.setPrompt("$ "); 65 | 66 | PrintWriter out = new PrintWriter(reader.getOutput()); 67 | 68 | String line; 69 | while ((line = reader.readLine()) != null) { 70 | String key = line.toLowerCase().trim(); 71 | out.printf("[info] Querying key '%s'\n", key); 72 | 73 | try { 74 | long start = System.currentTimeMillis(); 75 | Optional count = client.queryState(EventCountJob.ITEM_COUNTS, key); 76 | long end = System.currentTimeMillis(); 77 | 78 | long duration = Math.max(0, end - start); 79 | 80 | if (count.isPresent()) { 81 | out.printf("%d (query took %d ms)\n", count.get(), duration); 82 | } else { 83 | out.printf("Unknown key %s (query took %d ms)\n", key, duration); 84 | } 85 | } catch (Exception e) { 86 | out.println("Query failed because of the following Exception:"); 87 | e.printStackTrace(out); 88 | } 89 | } 90 | } 91 | } 92 | 93 | private static void printUsage() { 94 | System.out.println("Enter a key to query."); 95 | System.out.println(); 96 | System.out.println("The EventCountJob " + EventCountJob.ITEM_COUNTS + " state instance " 97 | + "has String keys that are three characters long and alphanumeric, e.g. 'AP2' or 'LOL'."); 98 | System.out.println(); 99 | } 100 | 101 | } 102 | -------------------------------------------------------------------------------- /src/main/java/com/dataartisans/queryablestatedemo/EventCountJob.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.dataartisans.queryablestatedemo; 20 | 21 | import org.apache.flink.api.common.state.FoldingStateDescriptor; 22 | import org.apache.flink.api.java.utils.ParameterTool; 23 | import org.apache.flink.configuration.ConfigConstants; 24 | import org.apache.flink.configuration.Configuration; 25 | import org.apache.flink.configuration.QueryableStateOptions; 26 | import org.apache.flink.runtime.jobgraph.JobGraph; 27 | import org.apache.flink.runtime.minicluster.FlinkMiniCluster; 28 | import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; 29 | import org.apache.flink.streaming.api.datastream.DataStream; 30 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 31 | 32 | /** 33 | * A job generating {@link BumpEvent} instances and counting them. 34 | * 35 | *

The generated events are keyed by their item ID and then counted. The 36 | * count is maintained via Flink's managed state as exposed for external 37 | * queries. 38 | * 39 | *

Checkout the {@link EventCountClient} for the querying part. 40 | */ 41 | public class EventCountJob { 42 | 43 | /** 44 | * External name for the state instance. The count can be queried under this name. 45 | */ 46 | public final static String ITEM_COUNTS = "itemCounts"; 47 | 48 | public static void main(String[] args) throws Exception { 49 | ParameterTool params = ParameterTool.fromArgs(args); 50 | final boolean printThroughput = params.getBoolean("printThroughput", true); 51 | final int port = params.getInt("port", 6124); 52 | final int parallelism = params.getInt("parallelism", 4); 53 | 54 | // We use a mini cluster here for sake of simplicity, because I don't want 55 | // to require a Flink installation to run this demo. Everything should be 56 | // contained in this JAR. 57 | 58 | Configuration config = new Configuration(); 59 | config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, port); 60 | config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); 61 | config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, parallelism); 62 | // In a non MiniCluster setup queryable state is enabled by default. 63 | config.setBoolean(QueryableStateOptions.SERVER_ENABLE, true); 64 | 65 | FlinkMiniCluster flinkCluster = new LocalFlinkMiniCluster(config, false); 66 | try { 67 | flinkCluster.start(true); 68 | 69 | StreamExecutionEnvironment env = StreamExecutionEnvironment 70 | .createRemoteEnvironment("localhost", port, parallelism); 71 | 72 | DataStream bumps = env 73 | .addSource(new BumpEventGeneratorSource(printThroughput)); 74 | 75 | // Increment the count for each event (keyed on itemId) 76 | FoldingStateDescriptor countingState = new FoldingStateDescriptor<>( 77 | ITEM_COUNTS, 78 | 0L, // Initial value is 0 79 | (acc, event) -> acc + 1L, // Increment for each event 80 | Long.class); 81 | 82 | bumps.keyBy(BumpEvent::getItemId) 83 | .asQueryableState(ITEM_COUNTS, countingState); 84 | 85 | JobGraph jobGraph = env.getStreamGraph().getJobGraph(); 86 | 87 | System.out.println("[info] Job ID: " + jobGraph.getJobID()); 88 | System.out.println(); 89 | 90 | flinkCluster.submitJobAndWait(jobGraph, false); 91 | } finally { 92 | flinkCluster.shutdown(); 93 | } 94 | } 95 | 96 | } 97 | -------------------------------------------------------------------------------- /src/main/java/com/dataartisans/queryablestatedemo/QueryClientHelper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.dataartisans.queryablestatedemo; 20 | 21 | import java.util.Optional; 22 | import java.util.concurrent.TimeUnit; 23 | import java.util.concurrent.TimeoutException; 24 | import org.apache.flink.api.common.JobID; 25 | import org.apache.flink.api.common.time.Time; 26 | import org.apache.flink.api.common.typeutils.TypeSerializer; 27 | import org.apache.flink.configuration.ConfigConstants; 28 | import org.apache.flink.configuration.Configuration; 29 | import org.apache.flink.runtime.query.QueryableStateClient; 30 | import org.apache.flink.runtime.query.netty.UnknownKeyOrNamespace; 31 | import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer; 32 | import org.apache.flink.runtime.state.VoidNamespace; 33 | import org.apache.flink.runtime.state.VoidNamespaceSerializer; 34 | import org.apache.flink.runtime.util.DataInputDeserializer; 35 | import scala.concurrent.Await; 36 | import scala.concurrent.Future; 37 | import scala.concurrent.duration.FiniteDuration; 38 | 39 | /** 40 | * This is a wrapper around Flink's {@link QueryableStateClient} (as of Flink 41 | * 1.2.0) that hides the low-level type serialization details. 42 | * 43 | *

Queries are executed synchronously via {@link #queryState(String, Object)}. 44 | * 45 | * @param Type of the queried keys 46 | * @param Type of the queried values 47 | */ 48 | public class QueryClientHelper implements AutoCloseable { 49 | 50 | /** 51 | * ID of the job to query. 52 | */ 53 | private final JobID jobId; 54 | 55 | /** 56 | * Serializer for the keys. 57 | */ 58 | private final TypeSerializer keySerializer; 59 | 60 | /** 61 | * Serializer for the result values. 62 | */ 63 | private final TypeSerializer valueSerializer; 64 | 65 | /** 66 | * Timeout for each query. After this timeout, the query fails with a {@link TimeoutException}. 67 | */ 68 | private final FiniteDuration queryTimeout; 69 | 70 | /** 71 | * The wrapper low-level {@link QueryableStateClient}. 72 | */ 73 | private final QueryableStateClient client; 74 | 75 | /** 76 | * Creates the queryable state client wrapper. 77 | * 78 | * @param jobManagerHost Host for JobManager communication 79 | * @param jobManagerPort Port for JobManager communication. 80 | * @param jobId ID of the job to query. 81 | * @param keySerializer Serializer for keys. 82 | * @param valueSerializer Serializer for returned values. 83 | * @param queryTimeout Timeout for queries. 84 | * @throws Exception Thrown if creating the {@link QueryableStateClient} fails. 85 | */ 86 | QueryClientHelper( 87 | String jobManagerHost, 88 | int jobManagerPort, 89 | JobID jobId, 90 | TypeSerializer keySerializer, 91 | TypeSerializer valueSerializer, 92 | Time queryTimeout) throws Exception { 93 | 94 | this.jobId = jobId; 95 | this.keySerializer = keySerializer; 96 | this.valueSerializer = valueSerializer; 97 | this.queryTimeout = new FiniteDuration(queryTimeout.toMilliseconds(), TimeUnit.MILLISECONDS); 98 | 99 | Configuration config = new Configuration(); 100 | config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, jobManagerHost); 101 | config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort); 102 | 103 | this.client = new QueryableStateClient(config); 104 | } 105 | 106 | /** 107 | * Queries a state instance for the key. 108 | * 109 | * @param name Name of the state instance to query. This is the external name as given to {@link 110 | * org.apache.flink.api.common.state.StateDescriptor#setQueryable(String)} or {@link 111 | * org.apache.flink.streaming.api.datastream.KeyedStream#asQueryableState(String)}. 112 | * @param key The key to query 113 | * @return The returned value if it is available 114 | */ 115 | Optional queryState(String name, K key) throws Exception { 116 | if (name == null) { 117 | throw new NullPointerException("Name"); 118 | } 119 | 120 | if (key == null) { 121 | throw new NullPointerException("Key"); 122 | } 123 | 124 | // Serialize the key. The namespace is ignored as it's only relevant for 125 | // windows which are not yet exposed for queries. 126 | byte[] serializedKey = KvStateRequestSerializer.serializeKeyAndNamespace( 127 | key, 128 | keySerializer, 129 | VoidNamespace.INSTANCE, 130 | VoidNamespaceSerializer.INSTANCE); 131 | 132 | // Submit the query 133 | Future queryFuture = client.getKvState(jobId, name, key.hashCode(), serializedKey); 134 | 135 | try { 136 | // Wait for the result 137 | byte[] queryResult = Await.result(queryFuture, queryTimeout); 138 | 139 | DataInputDeserializer dis = new DataInputDeserializer( 140 | queryResult, 141 | 0, 142 | queryResult.length); 143 | 144 | V value = valueSerializer.deserialize(dis); 145 | 146 | return Optional.ofNullable(value); 147 | } catch (UnknownKeyOrNamespace e) { 148 | // The future is failed with this Exception if the key does not exist 149 | return Optional.empty(); 150 | } 151 | } 152 | 153 | @Override 154 | public void close() throws Exception { 155 | client.shutDown(); 156 | } 157 | 158 | } 159 | -------------------------------------------------------------------------------- /src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | log4j.rootLogger=OFF, console 20 | 21 | log4j.appender.console=org.apache.log4j.ConsoleAppender 22 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 23 | log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %t %-5p %-60c %x - %m%n 24 | --------------------------------------------------------------------------------