forked from skylinemarketing/kafka-streams-examples
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathOrderDetailsService.java
193 lines (172 loc) · 7.78 KB
/
OrderDetailsService.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
package io.confluent.examples.streams.microservices;
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.ORDER_DETAILS_CHECK;
import static io.confluent.examples.streams.microservices.domain.Schemas.Topics;
import static io.confluent.examples.streams.microservices.util.MicroserviceUtils.addShutdownHookAndBlock;
import static java.util.Collections.singletonList;
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.OrderValidationResult;
import io.confluent.examples.streams.microservices.util.MicroserviceUtils;
import io.confluent.examples.streams.utils.MonitoringInterceptorUtils;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.TopicPartition;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Validates the details of each order.
* - Is the quantity positive?
* - Is there a customerId
* - etc...
* <p>
* This service could be built with Kafka Streams but we've used a Producer/Consumer pair
* including the integration with Kafka's Exactly Once feature (Transactions) to demonstrate
* this other style of building event driven services. Care needs to be taken with this approach
* as in the current release multi-node support is not provided for the transactional consumer
* (but it is supported inside Kafka Streams)
*/
public class OrderDetailsService implements Service {
private static final Logger log = LoggerFactory.getLogger(OrderDetailsService.class);
private final String CONSUMER_GROUP_ID = getClass().getSimpleName();
private KafkaConsumer<String, Order> consumer;
private KafkaProducer<String, OrderValidation> producer;
private final ExecutorService executorService = Executors.newSingleThreadExecutor();
private volatile boolean running;
// Disable Exactly Once Semantics to enable Confluent Monitoring Interceptors
private boolean eosEnabled = false;
@Override
public void start(final String bootstrapServers, final String stateDir) {
executorService.execute(() -> startService(bootstrapServers));
running = true;
log.info("Started Service " + getClass().getSimpleName());
}
private void startService(final String bootstrapServers) {
startConsumer(bootstrapServers);
startProducer(bootstrapServers);
try {
final Map<TopicPartition, OffsetAndMetadata> consumedOffsets = new HashMap<>();
consumer.subscribe(singletonList(Topics.ORDERS.name()));
if (eosEnabled) {
producer.initTransactions();
}
while (running) {
final ConsumerRecords<String, Order> records = consumer.poll(100);
if (records.count() > 0) {
if (eosEnabled) {
producer.beginTransaction();
}
for (final ConsumerRecord<String, Order> record : records) {
final Order order = record.value();
if (OrderState.CREATED.equals(order.getState())) {
//Validate the order then send the result (but note we are in a transaction so
//nothing will be "seen" downstream until we commit the transaction below)
producer.send(result(order, isValid(order) ? PASS : FAIL));
if (eosEnabled) {
recordOffset(consumedOffsets, record);
}
}
}
if (eosEnabled) {
producer.sendOffsetsToTransaction(consumedOffsets, CONSUMER_GROUP_ID);
producer.commitTransaction();
}
}
}
} finally {
close();
}
}
private void recordOffset(final Map<TopicPartition, OffsetAndMetadata> consumedOffsets,
final ConsumerRecord<String, Order> record) {
final OffsetAndMetadata nextOffset = new OffsetAndMetadata(record.offset() + 1);
consumedOffsets.put(new TopicPartition(record.topic(), record.partition()), nextOffset);
}
private ProducerRecord<String, OrderValidation> result(final Order order,
final OrderValidationResult passOrFail) {
return new ProducerRecord<>(
Topics.ORDER_VALIDATIONS.name(),
order.getId(),
new OrderValidation(order.getId(), ORDER_DETAILS_CHECK, passOrFail)
);
}
private void startProducer(final String bootstrapServers) {
final Properties producerConfig = new Properties();
producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
if (eosEnabled) {
producerConfig.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "OrderDetailsServiceInstance1");
}
producerConfig.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true");
producerConfig.put(ProducerConfig.RETRIES_CONFIG, String.valueOf(Integer.MAX_VALUE));
producerConfig.put(ProducerConfig.ACKS_CONFIG, "all");
producerConfig.put(ProducerConfig.CLIENT_ID_CONFIG, "order-details-service-producer");
MonitoringInterceptorUtils.maybeConfigureInterceptorsProducer(producerConfig);
producer = new KafkaProducer<>(producerConfig,
Topics.ORDER_VALIDATIONS.keySerde().serializer(),
Topics.ORDER_VALIDATIONS.valueSerde().serializer());
}
private void startConsumer(final String bootstrapServers) {
final Properties consumerConfig = new Properties();
consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
consumerConfig.put(ConsumerConfig.GROUP_ID_CONFIG, CONSUMER_GROUP_ID);
consumerConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
consumerConfig.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, !eosEnabled);
consumerConfig.put(ConsumerConfig.CLIENT_ID_CONFIG, "order-details-service-consumer");
MonitoringInterceptorUtils.maybeConfigureInterceptorsConsumer(consumerConfig);
consumer = new KafkaConsumer<>(consumerConfig,
Topics.ORDERS.keySerde().deserializer(),
Topics.ORDERS.valueSerde().deserializer());
}
private void close() {
if (producer != null) {
producer.close();
}
if (consumer != null) {
consumer.close();
}
}
@Override
public void stop() {
running = false;
try {
executorService.awaitTermination(1000, TimeUnit.MILLISECONDS);
} catch (final InterruptedException e) {
log.info("Failed to stop " + getClass().getSimpleName() + " in 1000ms");
}
log.info(getClass().getSimpleName() + " was stopped");
}
private boolean isValid(final Order order) {
if (order.getCustomerId() == null) {
return false;
}
if (order.getQuantity() < 0) {
return false;
}
if (order.getPrice() < 0) {
return false;
}
return order.getProduct() != null;
}
public void setEosEnabled(final boolean value) {
this.eosEnabled = value;
}
public static void main(final String[] args) throws Exception {
final OrderDetailsService service = new OrderDetailsService();
service.start(MicroserviceUtils.parseArgsAndConfigure(args), "/tmp/kafka-streams");
addShutdownHookAndBlock(service);
}
}