forked from skylinemarketing/kafka-streams-examples
-
Notifications
You must be signed in to change notification settings - Fork 0
/
EmailService.java
144 lines (118 loc) · 5.69 KB
/
EmailService.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
package io.confluent.examples.streams.microservices;
import static io.confluent.examples.streams.microservices.domain.Schemas.Topics.CUSTOMERS;
import static io.confluent.examples.streams.microservices.domain.Schemas.Topics.ORDERS;
import static io.confluent.examples.streams.microservices.domain.Schemas.Topics.ORDERS_ENRICHED;
import static io.confluent.examples.streams.microservices.domain.Schemas.Topics.PAYMENTS;
import static io.confluent.examples.streams.microservices.util.MicroserviceUtils.MIN;
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.parseArgsAndConfigure;
import io.confluent.examples.streams.avro.microservices.Customer;
import io.confluent.examples.streams.avro.microservices.Order;
import io.confluent.examples.streams.avro.microservices.OrderEnriched;
import io.confluent.examples.streams.avro.microservices.Payment;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.kstream.Consumed;
import org.apache.kafka.streams.kstream.GlobalKTable;
import org.apache.kafka.streams.kstream.JoinWindows;
import org.apache.kafka.streams.kstream.Joined;
import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.kstream.Produced;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A very simple service which sends emails. Order and Payment streams are joined
* using a window. The result is then joined to a lookup table of Customers.
* Finally an email is sent for each resulting tuple.
*/
public class EmailService implements Service {
private static final Logger log = LoggerFactory.getLogger(EmailService.class);
private final String SERVICE_APP_ID = getClass().getSimpleName();
private KafkaStreams streams;
private final Emailer emailer;
public EmailService(final Emailer emailer) {
this.emailer = emailer;
}
@Override
public void start(final String bootstrapServers, final String stateDir) {
streams = processStreams(bootstrapServers, stateDir);
streams.cleanUp(); //don't do this in prod as it clears your state stores
streams.start();
log.info("Started Service " + SERVICE_APP_ID);
}
private KafkaStreams processStreams(final String bootstrapServers, final String stateDir) {
final StreamsBuilder builder = new StreamsBuilder();
//Create the streams/tables for the join
final KStream<String, Order> orders = builder.stream(ORDERS.name(),
Consumed.with(ORDERS.keySerde(), ORDERS.valueSerde()));
final KStream<String, Payment> payments = builder.stream(PAYMENTS.name(),
Consumed.with(PAYMENTS.keySerde(), PAYMENTS.valueSerde()))
//Rekey payments to be by OrderId for the windowed join
.selectKey((s, payment) -> payment.getOrderId());
final GlobalKTable<Long, Customer> customers = builder.globalTable(CUSTOMERS.name(),
Consumed.with(CUSTOMERS.keySerde(), CUSTOMERS.valueSerde()));
final Joined<String, Order, Payment> serdes = Joined
.with(ORDERS.keySerde(), ORDERS.valueSerde(), PAYMENTS.valueSerde());
//Join the two streams and the table then send an email for each
orders.join(payments, EmailTuple::new,
//Join Orders and Payments streams
JoinWindows.of(MIN), serdes)
//Next join to the GKTable of Customers
.join(customers,
(key1, tuple) -> tuple.order.getCustomerId(),
// note how, because we use a GKtable, we can join on any attribute of the Customer.
EmailTuple::setCustomer)
//Now for each tuple send an email.
.peek((key, emailTuple)
-> emailer.sendEmail(emailTuple)
);
//Send the order to a topic whose name is the value of customer level
orders.join(customers, (orderId, order) -> order.getCustomerId(), (order, customer) -> new OrderEnriched (order.getId(), order.getCustomerId(), customer.getLevel()))
//TopicNameExtractor to get the topic name (i.e., customerLevel) from the enriched order record being sent
.to((orderId, orderEnriched, record) -> orderEnriched.getCustomerLevel(), Produced.with(ORDERS_ENRICHED.keySerde(), ORDERS_ENRICHED.valueSerde()));
return new KafkaStreams(builder.build(), baseStreamsConfig(bootstrapServers, stateDir, SERVICE_APP_ID));
}
public static void main(final String[] args) throws Exception {
final EmailService service = new EmailService(new LoggingEmailer());
service.start(parseArgsAndConfigure(args), "/tmp/kafka-streams");
addShutdownHookAndBlock(service);
}
private static class LoggingEmailer implements Emailer {
@Override
public void sendEmail(final EmailTuple details) {
//In a real implementation we would do something a little more useful
log.warn("Sending email: \nCustomer:{}\nOrder:{}\nPayment{}", details.customer, details.order, details.payment);
}
}
@Override
public void stop() {
if (streams != null) {
streams.close();
}
}
interface Emailer {
void sendEmail(EmailTuple details);
}
public class EmailTuple {
public Order order;
public Payment payment;
public Customer customer;
public EmailTuple(final Order order, final Payment payment) {
this.order = order;
this.payment = payment;
}
EmailTuple setCustomer(final Customer customer) {
this.customer = customer;
return this;
}
@Override
public String toString() {
return "EmailTuple{" +
"order=" + order +
", payment=" + payment +
", customer=" + customer +
'}';
}
}
}