forked from skylinemarketing/kafka-streams-examples
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathOrdersService.java
398 lines (357 loc) · 15.6 KB
/
OrdersService.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
package io.confluent.examples.streams.microservices;
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.errors.InvalidStateStoreException;
import org.apache.kafka.streams.kstream.Consumed;
import org.apache.kafka.streams.kstream.Materialized;
import org.apache.kafka.streams.kstream.Predicate;
import org.apache.kafka.streams.state.QueryableStoreTypes;
import org.apache.kafka.streams.state.ReadOnlyKeyValueStore;
import org.apache.kafka.streams.StreamsConfig;
import org.eclipse.jetty.server.Server;
import org.glassfish.jersey.jackson.JacksonFeature;
import org.glassfish.jersey.server.ManagedAsync;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.ConcurrentHashMap;
import javax.ws.rs.Consumes;
import javax.ws.rs.DefaultValue;
import javax.ws.rs.GET;
import javax.ws.rs.POST;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.client.Client;
import javax.ws.rs.client.ClientBuilder;
import javax.ws.rs.container.AsyncResponse;
import javax.ws.rs.container.Suspended;
import javax.ws.rs.core.GenericType;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import io.confluent.examples.streams.avro.microservices.Order;
import io.confluent.examples.streams.avro.microservices.OrderState;
import io.confluent.examples.streams.interactivequeries.HostStoreInfo;
import io.confluent.examples.streams.interactivequeries.MetadataService;
import io.confluent.examples.streams.microservices.domain.Schemas;
import io.confluent.examples.streams.microservices.domain.beans.OrderBean;
import io.confluent.examples.streams.microservices.util.Paths;
import static io.confluent.examples.streams.microservices.domain.Schemas.Topics.ORDERS;
import static io.confluent.examples.streams.microservices.domain.beans.OrderBean.fromBean;
import static io.confluent.examples.streams.microservices.domain.beans.OrderBean.toBean;
import static io.confluent.examples.streams.microservices.util.MicroserviceUtils.addShutdownHookAndBlock;
import static io.confluent.examples.streams.microservices.util.MicroserviceUtils.baseStreamsConfig;
import static io.confluent.examples.streams.microservices.util.MicroserviceUtils.setTimeout;
import static io.confluent.examples.streams.microservices.util.MicroserviceUtils.startJetty;
import static io.confluent.examples.streams.microservices.util.MicroserviceUtils.startProducer;
import static org.apache.kafka.streams.state.StreamsMetadata.NOT_AVAILABLE;
/**
* This class provides a REST interface to write and read orders using a CQRS pattern
* (https://martinfowler.com/bliki/CQRS.html). Three methods are exposed over HTTP:
* <p>
* - POST(Order) -> Writes and order and returns location of the resource.
* <p>
* - GET(OrderId) (Optional timeout) -> Returns requested order, blocking for timeout if no id present.
* <p>
* - GET(OrderId)/Validated (Optional timeout)
* <p>
* POST does what you might expect: it adds an Order to the system returning when Kafka sends the appropriate
* acknowledgement.
* <p>
* GET accesses an inbuilt Materialized View, of Orders, which are kept in a
* State Store inside the service. This CQRS-styled view is updated asynchronously wrt the HTTP
* POST.
* <p>
* Calling GET(id) when the ID is not present will block the caller until either the order
* is added to the view, or the passed TIMEOUT period elapses. This allows the caller to
* read-their-own-writes.
* <p>
* In addition HTTP POST returns the location of the order submitted in the response.
* <p>
* Calling GET/id/validated will block until the FAILED/VALIDATED order is available in
* the View.
* <p>
* The View can also be scaled out linearly simply by adding more instances of the
* view service, and requests to any of the REST endpoints will be automatically forwarded to the
* correct instance for the key requested orderId via Kafka's Queryable State feature.
* <p>
* Non-blocking IO is used for all operations other than the intialization of state stores on
* startup or rebalance which will block calling Jetty thread.
*<p>
* NB This demo code only includes a partial implementation of the holding of outstanding requests
* and as such would lead timeouts if used in a production use case.
*/
@Path("v1")
public class OrdersService implements Service {
private static final Logger log = LoggerFactory.getLogger(OrdersService.class);
private static final String CALL_TIMEOUT = "10000";
private static final String ORDERS_STORE_NAME = "orders-store";
private final String SERVICE_APP_ID = getClass().getSimpleName();
private final Client client = ClientBuilder.newBuilder().register(JacksonFeature.class).build();
private Server jettyServer;
private final String host;
private int port;
private KafkaStreams streams = null;
private MetadataService metadataService;
private KafkaProducer<String, Order> producer;
//In a real implementation we would need to (a) support outstanding requests for the same Id/filter from
// different users and (b) periodically purge old entries from this map.
private final Map<String, FilteredResponse<String, Order>> outstandingRequests = new ConcurrentHashMap<>();
public OrdersService(final String host, final int port) {
this.host = host;
this.port = port;
}
public OrdersService(final String host) {
this(host, 0);
}
/**
* Create a table of orders which we can query. When the table is updated
* we check to see if there is an outstanding HTTP GET request waiting to be
* fulfilled.
*/
private StreamsBuilder createOrdersMaterializedView() {
final StreamsBuilder builder = new StreamsBuilder();
builder.table(ORDERS.name(), Consumed.with(ORDERS.keySerde(), ORDERS.valueSerde()), Materialized.as(ORDERS_STORE_NAME))
.toStream().foreach(this::maybeCompleteLongPollGet);
return builder;
}
private void maybeCompleteLongPollGet(final String id, final Order order) {
final FilteredResponse<String, Order> callback = outstandingRequests.get(id);
if (callback != null && callback.predicate.test(id, order)) {
callback.asyncResponse.resume(toBean(order));
}
}
/**
* Perform a "Long-Poll" styled get. This method will attempt to get the value for the passed key
* blocking until the key is available or passed timeout is reached. Non-blocking IO is used to
* implement this, but the API will block the calling thread if no metastore data is available
* (for example on startup or during a rebalance)
*
* @param id - the key of the value to retrieve
* @param timeout - the timeout for the long-poll
* @param asyncResponse - async response used to trigger the poll early should the appropriate
* value become available
*/
@GET
@ManagedAsync
@Path("/orders/{id}")
@Produces({MediaType.APPLICATION_JSON, MediaType.TEXT_PLAIN})
public void getWithTimeout(@PathParam("id") final String id,
@QueryParam("timeout") @DefaultValue(CALL_TIMEOUT) final Long timeout,
@Suspended final AsyncResponse asyncResponse) {
setTimeout(timeout, asyncResponse);
final HostStoreInfo hostForKey = getKeyLocationOrBlock(id, asyncResponse);
if (hostForKey == null) { //request timed out so return
return;
}
//Retrieve the order locally or reach out to a different instance if the required partition is hosted elsewhere.
if (thisHost(hostForKey)) {
fetchLocal(id, asyncResponse, (k, v) -> true);
} else {
final String path = new Paths(hostForKey.getHost(), hostForKey.getPort()).urlGet(id);
fetchFromOtherHost(path, asyncResponse, timeout);
}
}
class FilteredResponse<K, V> {
private final AsyncResponse asyncResponse;
private final Predicate<K, V> predicate;
FilteredResponse(final AsyncResponse asyncResponse, final Predicate<K, V> predicate) {
this.asyncResponse = asyncResponse;
this.predicate = predicate;
}
}
/**
* Fetch the order from the local materialized view
*
* @param id ID to fetch
* @param asyncResponse the response to call once completed
* @param predicate a filter that for this fetch, so for example we might fetch only VALIDATED
* orders.
*/
private void fetchLocal(final String id, final AsyncResponse asyncResponse, final Predicate<String, Order> predicate) {
log.info("running GET on this node");
try {
final Order order = ordersStore().get(id);
if (order == null || !predicate.test(id, order)) {
log.info("Delaying get as order not present for id " + id);
outstandingRequests.put(id, new FilteredResponse<>(asyncResponse, predicate));
} else {
asyncResponse.resume(toBean(order));
}
} catch (final InvalidStateStoreException e) {
//Store not ready so delay
outstandingRequests.put(id, new FilteredResponse<>(asyncResponse, predicate));
}
}
private ReadOnlyKeyValueStore<String, Order> ordersStore() {
return streams.store(ORDERS_STORE_NAME, QueryableStoreTypes.keyValueStore());
}
/**
* Use Kafka Streams' Queryable State API to work out if a key/value pair is located on
* this node, or on another Kafka Streams node. This returned HostStoreInfo can be used
* to redirect an HTTP request to the node that has the data.
* <p>
* If metadata is available, which can happen on startup, or during a rebalance, block until it is.
*/
private HostStoreInfo getKeyLocationOrBlock(final String id, final AsyncResponse asyncResponse) {
HostStoreInfo locationOfKey;
while (locationMetadataIsUnavailable(locationOfKey = getHostForOrderId(id))) {
//The metastore is not available. This can happen on startup/rebalance.
if (asyncResponse.isDone()) {
//The response timed out so return
return null;
}
try {
//Sleep a bit until metadata becomes available
Thread.sleep(Math.min(Long.valueOf(CALL_TIMEOUT), 200));
} catch (final InterruptedException e) {
e.printStackTrace();
}
}
return locationOfKey;
}
private boolean locationMetadataIsUnavailable(final HostStoreInfo hostWithKey) {
return NOT_AVAILABLE.host().equals(hostWithKey.getHost())
&& NOT_AVAILABLE.port() == hostWithKey.getPort();
}
private boolean thisHost(final HostStoreInfo host) {
return host.getHost().equals(this.host) &&
host.getPort() == port;
}
private void fetchFromOtherHost(final String path, final AsyncResponse asyncResponse, final long timeout) {
log.info("Chaining GET to a different instance: " + path);
try {
final OrderBean bean = client.target(path)
.queryParam("timeout", timeout)
.request(MediaType.APPLICATION_JSON_TYPE)
.get(new GenericType<OrderBean>() {
});
asyncResponse.resume(bean);
} catch (final Exception swallowed) {
}
}
@GET
@ManagedAsync
@Path("orders/{id}/validated")
public void getPostValidationWithTimeout(@PathParam("id") final String id,
@QueryParam("timeout") @DefaultValue(CALL_TIMEOUT) final Long timeout,
@Suspended final AsyncResponse asyncResponse) {
setTimeout(timeout, asyncResponse);
final HostStoreInfo hostForKey = getKeyLocationOrBlock(id, asyncResponse);
if (hostForKey == null) { //request timed out so return
return;
}
//Retrieve the order locally or reach out to a different instance if the required partition is hosted elsewhere.
if (thisHost(hostForKey)) {
fetchLocal(id, asyncResponse,
(k, v) -> (v.getState() == OrderState.VALIDATED || v.getState() == OrderState.FAILED));
} else {
fetchFromOtherHost(new Paths(hostForKey.getHost(), hostForKey.getPort()).urlGetValidated(id),
asyncResponse, timeout);
}
}
/**
* Persist an Order to Kafka. Returns once the order is successfully written to R nodes where
* R is the replication factor configured in Kafka.
*
* @param order the order to add
* @param timeout the max time to wait for the response from Kafka before timing out the POST
*/
@POST
@ManagedAsync
@Path("/orders")
@Consumes(MediaType.APPLICATION_JSON)
public void submitOrder(final OrderBean order,
@QueryParam("timeout") @DefaultValue(CALL_TIMEOUT) final Long timeout,
@Suspended final AsyncResponse response) {
setTimeout(timeout, response);
final Order bean = fromBean(order);
producer.send(new ProducerRecord<>(ORDERS.name(), bean.getId(), bean),
callback(response, bean.getId()));
}
@SuppressWarnings("unchecked")
@Override
public void start(final String bootstrapServers, final String stateDir) {
jettyServer = startJetty(port, this);
port = jettyServer.getURI().getPort(); // update port, in case port was zero
producer = startProducer(bootstrapServers, ORDERS);
streams = startKStreams(bootstrapServers);
log.info("Started Service " + getClass().getSimpleName());
}
private KafkaStreams startKStreams(final String bootstrapServers) {
final KafkaStreams streams = new KafkaStreams(
createOrdersMaterializedView().build(),
config(bootstrapServers));
metadataService = new MetadataService(streams);
streams.cleanUp(); //don't do this in prod as it clears your state stores
streams.start();
return streams;
}
private Properties config(final String bootstrapServers) {
final Properties props = baseStreamsConfig(bootstrapServers, "/tmp/kafka-streams", SERVICE_APP_ID);
props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, host + ":" + port);
return props;
}
@Override
public void stop() {
if (streams != null) {
streams.close();
}
if (producer != null) {
producer.close();
}
if (jettyServer != null) {
try {
jettyServer.stop();
} catch (final Exception e) {
e.printStackTrace();
}
}
}
// for testing only
void cleanLocalState() {
if (streams != null) {
streams.cleanUp();
}
}
public int port() {
return port;
}
private HostStoreInfo getHostForOrderId(final String orderId) {
return metadataService
.streamsMetadataForStoreAndKey(ORDERS_STORE_NAME, orderId, Serdes.String().serializer());
}
private Callback callback(final AsyncResponse response, final String orderId) {
return (recordMetadata, e) -> {
if (e != null) {
response.resume(e);
} else {
try {
//Return the location of the newly created resource
final Response uri = Response.created(new URI("/v1/orders/" + orderId)).build();
response.resume(uri);
} catch (final URISyntaxException e2) {
e2.printStackTrace();
}
}
};
}
public static void main(final String[] args) throws Exception {
final String bootstrapServers = args.length > 0 ? args[0] : "localhost:9092";
final String schemaRegistryUrl = args.length > 1 ? args[1] : "http://localhost:8081";
final String restHostname = args.length > 2 ? args[2] : "localhost";
final String restPort = args.length > 3 ? args[3] : null;
Schemas.configureSerdesWithSchemaRegistryUrl(schemaRegistryUrl);
final OrdersService service = new OrdersService(restHostname, restPort == null ? 0 : Integer.valueOf(restPort));
service.start(bootstrapServers, "/tmp/kafka-streams");
addShutdownHookAndBlock(service);
}
}