-
Notifications
You must be signed in to change notification settings - Fork 1.1k
/
Copy pathInventoryService.java
224 lines (196 loc) · 9.63 KB
/
InventoryService.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
package io.confluent.examples.streams.microservices;
import java.io.IOException;
import java.util.Optional;
import java.util.Properties;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import io.confluent.examples.streams.microservices.domain.Schemas;
import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig;
import org.apache.commons.cli.*;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.KafkaStreams.State;
import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.StreamsBuilder;
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.Produced;
import org.apache.kafka.streams.processor.api.Processor;
import org.apache.kafka.streams.processor.api.ProcessorContext;
import org.apache.kafka.streams.processor.api.Record;
import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.StoreBuilder;
import org.apache.kafka.streams.state.Stores;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.HashMap;
import io.confluent.examples.streams.avro.microservices.Order;
import io.confluent.examples.streams.avro.microservices.OrderState;
import io.confluent.examples.streams.avro.microservices.OrderValidation;
import io.confluent.examples.streams.avro.microservices.Product;
import io.confluent.examples.streams.microservices.domain.Schemas.Topics;
import io.confluent.examples.streams.microservices.util.MicroserviceUtils;
import static io.confluent.examples.streams.avro.microservices.OrderValidationResult.FAIL;
import static io.confluent.examples.streams.avro.microservices.OrderValidationResult.PASS;
import static io.confluent.examples.streams.avro.microservices.OrderValidationType.INVENTORY_CHECK;
import static io.confluent.examples.streams.microservices.util.MicroserviceUtils.*;
/**
* This service validates incoming orders to ensure there is sufficient stock to
* fulfill them. This validation process considers both the inventory in the warehouse
* as well as a set "reserved" items which is maintained by this service. Reserved
* items are those that are in the warehouse, but have been allocated to a pending
* order.
* <p>
* Currently there is nothing implemented that decrements the reserved items. This
* would happen, inside this service, in response to an order being shipped.
*/
public class InventoryService implements Service {
private static final Logger log = LoggerFactory.getLogger(InventoryService.class);
public static final String SERVICE_APP_ID = "InventoryService";
public static final String RESERVED_STOCK_STORE_NAME = "store-of-reserved-stock";
private KafkaStreams streams;
@Override
public void start(final String bootstrapServers,
final String stateDir,
final Properties defaultConfig) {
streams = processStreams(bootstrapServers, stateDir, defaultConfig);
streams.cleanUp(); //don't do this in prod as it clears your state stores
final CountDownLatch startLatch = new CountDownLatch(1);
streams.setStateListener((newState, oldState) -> {
if (newState == State.RUNNING && oldState != KafkaStreams.State.RUNNING) {
startLatch.countDown();
}
});
streams.start();
try {
if (!startLatch.await(60, TimeUnit.SECONDS)) {
throw new RuntimeException("Streams never finished rebalancing on startup");
}
} catch (final InterruptedException e) {
Thread.currentThread().interrupt();
}
log.info("Started Service " + getClass().getSimpleName());
}
@Override
public void stop() {
if (streams != null) {
streams.close();
}
}
private KafkaStreams processStreams(final String bootstrapServers,
final String stateDir,
final Properties defaultConfig) {
//Latch onto instances of the orders and inventory topics
final StreamsBuilder builder = new StreamsBuilder();
final KStream<String, Order> orders = builder
.stream(Topics.ORDERS.name(),
Consumed.with(Topics.ORDERS.keySerde(), Topics.ORDERS.valueSerde()));
final KTable<Product, Integer> warehouseInventory = builder
.table(Topics.WAREHOUSE_INVENTORY.name(),
Consumed.with(Topics.WAREHOUSE_INVENTORY.keySerde(), Topics.WAREHOUSE_INVENTORY.valueSerde()));
//Create a store to reserve inventory whilst the order is processed.
//This will be prepopulated from Kafka before the service starts processing
final StoreBuilder<KeyValueStore<Product, Long>> reservedStock = Stores
.keyValueStoreBuilder(Stores.persistentKeyValueStore(RESERVED_STOCK_STORE_NAME),
Topics.WAREHOUSE_INVENTORY.keySerde(), Serdes.Long())
.withLoggingEnabled(new HashMap<>());
builder.addStateStore(reservedStock);
//First change orders stream to be keyed by Product (so we can join with warehouse inventory)
orders.selectKey((id, order) -> order.getProduct())
//Limit to newly created orders
.filter((id, order) -> OrderState.CREATED.equals(order.getState()))
//Join Orders to Inventory so we can compare each order to its corresponding stock value
.join(warehouseInventory, KeyValue::new, Joined.with(Topics.WAREHOUSE_INVENTORY.keySerde(),
Topics.ORDERS.valueSerde(), Serdes.Integer()))
//Validate the order based on how much stock we have both in the warehouse and locally 'reserved' stock
.process(InventoryValidator::new, RESERVED_STOCK_STORE_NAME)
//Push the result into the Order Validations topic
.to(Topics.ORDER_VALIDATIONS.name(), Produced.with(Topics.ORDER_VALIDATIONS.keySerde(),
Topics.ORDER_VALIDATIONS.valueSerde()));
return new KafkaStreams(builder.build(),
MicroserviceUtils.baseStreamsConfig(bootstrapServers, stateDir, SERVICE_APP_ID, defaultConfig));
}
private static class InventoryValidator implements
Processor<Product, KeyValue<Order, Integer>, String, OrderValidation> {
private ProcessorContext<String, OrderValidation> context;
private KeyValueStore<Product, Long> reservedStocksStore;
@Override
public void init(final ProcessorContext<String, OrderValidation> context) {
this.context = context;
reservedStocksStore = context.getStateStore(RESERVED_STOCK_STORE_NAME);
}
@Override
public void process(final Record<Product, KeyValue<Order, Integer>> record) {
//Process each order/inventory pair one at a time
final OrderValidation validated;
final Order order = record.value().key;
final Integer warehouseStockCount = record.value().value;
//Look up locally 'reserved' stock from our state store
Long reserved = reservedStocksStore.get(order.getProduct());
if (reserved == null) {
reserved = 0L;
}
//If there is enough stock available (considering both warehouse inventory and reserved stock) validate the order
if (warehouseStockCount - reserved - order.getQuantity() >= 0) {
//reserve the stock by adding it to the 'reserved' store
reservedStocksStore.put(order.getProduct(), reserved + order.getQuantity());
//validate the order
validated = new OrderValidation(order.getId(), INVENTORY_CHECK, PASS);
} else {
//fail the order
validated = new OrderValidation(order.getId(), INVENTORY_CHECK, FAIL);
}
context.forward(record.withKey(validated.getOrderId()).withValue(validated));
}
}
public static void main(final String[] args) throws Exception {
final Options opts = new Options();
opts.addOption(Option.builder("b")
.longOpt("bootstrap-servers")
.hasArg()
.desc("Kafka cluster bootstrap server string (ex: broker:9092)")
.build());
opts.addOption(Option.builder("s")
.longOpt("schema-registry")
.hasArg()
.desc("Schema Registry URL")
.build());
opts.addOption(Option.builder("c")
.longOpt("config-file")
.hasArg()
.desc("Java properties file with configurations for Kafka Clients")
.build());
opts.addOption(Option.builder("t")
.longOpt("state-dir")
.hasArg()
.desc("The directory for state storage")
.build());
opts.addOption(Option.builder("h").longOpt("help").hasArg(false).desc("Show usage information").build());
final CommandLine cl = new DefaultParser().parse(opts, args);
if (cl.hasOption("h")) {
final HelpFormatter formatter = new HelpFormatter();
formatter.printHelp("Inventory Service", opts);
return;
}
final Properties defaultConfig = Optional.ofNullable(cl.getOptionValue("config-file", null))
.map(path -> {
try {
return buildPropertiesFromConfigFile(path);
} catch (final IOException e) {
throw new RuntimeException(e);
}
})
.orElse(new Properties());
final String schemaRegistryUrl = cl.getOptionValue("schema-registry", DEFAULT_SCHEMA_REGISTRY_URL);
defaultConfig.put(AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistryUrl);
Schemas.configureSerdes(defaultConfig);
final InventoryService service = new InventoryService();
service.start(
cl.getOptionValue("bootstrap-servers", DEFAULT_BOOTSTRAP_SERVERS),
cl.getOptionValue("state-dir", "/tmp/kafka-streams-examples"),
defaultConfig);
addShutdownHookAndBlock(service);
}
}