> userPageViews = appDescriptor.getOutputStream(userPageViewsOutputDescriptor);
108 |
109 | pageViews
110 | .partitionBy(kv -> kv.value.userId, kv -> kv.value, pageViewKVSerde, "pageview")
111 | .window(Windows.keyedSessionWindow(kv -> kv.value.userId,
112 | Duration.ofSeconds(10), stringSerde, pageViewKVSerde), "usersession")
113 | .map(windowPane -> {
114 | String userId = windowPane.getKey().getKey();
115 | int views = windowPane.getMessage().size();
116 | return KV.of(userId, new UserPageViews(userId, views));
117 | })
118 | .sendTo(userPageViews);
119 | }
120 | }
121 |
--------------------------------------------------------------------------------
/src/main/java/samza/examples/cookbook/TumblingWindowExample.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,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 | package samza.examples.cookbook;
20 |
21 | import java.io.Serializable;
22 | import org.apache.samza.application.StreamApplication;
23 | import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
24 | import org.apache.samza.operators.KV;
25 | import org.apache.samza.operators.MessageStream;
26 | import org.apache.samza.operators.OutputStream;
27 | import org.apache.samza.operators.windows.Windows;
28 | import org.apache.samza.serializers.IntegerSerde;
29 | import org.apache.samza.serializers.JsonSerdeV2;
30 | import org.apache.samza.serializers.KVSerde;
31 | import org.apache.samza.serializers.StringSerde;
32 | import org.apache.samza.system.kafka.descriptors.KafkaInputDescriptor;
33 | import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
34 | import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
35 |
36 | import com.google.common.collect.ImmutableList;
37 | import com.google.common.collect.ImmutableMap;
38 | import samza.examples.cookbook.data.PageView;
39 | import samza.examples.cookbook.data.UserPageViews;
40 |
41 | import java.time.Duration;
42 | import java.util.List;
43 | import java.util.Map;
44 |
45 | /**
46 | * In this example, we group a stream of page views by country, and compute the number of page views over a tumbling time
47 | * window.
48 | *
49 | * Concepts covered: Performing Group-By style aggregations on tumbling time windows.
50 | *
51 | *
Tumbling windows divide a stream into a set of contiguous, fixed-sized, non-overlapping time intervals.
52 | *
53 | * To run the below example:
54 | *
55 | *
56 | * -
57 | * Ensure that the topic "pageview-tumbling-input" is created
58 | * ./deploy/kafka/bin/kafka-topics.sh --zookeeper localhost:2181 --create --topic pageview-tumbling-input --partitions 2 --replication-factor 1
59 | *
60 | * -
61 | * Run the application using the run-app.sh script
62 | * ./deploy/samza/bin/run-app.sh --config-path=$PWD/deploy/samza/config/tumbling-window-example.properties
63 | *
64 | * -
65 | * Produce some messages to the "pageview-tumbling-input" topic, waiting for some time between messages
66 | ./deploy/kafka/bin/kafka-console-producer.sh --topic pageview-tumbling-input --broker-list localhost:9092
67 | * {"userId": "user1", "country": "india", "pageId":"google.com/home"}
68 | * {"userId": "user1", "country": "india", "pageId":"google.com/search"}
69 | * {"userId": "user2", "country": "china", "pageId":"yahoo.com/home"}
70 | * {"userId": "user2", "country": "china", "pageId":"yahoo.com/sports"}
71 | * {"userId": "user1", "country": "india", "pageId":"google.com/news"}
72 | * {"userId": "user2", "country": "china", "pageId":"yahoo.com/fashion"}
73 | *
74 | * -
75 | * Consume messages from the "pageview-tumbling-output" topic (e.g. bin/kafka-console-consumer.sh)
76 | * ./deploy/kafka/bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic pageview-tumbling-output --property print.key=true
77 | *
78 | *
79 | *
80 | */
81 | public class TumblingWindowExample implements StreamApplication, Serializable {
82 | private static final String KAFKA_SYSTEM_NAME = "kafka";
83 | private static final List KAFKA_CONSUMER_ZK_CONNECT = ImmutableList.of("localhost:2181");
84 | private static final List KAFKA_PRODUCER_BOOTSTRAP_SERVERS = ImmutableList.of("localhost:9092");
85 | private static final Map KAFKA_DEFAULT_STREAM_CONFIGS = ImmutableMap.of("replication.factor", "1");
86 |
87 | private static final String INPUT_STREAM_ID = "pageview-tumbling-input";
88 | private static final String OUTPUT_STREAM_ID = "pageview-tumbling-output";
89 |
90 | @Override
91 | public void describe(StreamApplicationDescriptor appDescriptor) {
92 | KafkaSystemDescriptor kafkaSystemDescriptor = new KafkaSystemDescriptor(KAFKA_SYSTEM_NAME)
93 | .withConsumerZkConnect(KAFKA_CONSUMER_ZK_CONNECT)
94 | .withProducerBootstrapServers(KAFKA_PRODUCER_BOOTSTRAP_SERVERS)
95 | .withDefaultStreamConfigs(KAFKA_DEFAULT_STREAM_CONFIGS);
96 |
97 | KVSerde pageViewSerde = KVSerde.of(new StringSerde(), new JsonSerdeV2<>(PageView.class));
98 | KVSerde userPageViewSerde = KVSerde.of(new StringSerde(), new JsonSerdeV2<>(UserPageViews.class));
99 |
100 | KafkaInputDescriptor> pageViewInputDescriptor =
101 | kafkaSystemDescriptor.getInputDescriptor(INPUT_STREAM_ID, pageViewSerde);
102 | KafkaOutputDescriptor> userPageViewOutputDescriptor =
103 | kafkaSystemDescriptor.getOutputDescriptor(OUTPUT_STREAM_ID, userPageViewSerde);
104 |
105 | appDescriptor.withDefaultSystem(kafkaSystemDescriptor);
106 | MessageStream> pageViews = appDescriptor.getInputStream(pageViewInputDescriptor);
107 | OutputStream> outputStream = appDescriptor.getOutputStream(userPageViewOutputDescriptor);
108 |
109 | pageViews
110 | .partitionBy(kv -> kv.value.userId, kv -> kv.value, pageViewSerde, "userId")
111 | .window(Windows.keyedTumblingWindow(
112 | kv -> kv.key, Duration.ofSeconds(5), () -> 0, (m, prevCount) -> prevCount + 1,
113 | new StringSerde(), new IntegerSerde()), "count")
114 | .map(windowPane -> {
115 | String userId = windowPane.getKey().getKey();
116 | int views = windowPane.getMessage();
117 | return KV.of(userId, new UserPageViews(userId, views));
118 | })
119 | .sendTo(outputStream);
120 | }
121 | }
122 |
--------------------------------------------------------------------------------
/src/main/java/samza/examples/azure/AzureBlobApplication.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,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package samza.examples.azure;
21 |
22 | import com.google.common.collect.ImmutableList;
23 | import com.google.common.collect.ImmutableMap;
24 | import java.util.List;
25 | import java.util.Map;
26 | import org.apache.samza.application.StreamApplication;
27 | import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
28 | import org.apache.samza.operators.MessageStream;
29 | import org.apache.samza.operators.OutputStream;
30 | import org.apache.samza.serializers.JsonSerdeV2;
31 | import org.apache.samza.serializers.NoOpSerde;
32 | import org.apache.samza.system.descriptors.GenericOutputDescriptor;
33 | import org.apache.samza.system.descriptors.GenericSystemDescriptor;
34 | import org.apache.samza.system.kafka.descriptors.KafkaInputDescriptor;
35 | import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
36 | import org.slf4j.Logger;
37 | import org.slf4j.LoggerFactory;
38 | import samza.examples.azure.data.PageViewAvroRecord;
39 | import samza.examples.cookbook.data.PageView;
40 | import samza.examples.wikipedia.application.WikipediaApplication;
41 |
42 |
43 | /**
44 | * In this example, we demonstrate sending blobs to Azure Blob Storage.
45 | * This Samza job reads from Kafka topic "page-view-azure-blob-input" and produces blobs to Azure-Container "azure-blob-container" in your Azure Storage account.
46 | *
47 | * Currently, Samza supports sending Avro files are blobs.
48 | * Hence the incoming messages into the Samza job have to be converted to an Avro record.
49 | * For this job, we use input message as {@link samza.examples.cookbook.data.PageView} and
50 | * covert it to an Avro record defined as {@link samza.examples.azure.data.PageViewAvroRecord}.
51 | *
52 | * To run the below example:
53 | *
54 | *
55 | * -
56 | * Replace your-azure-storage-account-name and your-azure-storage-account-key with details of your Azure Storage Account.
57 | *
58 | * -
59 | * Ensure that the topic "page-view-azure-blob-input" is created
60 | * ./deploy/kafka/bin/kafka-topics.sh --zookeeper localhost:2181 --create --topic page-view-azure-blob-input --partitions 1 --replication-factor 1
61 | *
62 | * -
63 | * Run the application using the run-app.sh script
64 | * ./deploy/samza/bin/run-app.sh --config-path=$PWD/deploy/samza/config/azure-blob-application.properties
65 | *
66 | * -
67 | * Produce some messages to the "page-view-azure-blob-input" topic
68 | * ./deploy/kafka/bin/kafka-console-producer.sh --topic page-view-azure-blob-input --broker-list localhost:9092
69 | * {"userId": "user1", "country": "india", "pageId":"google.com"}
70 | * {"userId": "user2", "country": "france", "pageId":"facebook.com"}
71 | * {"userId": "user3", "country": "china", "pageId":"yahoo.com"}
72 | * {"userId": "user4", "country": "italy", "pageId":"linkedin.com"}
73 | * {"userId": "user5", "country": "germany", "pageId":"amazon.com"}
74 | * {"userId": "user6", "country": "denmark", "pageId":"apple.com"}
75 | *
76 | * -
77 | * Seeing Output:
78 | *
79 | * -
80 | * See blobs in your Azure portal at https://.blob.core.windows.net/azure-blob-container/PageViewEventStream/.avro
81 | *
82 | * -
83 | * system-name "azure-blob-container" in configs and code below maps to Azure-Container in Azure Storage account.
84 | *
85 | * -
86 | * is of the format yyyy/MM/dd/HH/mm-ss-randomString.avro. Hence navigate through the virtual folders on the portal to see your blobs.
87 | *
88 | * -
89 | * Due to network calls, allow a few minutes for blobs to appear on the portal.
90 | *
91 | * -
92 | * Config "maxMessagesPerBlob=2" ensures that a blob is created per 2 input messages. Adjust input or config accordingly.
93 | *
94 | *
95 | *
96 | *
97 | */
98 | public class AzureBlobApplication implements StreamApplication {
99 | private static final Logger LOG = LoggerFactory.getLogger(AzureBlobApplication.class);
100 |
101 | private static final List KAFKA_CONSUMER_ZK_CONNECT = ImmutableList.of("localhost:2181");
102 | private static final List KAFKA_PRODUCER_BOOTSTRAP_SERVERS = ImmutableList.of("localhost:9092");
103 | private static final Map KAFKA_DEFAULT_STREAM_CONFIGS = ImmutableMap.of("replication.factor", "1");
104 | private static final String INPUT_PAGEVIEW_STREAM_ID = "page-view-azure-blob-input";
105 | private static final String OUTPUT_SYSTEM = "azure-blob-container";
106 | private static final String OUTPUT_STREAM = "PageViewEventStream";
107 |
108 | @Override
109 | public void describe(StreamApplicationDescriptor appDescriptor) {
110 | // Define a system descriptor for Kafka
111 | KafkaSystemDescriptor kafkaSystemDescriptor =
112 | new KafkaSystemDescriptor("kafka").withConsumerZkConnect(KAFKA_CONSUMER_ZK_CONNECT)
113 | .withProducerBootstrapServers(KAFKA_PRODUCER_BOOTSTRAP_SERVERS)
114 | .withDefaultStreamConfigs(KAFKA_DEFAULT_STREAM_CONFIGS);
115 |
116 | KafkaInputDescriptor pageViewInputDescriptor =
117 | kafkaSystemDescriptor.getInputDescriptor(INPUT_PAGEVIEW_STREAM_ID, new JsonSerdeV2<>(PageView.class));
118 |
119 | // Define a system descriptor for Azure Blob Storage
120 | GenericSystemDescriptor azureBlobSystemDescriptor =
121 | new GenericSystemDescriptor(OUTPUT_SYSTEM, "org.apache.samza.system.azureblob.AzureBlobSystemFactory");
122 |
123 | GenericOutputDescriptor azureBlobOuputDescriptor =
124 | azureBlobSystemDescriptor.getOutputDescriptor(OUTPUT_STREAM, new NoOpSerde<>());
125 |
126 | // Set Kafka as the default system for the job
127 | appDescriptor.withDefaultSystem(kafkaSystemDescriptor);
128 |
129 | // Define the input and output streams with descriptors
130 | MessageStream pageViewInput = appDescriptor.getInputStream(pageViewInputDescriptor);
131 | OutputStream pageViewAvroRecordOutputStream = appDescriptor.getOutputStream(azureBlobOuputDescriptor);
132 |
133 | // Define the execution flow with the high-level API
134 | pageViewInput
135 | .map((message) -> {
136 | LOG.info("Sending: Received PageViewEvent with pageId: " + message.pageId);
137 | return PageViewAvroRecord.buildPageViewRecord(message);
138 | })
139 | .sendTo(pageViewAvroRecordOutputStream);
140 | }
141 | }
142 |
--------------------------------------------------------------------------------
/src/main/java/samza/examples/cookbook/JoinExample.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,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 | package samza.examples.cookbook;
20 |
21 | import java.io.Serializable;
22 | import org.apache.samza.application.StreamApplication;
23 | import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
24 | import org.apache.samza.operators.KV;
25 | import org.apache.samza.operators.MessageStream;
26 | import org.apache.samza.operators.OutputStream;
27 | import org.apache.samza.operators.functions.JoinFunction;
28 | import org.apache.samza.serializers.JsonSerdeV2;
29 | import org.apache.samza.serializers.KVSerde;
30 | import org.apache.samza.serializers.StringSerde;
31 | import org.apache.samza.system.kafka.descriptors.KafkaInputDescriptor;
32 | import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
33 | import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
34 |
35 | import com.google.common.collect.ImmutableList;
36 | import com.google.common.collect.ImmutableMap;
37 | import samza.examples.cookbook.data.AdClick;
38 | import samza.examples.cookbook.data.PageView;
39 |
40 | import java.time.Duration;
41 | import java.util.List;
42 | import java.util.Map;
43 |
44 | /**
45 | * In this example, we join a stream of Page views with a stream of Ad clicks. For instance, this is helpful for
46 | * analysis on what pages served an Ad that was clicked.
47 | *
48 | * Concepts covered: Performing stream to stream Joins.
49 | *
50 | * To run the below example:
51 | *
52 | *
53 | * -
54 | * Ensure that the topics "pageview-join-input", "adclick-join-input" are created
55 | * ./deploy/kafka/bin/kafka-topics.sh --zookeeper localhost:2181 --create --topic pageview-join-input --partitions 2 --replication-factor 1
56 | * ./deploy/kafka/bin/kafka-topics.sh --zookeeper localhost:2181 --create --topic adclick-join-input --partitions 2 --replication-factor 1
57 | *
58 | * -
59 | * Run the application using the run-app.sh script
60 | * ./deploy/samza/bin/run-app.sh --config-path=$PWD/deploy/samza/config/join-example.properties
61 | *
62 | * -
63 | * Produce some messages to the "pageview-join-input" topic
64 | * ./deploy/kafka/bin/kafka-console-producer.sh --topic pageview-join-input --broker-list localhost:9092
65 | * {"userId": "user1", "country": "india", "pageId":"google.com"}
66 | * {"userId": "user2", "country": "china", "pageId":"yahoo.com"}
67 | *
68 | * -
69 | * Produce some messages to the "adclick-join-input" topic with the same pageKey
70 | * ./deploy/kafka/bin/kafka-console-producer.sh --topic adclick-join-input --broker-list localhost:9092
71 | * {"userId": "user1", "adId": "adClickId1", "pageId":"google.com"}
72 | * {"userId": "user1", "adId": "adClickId2", "pageId":"yahoo.com"}
73 | *
74 | * -
75 | * Consume messages from the "pageview-adclick-join-output" topic
76 | * ./deploy/kafka/bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic pageview-adclick-join-output --property print.key=true
77 | *
78 | *
79 | *
80 | */
81 | public class JoinExample implements StreamApplication, Serializable {
82 | private static final String KAFKA_SYSTEM_NAME = "kafka";
83 | private static final List KAFKA_CONSUMER_ZK_CONNECT = ImmutableList.of("localhost:2181");
84 | private static final List KAFKA_PRODUCER_BOOTSTRAP_SERVERS = ImmutableList.of("localhost:9092");
85 | private static final Map KAFKA_DEFAULT_STREAM_CONFIGS = ImmutableMap.of("replication.factor", "1");
86 |
87 | private static final String PAGEVIEW_STREAM_ID = "pageview-join-input";
88 | private static final String ADCLICK_STREAM_ID = "adclick-join-input";
89 | private static final String OUTPUT_STREAM_ID = "pageview-adclick-join-output";
90 |
91 | @Override
92 | public void describe(StreamApplicationDescriptor appDescriptor) {
93 | StringSerde stringSerde = new StringSerde();
94 | JsonSerdeV2 pageViewSerde = new JsonSerdeV2<>(PageView.class);
95 | JsonSerdeV2 adClickSerde = new JsonSerdeV2<>(AdClick.class);
96 | JsonSerdeV2 joinResultSerde = new JsonSerdeV2<>(JoinResult.class);
97 |
98 | KafkaSystemDescriptor kafkaSystemDescriptor = new KafkaSystemDescriptor(KAFKA_SYSTEM_NAME)
99 | .withConsumerZkConnect(KAFKA_CONSUMER_ZK_CONNECT)
100 | .withProducerBootstrapServers(KAFKA_PRODUCER_BOOTSTRAP_SERVERS)
101 | .withDefaultStreamConfigs(KAFKA_DEFAULT_STREAM_CONFIGS);
102 |
103 | KafkaInputDescriptor pageViewInputDescriptor =
104 | kafkaSystemDescriptor.getInputDescriptor(PAGEVIEW_STREAM_ID, pageViewSerde);
105 | KafkaInputDescriptor adClickInputDescriptor =
106 | kafkaSystemDescriptor.getInputDescriptor(ADCLICK_STREAM_ID, adClickSerde);
107 | KafkaOutputDescriptor joinResultOutputDescriptor =
108 | kafkaSystemDescriptor.getOutputDescriptor(OUTPUT_STREAM_ID, joinResultSerde);
109 |
110 | appDescriptor.withDefaultSystem(kafkaSystemDescriptor);
111 |
112 | MessageStream pageViews = appDescriptor.getInputStream(pageViewInputDescriptor);
113 | MessageStream adClicks = appDescriptor.getInputStream(adClickInputDescriptor);
114 | OutputStream joinResults = appDescriptor.getOutputStream(joinResultOutputDescriptor);
115 |
116 | JoinFunction pageViewAdClickJoinFunction =
117 | new JoinFunction() {
118 | @Override
119 | public JoinResult apply(PageView pageView, AdClick adClick) {
120 | return new JoinResult(pageView.pageId, pageView.userId, pageView.country, adClick.getAdId());
121 | }
122 |
123 | @Override
124 | public String getFirstKey(PageView pageView) {
125 | return pageView.pageId;
126 | }
127 |
128 | @Override
129 | public String getSecondKey(AdClick adClick) {
130 | return adClick.getPageId();
131 | }
132 | };
133 |
134 | MessageStream repartitionedPageViews =
135 | pageViews
136 | .partitionBy(pv -> pv.pageId, pv -> pv, KVSerde.of(stringSerde, pageViewSerde), "pageview")
137 | .map(KV::getValue);
138 |
139 | MessageStream repartitionedAdClicks =
140 | adClicks
141 | .partitionBy(AdClick::getPageId, ac -> ac, KVSerde.of(stringSerde, adClickSerde), "adclick")
142 | .map(KV::getValue);
143 |
144 | repartitionedPageViews
145 | .join(repartitionedAdClicks, pageViewAdClickJoinFunction,
146 | stringSerde, pageViewSerde, adClickSerde, Duration.ofMinutes(3), "join")
147 | .sendTo(joinResults);
148 | }
149 |
150 | static class JoinResult {
151 | public String pageId;
152 | public String userId;
153 | public String country;
154 | public String adId;
155 |
156 | public JoinResult(String pageId, String userId, String country, String adId) {
157 | this.pageId = pageId;
158 | this.userId = userId;
159 | this.country = country;
160 | this.adId = adId;
161 | }
162 | }
163 | }
164 |
--------------------------------------------------------------------------------
/src/main/java/samza/examples/cookbook/StreamTableJoinExample.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,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 | package samza.examples.cookbook;
20 |
21 | import java.util.Objects;
22 | import org.apache.samza.application.StreamApplication;
23 | import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
24 | import org.apache.samza.operators.KV;
25 | import org.apache.samza.operators.MessageStream;
26 | import org.apache.samza.operators.OutputStream;
27 | import org.apache.samza.operators.functions.StreamTableJoinFunction;
28 | import org.apache.samza.serializers.JsonSerdeV2;
29 | import org.apache.samza.serializers.KVSerde;
30 | import org.apache.samza.serializers.Serde;
31 | import org.apache.samza.serializers.StringSerde;
32 | import org.apache.samza.storage.kv.descriptors.RocksDbTableDescriptor;
33 | import org.apache.samza.system.kafka.descriptors.KafkaInputDescriptor;
34 | import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
35 | import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
36 | import org.apache.samza.table.Table;
37 |
38 | import com.google.common.collect.ImmutableList;
39 | import com.google.common.collect.ImmutableMap;
40 | import samza.examples.cookbook.data.PageView;
41 | import samza.examples.cookbook.data.Profile;
42 |
43 | import java.util.List;
44 | import java.util.Map;
45 |
46 | /**
47 | * In this example, we join a stream of Page views with a table of user profiles, which is populated from an
48 | * user profile stream. For instance, this is helpful for analysis that required additional information from
49 | * user's profile.
50 | *
51 | * Concepts covered: Performing stream-to-table joins.
52 | *
53 | * To run the below example:
54 | *
55 | *
56 | * -
57 | * Ensure that the topics "pageview-join-input", "profile-table-input" are created
58 | * ./deploy/kafka/bin/kafka-topics.sh --zookeeper localhost:2181 --create --topic pageview-join-input --partitions 2 --replication-factor 1
59 | * ./deploy/kafka/bin/kafka-topics.sh --zookeeper localhost:2181 --create --topic profile-table-input --partitions 2 --replication-factor 1
60 | *
61 | * -
62 | * Run the application using the run-app.sh script
63 | * ./deploy/samza/bin/run-app.sh --config-path=$PWD/deploy/samza/config/stream-table-join-example.properties
64 | *
65 | * -
66 | * Consume messages from the "enriched-pageview-join-output" topic
67 | * ./deploy/kafka/bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic enriched-pageview-join-output
68 | *
69 | * -
70 | * Produce some messages to the "profile-table-input" topic with the same userId
71 | * ./deploy/kafka/bin/kafka-console-producer.sh --topic profile-table-input --broker-list localhost:9092
72 | * {"userId": "user1", "company": "LNKD"}
73 | * {"userId": "user2", "company": "MSFT"}
74 | *
75 | * -
76 | * Produce some messages to the "pageview-join-input" topic
77 | * ./deploy/kafka/bin/kafka-console-producer.sh --topic pageview-join-input --broker-list localhost:9092
78 | * {"userId": "user1", "country": "india", "pageId":"google.com"}
79 | * {"userId": "user2", "country": "china", "pageId":"yahoo.com"}
80 | *
81 | *
82 | *
83 | */
84 | public class StreamTableJoinExample implements StreamApplication {
85 | private static final String KAFKA_SYSTEM_NAME = "kafka";
86 | private static final List KAFKA_CONSUMER_ZK_CONNECT = ImmutableList.of("localhost:2181");
87 | private static final List KAFKA_PRODUCER_BOOTSTRAP_SERVERS = ImmutableList.of("localhost:9092");
88 | private static final Map KAFKA_DEFAULT_STREAM_CONFIGS = ImmutableMap.of("replication.factor", "1");
89 |
90 | private static final String PROFILE_STREAM_ID = "profile-table-input";
91 | private static final String PAGEVIEW_STREAM_ID = "pageview-join-input";
92 | private static final String OUTPUT_TOPIC = "enriched-pageview-join-output";
93 |
94 | @Override
95 | public void describe(StreamApplicationDescriptor appDescriptor) {
96 | Serde profileSerde = new JsonSerdeV2<>(Profile.class);
97 | Serde pageViewSerde = new JsonSerdeV2<>(PageView.class);
98 | Serde joinResultSerde = new JsonSerdeV2<>(EnrichedPageView.class);
99 |
100 | KafkaSystemDescriptor kafkaSystemDescriptor = new KafkaSystemDescriptor(KAFKA_SYSTEM_NAME)
101 | .withConsumerZkConnect(KAFKA_CONSUMER_ZK_CONNECT)
102 | .withProducerBootstrapServers(KAFKA_PRODUCER_BOOTSTRAP_SERVERS)
103 | .withDefaultStreamConfigs(KAFKA_DEFAULT_STREAM_CONFIGS);
104 |
105 | KafkaInputDescriptor profileInputDescriptor =
106 | kafkaSystemDescriptor.getInputDescriptor(PROFILE_STREAM_ID, profileSerde);
107 | KafkaInputDescriptor pageViewInputDescriptor =
108 | kafkaSystemDescriptor.getInputDescriptor(PAGEVIEW_STREAM_ID, pageViewSerde);
109 | KafkaOutputDescriptor joinResultOutputDescriptor =
110 | kafkaSystemDescriptor.getOutputDescriptor(OUTPUT_TOPIC, joinResultSerde);
111 |
112 | RocksDbTableDescriptor profileTableDescriptor =
113 | new RocksDbTableDescriptor("profile-table", KVSerde.of(new StringSerde(), profileSerde));
114 |
115 | appDescriptor.withDefaultSystem(kafkaSystemDescriptor);
116 |
117 | MessageStream profileStream = appDescriptor.getInputStream(profileInputDescriptor);
118 | MessageStream pageViewStream = appDescriptor.getInputStream(pageViewInputDescriptor);
119 | OutputStream joinResultStream = appDescriptor.getOutputStream(joinResultOutputDescriptor);
120 | Table> profileTable = appDescriptor.getTable(profileTableDescriptor);
121 |
122 | profileStream
123 | .map(profile -> KV.of(profile.userId, profile))
124 | .sendTo(profileTable);
125 |
126 | pageViewStream
127 | .partitionBy(pv -> pv.userId, pv -> pv, KVSerde.of(new StringSerde(), pageViewSerde), "join")
128 | .join(profileTable, new JoinFn())
129 | .sendTo(joinResultStream);
130 | }
131 |
132 | private static class JoinFn implements StreamTableJoinFunction, KV, EnrichedPageView> {
133 | @Override
134 | public EnrichedPageView apply(KV message, KV record) {
135 | return record == null ? null :
136 | new EnrichedPageView(message.getKey(), record.getValue().company, message.getValue().pageId);
137 | }
138 | @Override
139 | public String getMessageKey(KV message) {
140 | return message.getKey();
141 | }
142 | @Override
143 | public String getRecordKey(KV record) {
144 | return record.getKey();
145 | }
146 | }
147 |
148 | static public class EnrichedPageView {
149 |
150 | public final String userId;
151 | public final String company;
152 | public final String pageId;
153 |
154 | public EnrichedPageView(String userId, String company, String pageId) {
155 | this.userId = userId;
156 | this.company = company;
157 | this.pageId = pageId;
158 | }
159 |
160 | @Override
161 | public boolean equals(Object o) {
162 | if (this == o) {
163 | return true;
164 | }
165 | if (o == null || getClass() != o.getClass()) {
166 | return false;
167 | }
168 | EnrichedPageView that = (EnrichedPageView) o;
169 | return Objects.equals(userId, that.userId) && Objects.equals(company, that.company) && Objects.equals(pageId,
170 | that.pageId);
171 | }
172 | }
173 |
174 | }
175 |
--------------------------------------------------------------------------------
/src/test/java/samza/examples/cookbook/test/TestSamzaCookBookExamples.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,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 | package samza.examples.cookbook.test;
20 |
21 | import java.time.Duration;
22 | import java.util.ArrayList;
23 | import java.util.List;
24 | import org.apache.samza.operators.KV;
25 | import org.apache.samza.serializers.NoOpSerde;
26 | import org.apache.samza.test.framework.StreamAssert;
27 | import org.apache.samza.test.framework.TestRunner;
28 | import org.apache.samza.test.framework.system.descriptors.InMemoryInputDescriptor;
29 | import org.apache.samza.test.framework.system.descriptors.InMemoryOutputDescriptor;
30 | import org.apache.samza.test.framework.system.descriptors.InMemorySystemDescriptor;
31 | import org.junit.Assert;
32 | import org.junit.Test;
33 | import samza.examples.cookbook.FilterExample;
34 | import samza.examples.cookbook.JoinExample;
35 | import samza.examples.cookbook.SessionWindowExample;
36 | import samza.examples.cookbook.StreamTableJoinExample;
37 | import samza.examples.cookbook.TumblingWindowExample;
38 | import samza.examples.cookbook.data.AdClick;
39 | import samza.examples.cookbook.data.PageView;
40 | import samza.examples.cookbook.data.Profile;
41 | import samza.examples.cookbook.data.UserPageViews;
42 | import samza.examples.test.utils.TestUtils;
43 |
44 | import static samza.examples.cookbook.StreamTableJoinExample.EnrichedPageView;
45 |
46 |
47 | public class TestSamzaCookBookExamples {
48 | @Test
49 | public void testFilterExample() {
50 | List rawPageViewEvents = new ArrayList<>();
51 | rawPageViewEvents.add(new PageView("google.com", "user1", "india"));
52 | rawPageViewEvents.add(new PageView("facebook.com", "invalidUserId", "france"));
53 | rawPageViewEvents.add(new PageView("yahoo.com", "user2", "china"));
54 |
55 | InMemorySystemDescriptor inMemorySystem = new InMemorySystemDescriptor("kafka");
56 |
57 | InMemoryInputDescriptor badPageViewEvents =
58 | inMemorySystem.getInputDescriptor("pageview-filter-input", new NoOpSerde());
59 |
60 | InMemoryOutputDescriptor goodPageViewEvents =
61 | inMemorySystem.getOutputDescriptor("pageview-filter-output", new NoOpSerde());
62 |
63 | TestRunner
64 | .of(new FilterExample())
65 | .addInputStream(badPageViewEvents, rawPageViewEvents)
66 | .addOutputStream(goodPageViewEvents, 1)
67 | .run(Duration.ofMillis(1500));
68 |
69 | Assert.assertEquals(TestRunner.consumeStream(goodPageViewEvents, Duration.ofMillis(1000)).get(0).size(), 2);
70 | }
71 |
72 | @Test
73 | public void testJoinExample() {
74 | List pageViewEvents = new ArrayList<>();
75 | pageViewEvents.add(new PageView("google.com", "user1", "india"));
76 | pageViewEvents.add(new PageView("yahoo.com", "user2", "china"));
77 | List adClickEvents = new ArrayList<>();
78 | adClickEvents.add(new AdClick("google.com", "adClickId1", "user1"));
79 | adClickEvents.add(new AdClick("yahoo.com", "adClickId2", "user1"));
80 |
81 | InMemorySystemDescriptor inMemorySystem = new InMemorySystemDescriptor("kafka");
82 |
83 | InMemoryInputDescriptor pageViews =
84 | inMemorySystem.getInputDescriptor("pageview-join-input", new NoOpSerde());
85 |
86 | InMemoryInputDescriptor adClicks =
87 | inMemorySystem.getInputDescriptor("adclick-join-input", new NoOpSerde());
88 |
89 | InMemoryOutputDescriptor pageViewAdClickJoin =
90 | inMemorySystem.getOutputDescriptor("pageview-adclick-join-output", new NoOpSerde<>());
91 |
92 | TestRunner
93 | .of(new JoinExample())
94 | .addInputStream(pageViews, pageViewEvents)
95 | .addInputStream(adClicks, adClickEvents)
96 | .addOutputStream(pageViewAdClickJoin, 1)
97 | .run(Duration.ofMillis(1500));
98 |
99 | Assert.assertEquals(TestRunner.consumeStream(pageViewAdClickJoin, Duration.ofMillis(1000)).get(0).size(), 2);
100 | }
101 |
102 | @Test
103 | public void testTumblingWindowExample() {
104 | List pageViewEvents = TestUtils.genSamplePageViewData();
105 |
106 | InMemorySystemDescriptor inMemorySystem = new InMemorySystemDescriptor("kafka");
107 |
108 | InMemoryInputDescriptor> pageViewInputDescriptor =
109 | inMemorySystem.getInputDescriptor("pageview-tumbling-input", new NoOpSerde>());
110 |
111 | InMemoryOutputDescriptor> userPageViewOutputDescriptor =
112 | inMemorySystem.getOutputDescriptor("pageview-tumbling-output", new NoOpSerde>());
113 |
114 | TestRunner
115 | .of(new TumblingWindowExample())
116 | .addInputStream(pageViewInputDescriptor, pageViewEvents)
117 | .addOutputStream(userPageViewOutputDescriptor, 1)
118 | .run(Duration.ofMinutes(1));
119 |
120 | Assert.assertTrue(TestRunner.consumeStream(userPageViewOutputDescriptor, Duration.ofMillis(1000)).get(0).size() > 1);
121 | }
122 |
123 | @Test
124 | public void testSessionWindowExample() {
125 | List pageViewEvents = TestUtils.genSamplePageViewData();
126 |
127 | InMemorySystemDescriptor inMemorySystem = new InMemorySystemDescriptor("kafka");
128 |
129 | InMemoryInputDescriptor> pageViewInputDescriptor =
130 | inMemorySystem.getInputDescriptor("pageview-session-input", new NoOpSerde>());
131 |
132 | InMemoryOutputDescriptor> userPageViewOutputDescriptor =
133 | inMemorySystem.getOutputDescriptor("pageview-session-output", new NoOpSerde>());
134 |
135 | TestRunner
136 | .of(new SessionWindowExample())
137 | .addInputStream(pageViewInputDescriptor, pageViewEvents)
138 | .addOutputStream(userPageViewOutputDescriptor, 1)
139 | .run(Duration.ofMinutes(1));
140 |
141 | Assert.assertEquals(2, TestRunner.consumeStream(userPageViewOutputDescriptor, Duration.ofMillis(1000)).get(0).size());
142 | }
143 |
144 | @Test
145 | public void testStreamTableJoinExample() throws InterruptedException{
146 | List pageViewEvents = new ArrayList<>();
147 | pageViewEvents.add(new PageView("google.com", "user1", "india"));
148 | pageViewEvents.add(new PageView("yahoo.com", "user2", "china"));
149 | List profiles = new ArrayList<>();
150 | profiles.add(new Profile("user1", "LNKD"));
151 | profiles.add(new Profile("user2", "MSFT"));
152 |
153 | InMemorySystemDescriptor inMemorySystem = new InMemorySystemDescriptor("kafka");
154 |
155 | InMemoryInputDescriptor pageViews =
156 | inMemorySystem.getInputDescriptor("pageview-join-input", new NoOpSerde());
157 |
158 | InMemoryInputDescriptor profileViews =
159 | inMemorySystem.getInputDescriptor("profile-table-input", new NoOpSerde());
160 |
161 | InMemoryOutputDescriptor joinResultOutputDescriptor =
162 | inMemorySystem.getOutputDescriptor("enriched-pageview-join-output", new NoOpSerde());
163 |
164 | TestRunner
165 | .of(new StreamTableJoinExample())
166 | .addInputStream(pageViews, pageViewEvents)
167 | .addInputStream(profileViews, profiles)
168 | .addOutputStream(joinResultOutputDescriptor, 1)
169 | .run(Duration.ofMillis(1500));
170 |
171 | List expectedOutput = new ArrayList<>();
172 | expectedOutput.add(new EnrichedPageView("user1", "LNKD", "google.com"));
173 | expectedOutput.add(new EnrichedPageView("user2", "MSFT", "yahoo.com"));
174 |
175 | StreamAssert.containsInAnyOrder(expectedOutput, joinResultOutputDescriptor, Duration.ofMillis(200));
176 |
177 | }
178 |
179 | }
180 |
--------------------------------------------------------------------------------
/bin/grid:
--------------------------------------------------------------------------------
1 | #!/bin/bash -e
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,
13 | # software distributed under the License is distributed on an
14 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | # KIND, either express or implied. See the License for the
16 | # specific language governing permissions and limitations
17 | # under the License.
18 |
19 | # This script will download, setup, start, and stop servers for Kafka, YARN, and ZooKeeper,
20 | # as well as downloading, building and locally publishing Samza
21 |
22 | if [ -z "$JAVA_HOME" ]; then
23 | if [ -x /usr/libexec/java_home ]; then
24 | export JAVA_HOME="$(/usr/libexec/java_home)"
25 | else
26 | echo "JAVA_HOME not set. Exiting."
27 | exit 1
28 | fi
29 | fi
30 |
31 | DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
32 | BASE_DIR=$(dirname $DIR)
33 | DEPLOY_ROOT_DIR=$BASE_DIR/deploy
34 | DOWNLOAD_CACHE_DIR=$HOME/.samza/download
35 | COMMAND=$1
36 | SYSTEM=$2
37 |
38 | DOWNLOAD_KAFKA=https://archive.apache.org/dist/kafka/2.1.1/kafka_2.11-2.1.1.tgz
39 | DOWNLOAD_YARN=https://archive.apache.org/dist/hadoop/common/hadoop-2.9.2/hadoop-2.9.2.tar.gz
40 | DOWNLOAD_ZOOKEEPER=https://archive.apache.org/dist/zookeeper/zookeeper-3.4.14/zookeeper-3.4.14.tar.gz
41 |
42 | SERVICE_WAIT_TIMEOUT_SEC=20
43 | ZOOKEEPER_PORT=2181
44 | RESOURCEMANAGER_PORT=8032
45 | NODEMANAGER_PORT=8042
46 | KAFKA_PORT=9092
47 |
48 | bootstrap() {
49 | echo "Bootstrapping the system..."
50 | stop_all
51 | rm -rf "$DEPLOY_ROOT_DIR"
52 | mkdir "$DEPLOY_ROOT_DIR"
53 | install_all
54 | start_all
55 | exit 0
56 | }
57 |
58 | standalone() {
59 | echo "Setting up the system..."
60 | stop_all
61 | rm -rf "$DEPLOY_ROOT_DIR"
62 | mkdir "$DEPLOY_ROOT_DIR"
63 | install_all_without_yarn
64 | start_all_without_yarn
65 | exit 0
66 | }
67 |
68 | install_all() {
69 | $DIR/grid install samza
70 | $DIR/grid install zookeeper
71 | $DIR/grid install yarn
72 | $DIR/grid install kafka
73 | }
74 |
75 | install_all_without_yarn() {
76 | $DIR/grid install samza
77 | $DIR/grid install zookeeper
78 | $DIR/grid install kafka
79 | }
80 |
81 | install_samza() {
82 | echo "Building samza from master..."
83 | mkdir -p "$DEPLOY_ROOT_DIR"
84 | if [ -d "$DOWNLOAD_CACHE_DIR/samza/.git" ]; then
85 | pushd "$DOWNLOAD_CACHE_DIR/samza"
86 | git fetch origin
87 | git reset --hard origin/master
88 | else
89 | mkdir -p $DOWNLOAD_CACHE_DIR
90 | pushd $DOWNLOAD_CACHE_DIR
91 | git clone https://gitbox.apache.org/repos/asf/samza.git
92 | cd samza
93 | fi
94 | ./gradlew -PscalaSuffix=2.11 clean publishToMavenLocal
95 | popd
96 | }
97 |
98 | install_zookeeper() {
99 | mkdir -p "$DEPLOY_ROOT_DIR"
100 | install zookeeper $DOWNLOAD_ZOOKEEPER zookeeper-3.4.14
101 | cp "$DEPLOY_ROOT_DIR/zookeeper/conf/zoo_sample.cfg" "$DEPLOY_ROOT_DIR/zookeeper/conf/zoo.cfg"
102 | }
103 |
104 | install_yarn() {
105 | mkdir -p "$DEPLOY_ROOT_DIR"
106 | install yarn $DOWNLOAD_YARN hadoop-2.9.2
107 | cp "$BASE_DIR/conf/yarn-site.xml" "$DEPLOY_ROOT_DIR/yarn/etc/hadoop/yarn-site.xml"
108 | if [ ! -f "$HOME/.samza/conf/yarn-site.xml" ]; then
109 | mkdir -p "$HOME/.samza/conf"
110 | cp "$BASE_DIR/conf/yarn-site.xml" "$HOME/.samza/conf/yarn-site.xml"
111 | fi
112 | }
113 |
114 | install_kafka() {
115 | mkdir -p "$DEPLOY_ROOT_DIR"
116 | install kafka $DOWNLOAD_KAFKA kafka_2.11-2.1.1
117 | # have to use SIGTERM since nohup on appears to ignore SIGINT
118 | # and Kafka switched to SIGINT in KAFKA-1031.
119 | sed -i.bak 's/SIGINT/SIGTERM/g' $DEPLOY_ROOT_DIR/kafka/bin/kafka-server-stop.sh
120 | # in order to simplify the wikipedia-stats example job, set topic to have just 1 partition by default
121 | sed -i.bak 's/^num\.partitions *=.*/num.partitions=1/' $DEPLOY_ROOT_DIR/kafka/config/server.properties
122 | }
123 |
124 | install() {
125 | DESTINATION_DIR="$DEPLOY_ROOT_DIR/$1"
126 | DOWNLOAD_URL=$2
127 | PACKAGE_DIR="$DOWNLOAD_CACHE_DIR/$3"
128 | PACKAGE_FILE="$DOWNLOAD_CACHE_DIR/$(basename $DOWNLOAD_URL)"
129 | if [ -f "$PACKAGE_FILE" ]; then
130 | echo "Using previously downloaded file $PACKAGE_FILE"
131 | else
132 | echo "Downloading $(basename $DOWNLOAD_URL)..."
133 | mkdir -p $DOWNLOAD_CACHE_DIR
134 | curl "$DOWNLOAD_URL" > "${PACKAGE_FILE}.tmp"
135 | mv "${PACKAGE_FILE}.tmp" "$PACKAGE_FILE"
136 | fi
137 | rm -rf "$DESTINATION_DIR" "$PACKAGE_DIR"
138 | tar -xf "$PACKAGE_FILE" -C $DOWNLOAD_CACHE_DIR
139 | mv "$PACKAGE_DIR" "$DESTINATION_DIR"
140 | }
141 |
142 | start_all() {
143 | $DIR/grid start zookeeper
144 | $DIR/grid start yarn
145 | $DIR/grid start kafka
146 | }
147 |
148 | start_all_without_yarn() {
149 | $DIR/grid start zookeeper
150 | $DIR/grid start kafka
151 | }
152 |
153 | start_zookeeper() {
154 | if [ -f $DEPLOY_ROOT_DIR/$SYSTEM/bin/zkServer.sh ]; then
155 | cd $DEPLOY_ROOT_DIR/$SYSTEM
156 | bin/zkServer.sh start
157 | wait_for_service "zookeeper" $ZOOKEEPER_PORT
158 | cd - > /dev/null
159 | else
160 | echo 'Zookeeper is not installed. Run: bin/grid install zookeeper'
161 | fi
162 | }
163 |
164 | start_yarn() {
165 | if [ -f $DEPLOY_ROOT_DIR/$SYSTEM/sbin/yarn-daemon.sh ]; then
166 | $DEPLOY_ROOT_DIR/$SYSTEM/sbin/yarn-daemon.sh start resourcemanager
167 | wait_for_service "resourcemanager" $RESOURCEMANAGER_PORT
168 | $DEPLOY_ROOT_DIR/$SYSTEM/sbin/yarn-daemon.sh start nodemanager
169 | wait_for_service "nodemanager" $NODEMANAGER_PORT
170 | else
171 | echo 'YARN is not installed. Run: bin/grid install yarn'
172 | fi
173 | }
174 |
175 | start_kafka() {
176 | if [ -f $DEPLOY_ROOT_DIR/$SYSTEM/bin/kafka-server-start.sh ]; then
177 | mkdir -p $DEPLOY_ROOT_DIR/$SYSTEM/logs
178 | cd $DEPLOY_ROOT_DIR/$SYSTEM
179 | nohup bin/kafka-server-start.sh config/server.properties > logs/kafka.log 2>&1 &
180 | cd - > /dev/null
181 | wait_for_service "kafka" $KAFKA_PORT
182 | else
183 | echo 'Kafka is not installed. Run: bin/grid install kafka'
184 | fi
185 | }
186 |
187 | wait_for_service() {
188 | local SERVICE_NAME=$1
189 | local PORT=$2
190 | echo "Waiting for $SERVICE_NAME to start..."
191 | local CURRENT_WAIT_TIME=0
192 |
193 | while [[ $(echo | nc -w1 localhost $PORT >/dev/null 2>&1 ;echo $?) -ne 0 ]]; do
194 | printf '.'
195 | sleep 1
196 | if [ $((++CURRENT_WAIT_TIME)) -eq $SERVICE_WAIT_TIMEOUT_SEC ]; then
197 | printf "\nError: timed out while waiting for $SERVICE_NAME to start.\n"
198 | exit 1
199 | fi
200 | done
201 | printf '\n'
202 | echo "$SERVICE_NAME has started";
203 | }
204 |
205 | stop_all() {
206 | $DIR/grid stop kafka
207 | $DIR/grid stop yarn
208 | $DIR/grid stop zookeeper
209 | }
210 |
211 | stop_zookeeper() {
212 | if [ -f $DEPLOY_ROOT_DIR/$SYSTEM/bin/zkServer.sh ]; then
213 | cd $DEPLOY_ROOT_DIR/$SYSTEM
214 | bin/zkServer.sh stop
215 | cd - > /dev/null
216 | else
217 | echo 'Zookeeper is not installed. Run: bin/grid install zookeeper'
218 | fi
219 | }
220 |
221 | stop_yarn() {
222 | if [ -f $DEPLOY_ROOT_DIR/$SYSTEM/sbin/yarn-daemon.sh ]; then
223 | $DEPLOY_ROOT_DIR/$SYSTEM/sbin/yarn-daemon.sh stop resourcemanager
224 | $DEPLOY_ROOT_DIR/$SYSTEM/sbin/yarn-daemon.sh stop nodemanager
225 | else
226 | echo 'YARN is not installed. Run: bin/grid install yarn'
227 | fi
228 | }
229 |
230 | stop_kafka() {
231 | if [ -f $DEPLOY_ROOT_DIR/$SYSTEM/bin/kafka-server-stop.sh ]; then
232 | cd $DEPLOY_ROOT_DIR/$SYSTEM
233 | bin/kafka-server-stop.sh || true # tolerate nonzero exit status if Kafka isn't running
234 | cd - > /dev/null
235 | else
236 | echo 'Kafka is not installed. Run: bin/grid install kafka'
237 | fi
238 | }
239 |
240 | # Check arguments
241 | if [ "$COMMAND" == "bootstrap" ] && test -z "$SYSTEM"; then
242 | bootstrap
243 | exit 0
244 | elif [ "$COMMAND" == "standalone" ] && test -z "$SYSTEM"; then
245 | standalone
246 | exit 0
247 | elif (test -z "$COMMAND" && test -z "$SYSTEM") \
248 | || ( [ "$COMMAND" == "help" ] || test -z "$COMMAND" || test -z "$SYSTEM"); then
249 | echo
250 | echo " Usage.."
251 | echo
252 | echo " $ grid"
253 | echo " $ grid bootstrap"
254 | echo " $ grid standalone"
255 | echo " $ grid install [yarn|kafka|zookeeper|samza|all]"
256 | echo " $ grid start [yarn|kafka|zookeeper|all]"
257 | echo " $ grid stop [yarn|kafka|zookeeper|all]"
258 | echo
259 | exit 1
260 | else
261 | echo "EXECUTING: $COMMAND $SYSTEM"
262 |
263 | "$COMMAND"_"$SYSTEM"
264 | fi
265 |
--------------------------------------------------------------------------------
/src/main/java/samza/examples/cookbook/RemoteTableJoinExample.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,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 | package samza.examples.cookbook;
20 |
21 | import com.google.common.collect.ImmutableList;
22 | import com.google.common.collect.ImmutableMap;
23 |
24 | import java.io.Serializable;
25 | import java.net.URL;
26 | import java.time.Duration;
27 | import java.util.List;
28 | import java.util.Map;
29 | import java.util.concurrent.CompletableFuture;
30 | import org.apache.samza.SamzaException;
31 | import org.apache.samza.application.StreamApplication;
32 | import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
33 | import org.apache.samza.operators.KV;
34 | import org.apache.samza.operators.MessageStream;
35 | import org.apache.samza.operators.OutputStream;
36 | import org.apache.samza.operators.functions.StreamTableJoinFunction;
37 | import org.apache.samza.serializers.JsonSerdeV2;
38 | import org.apache.samza.serializers.StringSerde;
39 | import org.apache.samza.system.kafka.descriptors.KafkaInputDescriptor;
40 | import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
41 | import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
42 | import org.apache.samza.table.Table;
43 | import org.apache.samza.table.descriptors.CachingTableDescriptor;
44 | import org.apache.samza.table.remote.BaseTableFunction;
45 | import org.apache.samza.table.remote.TableReadFunction;
46 | import org.apache.samza.table.descriptors.RemoteTableDescriptor;
47 | import org.apache.samza.util.ExponentialSleepStrategy;
48 | import org.apache.samza.util.HttpUtil;
49 | import org.codehaus.jackson.JsonFactory;
50 | import org.codehaus.jackson.JsonParser;
51 | import org.codehaus.jackson.JsonToken;
52 | import org.codehaus.jackson.annotate.JsonProperty;
53 |
54 | /**
55 | * In this example, we join a stream of stock symbols with a remote table backed by a RESTful service,
56 | * which delivers latest stock quotes. The join results contain stock symbol and latest price, and are
57 | * delivered to an output stream.
58 | *
59 | * A rate limit of 10 requests/second is set of the entire job, internally Samza uses an embedded
60 | * rate limiter, which evenly distributes the total rate limit among tasks.
61 | *
62 | * A caching table is used over the remote table with a read TTL of 5 seconds, therefore one would
63 | * receive the same quote with this time span.
64 | *
65 | * Concepts covered: remote table, rate limiter, caching table, stream to table joins.
66 | *
67 | * To run the below example:
68 | *
69 | *
70 | * -
71 | * Create Kafka topics "stock-symbol-input", "stock-price-output" are created
72 | * ./deploy/kafka/bin/kafka-topics.sh --zookeeper localhost:2181 --create --topic stock-symbol-input --partitions 2 --replication-factor 1
73 | * ./deploy/kafka/bin/kafka-topics.sh --zookeeper localhost:2181 --create --topic stock-price-output --partitions 2 --replication-factor 1
74 | *
75 | * -
76 | * Run the application using the run-app.sh script
77 | * ./deploy/samza/bin/run-app.sh --config-path=$PWD/deploy/samza/config/remote-table-join-example.properties
78 | *
79 | * -
80 | * Consume messages from the output topic
81 | * ./deploy/kafka/bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic stock-price-output
82 | *
83 | * -
84 | * Produce some messages to the input topic
85 | * ./deploy/kafka/bin/kafka-console-producer.sh --topic stock-symbol-input --broker-list localhost:9092
86 | *
87 | * After the console producer is started, type
88 | * MSFT
89 | *
90 | * You should see messages like below from the console consumer window
91 | * {"symbol":"MSFT","close":107.64}
92 | *
93 | * Note: you will need a free API key for symbols other than MSFT, see below for more information.
94 | *
95 | *
96 | *
97 | */
98 | public class RemoteTableJoinExample implements StreamApplication {
99 | private static final String KAFKA_SYSTEM_NAME = "kafka";
100 | private static final List KAFKA_CONSUMER_ZK_CONNECT = ImmutableList.of("localhost:2181");
101 | private static final List KAFKA_PRODUCER_BOOTSTRAP_SERVERS = ImmutableList.of("localhost:9092");
102 | private static final Map KAFKA_DEFAULT_STREAM_CONFIGS = ImmutableMap.of("replication.factor", "1");
103 |
104 | /**
105 | * Default API key "demo" only works for symbol "MSFT"; however you can get an
106 | * API key for free at https://www.alphavantage.co/, which will work for other symbols.
107 | */
108 | private static final String API_KEY = "demo";
109 |
110 | private static final String URL_TEMPLATE =
111 | "https://www.alphavantage.co/query?function=TIME_SERIES_DAILY&symbol=%s&apikey=" + API_KEY;
112 |
113 | private static final String INPUT_STREAM_ID = "stock-symbol-input";
114 | private static final String OUTPUT_STREAM_ID = "stock-price-output";
115 |
116 | @Override
117 | public void describe(StreamApplicationDescriptor appDescriptor) {
118 | KafkaSystemDescriptor kafkaSystemDescriptor = new KafkaSystemDescriptor(KAFKA_SYSTEM_NAME)
119 | .withConsumerZkConnect(KAFKA_CONSUMER_ZK_CONNECT)
120 | .withProducerBootstrapServers(KAFKA_PRODUCER_BOOTSTRAP_SERVERS)
121 | .withDefaultStreamConfigs(KAFKA_DEFAULT_STREAM_CONFIGS);
122 |
123 | KafkaInputDescriptor stockSymbolInputDescriptor =
124 | kafkaSystemDescriptor.getInputDescriptor(INPUT_STREAM_ID, new StringSerde());
125 | KafkaOutputDescriptor stockPriceOutputDescriptor =
126 | kafkaSystemDescriptor.getOutputDescriptor(OUTPUT_STREAM_ID, new JsonSerdeV2<>(StockPrice.class));
127 | appDescriptor.withDefaultSystem(kafkaSystemDescriptor);
128 | MessageStream stockSymbolStream = appDescriptor.getInputStream(stockSymbolInputDescriptor);
129 | OutputStream stockPriceStream = appDescriptor.getOutputStream(stockPriceOutputDescriptor);
130 |
131 | RemoteTableDescriptor remoteTableDescriptor =
132 | new RemoteTableDescriptor("remote-table")
133 | .withReadRateLimit(10)
134 | .withReadFunction(new StockPriceReadFunction());
135 | CachingTableDescriptor cachedRemoteTableDescriptor =
136 | new CachingTableDescriptor<>("cached-remote-table", remoteTableDescriptor)
137 | .withReadTtl(Duration.ofSeconds(5));
138 | Table> cachedRemoteTable = appDescriptor.getTable(cachedRemoteTableDescriptor);
139 |
140 | stockSymbolStream
141 | .map(symbol -> new KV(symbol, null))
142 | .join(cachedRemoteTable, new JoinFn())
143 | .sendTo(stockPriceStream);
144 |
145 | }
146 |
147 | static class JoinFn implements StreamTableJoinFunction, KV, StockPrice> {
148 | @Override
149 | public StockPrice apply(KV message, KV record) {
150 | return record == null ? null : new StockPrice(message.getKey(), record.getValue());
151 | }
152 | @Override
153 | public String getMessageKey(KV message) {
154 | return message.getKey();
155 | }
156 | @Override
157 | public String getRecordKey(KV record) {
158 | return record.getKey();
159 | }
160 | }
161 |
162 | static class StockPriceReadFunction extends BaseTableFunction
163 | implements TableReadFunction {
164 | @Override
165 | public CompletableFuture getAsync(String symbol) {
166 | return CompletableFuture.supplyAsync(() -> {
167 | try {
168 | URL url = new URL(String.format(URL_TEMPLATE, symbol));
169 | String response = HttpUtil.read(url, 5000, new ExponentialSleepStrategy());
170 | JsonParser parser = new JsonFactory().createJsonParser(response);
171 | while (!parser.isClosed()) {
172 | if (JsonToken.FIELD_NAME.equals(parser.nextToken()) && "4. close".equalsIgnoreCase(parser.getCurrentName())) {
173 | return Double.valueOf(parser.nextTextValue());
174 | }
175 | }
176 | return -1d;
177 | } catch (Exception ex) {
178 | throw new SamzaException(ex);
179 | }
180 | });
181 | }
182 |
183 | @Override
184 | public boolean isRetriable(Throwable throwable) {
185 | return false;
186 | }
187 | }
188 |
189 | public static class StockPrice implements Serializable {
190 |
191 | public final String symbol;
192 | public final Double close;
193 |
194 | public StockPrice(
195 | @JsonProperty("symbol") String symbol,
196 | @JsonProperty("close") Double close) {
197 | this.symbol = symbol;
198 | this.close = close;
199 | }
200 | }
201 |
202 | }
203 |
--------------------------------------------------------------------------------
/src/main/java/samza/examples/wikipedia/system/WikipediaFeed.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,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package samza.examples.wikipedia.system;
21 |
22 | import java.io.IOException;
23 | import java.util.HashMap;
24 | import java.util.HashSet;
25 | import java.util.Map;
26 | import java.util.Random;
27 | import java.util.Set;
28 | import org.apache.samza.SamzaException;
29 | import org.codehaus.jackson.map.ObjectMapper;
30 | import org.schwering.irc.lib.IRCConnection;
31 | import org.schwering.irc.lib.IRCEventListener;
32 | import org.schwering.irc.lib.IRCModeParser;
33 | import org.schwering.irc.lib.IRCUser;
34 | import org.slf4j.Logger;
35 | import org.slf4j.LoggerFactory;
36 |
37 | public class WikipediaFeed {
38 | private static final Logger log = LoggerFactory.getLogger(WikipediaFeed.class);
39 | private static final Random random = new Random();
40 | private static final ObjectMapper jsonMapper = new ObjectMapper();
41 |
42 | private final Map> channelListeners;
43 | private final String host;
44 | private final int port;
45 | private final IRCConnection conn;
46 | private final String nick;
47 |
48 | public WikipediaFeed(String host, int port) {
49 | this.channelListeners = new HashMap>();
50 | this.host = host;
51 | this.port = port;
52 | this.nick = "samza-bot-" + Math.abs(random.nextInt());
53 | this.conn = new IRCConnection(host, new int[] { port }, "", nick, nick, nick);
54 | this.conn.addIRCEventListener(new WikipediaFeedIrcListener());
55 | this.conn.setEncoding("UTF-8");
56 | this.conn.setPong(true);
57 | this.conn.setColors(false);
58 | }
59 |
60 | public void start() {
61 | try {
62 | this.conn.connect();
63 | } catch (IOException e) {
64 | throw new RuntimeException("Unable to connect to " + host + ":" + port + ".", e);
65 | }
66 | }
67 |
68 | public void stop() {
69 | this.conn.interrupt();
70 |
71 | try {
72 | this.conn.join();
73 | } catch (InterruptedException e) {
74 | throw new RuntimeException("Interrupted while trying to shutdown IRC connection for " + host + ":" + port, e);
75 | }
76 |
77 | if (this.conn.isAlive()) {
78 | throw new RuntimeException("Unable to shutdown IRC connection for " + host + ":" + port);
79 | }
80 | }
81 |
82 | public void listen(String channel, WikipediaFeedListener listener) {
83 | Set listeners = channelListeners.get(channel);
84 |
85 | if (listeners == null) {
86 | listeners = new HashSet();
87 | channelListeners.put(channel, listeners);
88 | join(channel);
89 | }
90 |
91 | listeners.add(listener);
92 | }
93 |
94 | public void unlisten(String channel, WikipediaFeedListener listener) {
95 | Set listeners = channelListeners.get(channel);
96 |
97 | if (listeners == null) {
98 | throw new RuntimeException("Trying to unlisten to a channel that has no listeners in it.");
99 | } else if (!listeners.contains(listener)) {
100 | throw new RuntimeException("Trying to unlisten to a channel that listener is not listening to.");
101 | }
102 |
103 | listeners.remove(listener);
104 |
105 | if (listeners.size() == 0) {
106 | leave(channel);
107 | }
108 | }
109 |
110 | public void join(String channel) {
111 | conn.send("JOIN " + channel);
112 | }
113 |
114 | public void leave(String channel) {
115 | conn.send("PART " + channel);
116 | }
117 |
118 | public class WikipediaFeedIrcListener implements IRCEventListener {
119 | public void onRegistered() {
120 | log.info("Connected");
121 | }
122 |
123 | public void onDisconnected() {
124 | log.info("Disconnected");
125 | }
126 |
127 | public void onError(String msg) {
128 | log.info("Error: " + msg);
129 | }
130 |
131 | public void onError(int num, String msg) {
132 | log.info("Error #" + num + ": " + msg);
133 | }
134 |
135 | public void onInvite(String chan, IRCUser u, String nickPass) {
136 | log.info(chan + "> " + u.getNick() + " invites " + nickPass);
137 | }
138 |
139 | public void onJoin(String chan, IRCUser u) {
140 | log.info(chan + "> " + u.getNick() + " joins");
141 | }
142 |
143 | public void onKick(String chan, IRCUser u, String nickPass, String msg) {
144 | log.info(chan + "> " + u.getNick() + " kicks " + nickPass);
145 | }
146 |
147 | public void onMode(IRCUser u, String nickPass, String mode) {
148 | log.info("Mode: " + u.getNick() + " sets modes " + mode + " " + nickPass);
149 | }
150 |
151 | public void onMode(String chan, IRCUser u, IRCModeParser mp) {
152 | log.info(chan + "> " + u.getNick() + " sets mode: " + mp.getLine());
153 | }
154 |
155 | public void onNick(IRCUser u, String nickNew) {
156 | log.info("Nick: " + u.getNick() + " is now known as " + nickNew);
157 | }
158 |
159 | public void onNotice(String target, IRCUser u, String msg) {
160 | log.info(target + "> " + u.getNick() + " (notice): " + msg);
161 | }
162 |
163 | public void onPart(String chan, IRCUser u, String msg) {
164 | log.info(chan + "> " + u.getNick() + " parts");
165 | }
166 |
167 | public void onPrivmsg(String chan, IRCUser u, String msg) {
168 | Set listeners = channelListeners.get(chan);
169 |
170 | if (listeners != null) {
171 | WikipediaFeedEvent event = new WikipediaFeedEvent(System.currentTimeMillis(), chan, u.getNick(), msg);
172 |
173 | for (WikipediaFeedListener listener : listeners) {
174 | listener.onEvent(event);
175 | }
176 | }
177 |
178 | log.debug(chan + "> " + u.getNick() + ": " + msg);
179 | }
180 |
181 | public void onQuit(IRCUser u, String msg) {
182 | log.info("Quit: " + u.getNick());
183 | }
184 |
185 | public void onReply(int num, String value, String msg) {
186 | log.info("Reply #" + num + ": " + value + " " + msg);
187 | }
188 |
189 | public void onTopic(String chan, IRCUser u, String topic) {
190 | log.info(chan + "> " + u.getNick() + " changes topic into: " + topic);
191 | }
192 |
193 | public void onPing(String p) {
194 | }
195 |
196 | public void unknown(String a, String b, String c, String d) {
197 | log.warn("UNKNOWN: " + a + " " + b + " " + c + " " + d);
198 | }
199 | }
200 |
201 | public static interface WikipediaFeedListener {
202 | void onEvent(WikipediaFeedEvent event);
203 | }
204 |
205 | public static final class WikipediaFeedEvent {
206 | private final long time;
207 | private final String channel;
208 | private final String source;
209 | private final String rawEvent;
210 |
211 | public WikipediaFeedEvent(long time, String channel, String source, String rawEvent) {
212 | this.time = time;
213 | this.channel = channel;
214 | this.source = source;
215 | this.rawEvent = rawEvent;
216 | }
217 |
218 | public WikipediaFeedEvent(Map jsonObject) {
219 | this((Long) jsonObject.get("time"), (String) jsonObject.get("channel"), (String) jsonObject.get("source"), (String) jsonObject.get("raw"));
220 | }
221 |
222 | public long getTime() {
223 | return time;
224 | }
225 |
226 | public String getChannel() {
227 | return channel;
228 | }
229 |
230 | public String getSource() {
231 | return source;
232 | }
233 |
234 | public String getRawEvent() {
235 | return rawEvent;
236 | }
237 |
238 | @Override
239 | public int hashCode() {
240 | final int prime = 31;
241 | int result = 1;
242 | result = prime * result + ((channel == null) ? 0 : channel.hashCode());
243 | result = prime * result + ((rawEvent == null) ? 0 : rawEvent.hashCode());
244 | result = prime * result + ((source == null) ? 0 : source.hashCode());
245 | result = prime * result + (int) (time ^ (time >>> 32));
246 | return result;
247 | }
248 |
249 | @Override
250 | public boolean equals(Object obj) {
251 | if (this == obj)
252 | return true;
253 | if (obj == null)
254 | return false;
255 | if (getClass() != obj.getClass())
256 | return false;
257 | WikipediaFeedEvent other = (WikipediaFeedEvent) obj;
258 | if (channel == null) {
259 | if (other.channel != null)
260 | return false;
261 | } else if (!channel.equals(other.channel))
262 | return false;
263 | if (rawEvent == null) {
264 | if (other.rawEvent != null)
265 | return false;
266 | } else if (!rawEvent.equals(other.rawEvent))
267 | return false;
268 | if (source == null) {
269 | if (other.source != null)
270 | return false;
271 | } else if (!source.equals(other.source))
272 | return false;
273 | if (time != other.time)
274 | return false;
275 | return true;
276 | }
277 |
278 | @Override
279 | public String toString() {
280 | return "WikipediaFeedEvent [time=" + time + ", channel=" + channel + ", source=" + source + ", rawEvent=" + rawEvent + "]";
281 | }
282 |
283 | public String toJson() {
284 | return toJson(this);
285 | }
286 |
287 | public static Map toMap(WikipediaFeedEvent event) {
288 | Map jsonObject = new HashMap();
289 |
290 | jsonObject.put("time", event.getTime());
291 | jsonObject.put("channel", event.getChannel());
292 | jsonObject.put("source", event.getSource());
293 | jsonObject.put("raw", event.getRawEvent());
294 |
295 | return jsonObject;
296 | }
297 |
298 | public static String toJson(WikipediaFeedEvent event) {
299 | Map jsonObject = toMap(event);
300 |
301 | try {
302 | return jsonMapper.writeValueAsString(jsonObject);
303 | } catch (Exception e) {
304 | throw new SamzaException(e);
305 | }
306 | }
307 |
308 | @SuppressWarnings("unchecked")
309 | public static WikipediaFeedEvent fromJson(String json) {
310 | try {
311 | return new WikipediaFeedEvent((Map) jsonMapper.readValue(json, Map.class));
312 | } catch (Exception e) {
313 | throw new SamzaException(e);
314 | }
315 | }
316 | }
317 |
318 | public static void main(String[] args) throws InterruptedException {
319 | WikipediaFeed feed = new WikipediaFeed("irc.wikimedia.org", 6667);
320 | feed.start();
321 |
322 | feed.listen("#en.wikipedia", new WikipediaFeedListener() {
323 | @Override
324 | public void onEvent(WikipediaFeedEvent event) {
325 | System.out.println(event);
326 | }
327 | });
328 |
329 | Thread.sleep(20000);
330 | feed.stop();
331 | }
332 | }
333 |
--------------------------------------------------------------------------------
/src/main/java/samza/examples/cookbook/CouchbaseTableExample.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,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 | package samza.examples.cookbook;
20 |
21 | import com.couchbase.client.java.document.json.JsonObject;
22 | import com.google.common.base.Preconditions;
23 | import com.google.common.collect.ImmutableList;
24 | import com.google.common.collect.ImmutableMap;
25 | import java.text.SimpleDateFormat;
26 | import java.time.Duration;
27 | import java.util.Arrays;
28 | import java.util.Date;
29 | import java.util.List;
30 | import java.util.Map;
31 | import java.util.concurrent.CompletableFuture;
32 | import java.util.concurrent.TimeUnit;
33 | import org.apache.samza.SamzaException;
34 | import org.apache.samza.application.StreamApplication;
35 | import org.apache.samza.application.descriptors.StreamApplicationDescriptor;
36 | import org.apache.samza.context.Context;
37 | import org.apache.samza.operators.MessageStream;
38 | import org.apache.samza.operators.OutputStream;
39 | import org.apache.samza.operators.functions.MapFunction;
40 | import org.apache.samza.serializers.StringSerde;
41 | import org.apache.samza.system.kafka.descriptors.KafkaInputDescriptor;
42 | import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor;
43 | import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor;
44 | import org.apache.samza.table.descriptors.RemoteTableDescriptor;
45 | import org.apache.samza.table.remote.NoOpTableReadFunction;
46 | import org.apache.samza.table.remote.RemoteTable;
47 | import org.apache.samza.table.remote.couchbase.CouchbaseTableWriteFunction;
48 | import org.apache.samza.table.retry.TableRetryPolicy;
49 |
50 |
51 | /**
52 | * This is a simple word count example using a remote store.
53 | *
54 | * In this example, we use Couchbase to demonstrate how to invoke API's on a remote store other than get, put or delete
55 | * as defined in {@link org.apache.samza.table.remote.AsyncRemoteTable}. Input messages are collected from user through
56 | * a Kafka console producer, and tokenized using space. For each word, we increment a counter for this word
57 | * as well as a counter for all words on Couchbase. We also output the current value of both counters to Kafka console
58 | * consumer.
59 | *
60 | * A rate limit of 4 requests/second to Couchbase is set of the entire job, internally Samza uses an embedded
61 | * rate limiter, which evenly distributes the total rate limit among tasks. As we invoke 2 calls on Couchbase
62 | * for each word, you should see roughly 2 messages per second in the Kafka console consumer
63 | * window.
64 | *
65 | * A retry policy with 1 second fixed backoff time and max 3 retries is attached to the remote table.
66 | *
67 | * Concepts covered: remote table, rate limiter, retry, arbitrary operation on remote store.
68 | *
69 | * To run the below example:
70 | *
71 | *
72 | * -
73 | * Create a Couchbase instance using docker; Log into the admin UI at http://localhost:8091 (Administrator/password)
74 | * create a bucket called "my-bucket"
75 | * Under Security tab, create a user with the same name, set 123456 as the password, and give it "Data Reader"
76 | * and "Data Writer" privilege for this bucket.
77 | * More information can be found at https://docs.couchbase.com/server/current/getting-started/do-a-quick-install.html
78 | *
79 | * -
80 | * Create Kafka topics "word-input" and "count-output"
81 | * ./deploy/kafka/bin/kafka-topics.sh --zookeeper localhost:2181 --create --topic word-input --partitions 2 --replication-factor 1
82 | * ./deploy/kafka/bin/kafka-topics.sh --zookeeper localhost:2181 --create --topic count-output --partitions 2 --replication-factor 1
83 | *
84 | * -
85 | * Run the application using the run-app.sh script
86 | * ./deploy/samza/bin/run-app.sh --config-path=$PWD/deploy/samza/config/couchbase-table-example.properties
87 | *
88 | * -
89 | * Consume messages from the output topic
90 | * ./deploy/kafka/bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic count-output
91 | *
92 | * -
93 | * Produce some messages to the input topic
94 | * ./deploy/kafka/bin/kafka-console-producer.sh --topic word-input --broker-list localhost:9092
95 | *
96 | * After the console producer is started, type
97 | * 1
98 | * 2
99 | * 3
100 | * 4
101 | * 5
102 | * 4
103 | * 3
104 | * 2
105 | * 1
106 | *
107 | * You should see messages like below from the console consumer window
108 | *
109 | * 2019-05-23 21:18:07 2019-05-23 21:18:07 word=2, count=1, total-count=1
110 | * 2019-05-23 21:18:07 2019-05-23 21:18:07 word=1, count=1, total-count=2
111 | * 2019-05-23 21:18:07 2019-05-23 21:18:07 word=4, count=1, total-count=3
112 | * 2019-05-23 21:18:07 2019-05-23 21:18:07 word=3, count=1, total-count=4
113 | * 2019-05-23 21:18:08 2019-05-23 21:18:08 word=4, count=2, total-count=5
114 | * 2019-05-23 21:18:08 2019-05-23 21:18:08 word=5, count=1, total-count=6
115 | * 2019-05-23 21:18:09 2019-05-23 21:18:09 word=2, count=2, total-count=7
116 | * 2019-05-23 21:18:09 2019-05-23 21:18:09 word=3, count=2, total-count=8
117 | * 2019-05-23 21:18:10 2019-05-23 21:18:10 word=1, count=2, total-count=9
118 | *
119 | * You can examine the result on Couchbase Admin GUI as well.
120 | *
121 | * Note:
122 | * - If you enter "1 2 3 4 5 4 3 2 1", you should see roughly 1 QPS as
123 | * the input is processed by only one task
124 | *
125 | *
126 | *
127 | *
128 | *
129 | */
130 | public class CouchbaseTableExample implements StreamApplication {
131 |
132 | private static final SimpleDateFormat DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
133 |
134 | private static final String KAFKA_SYSTEM_NAME = "kafka";
135 | private static final List KAFKA_CONSUMER_ZK_CONNECT = ImmutableList.of("localhost:2181");
136 | private static final List KAFKA_PRODUCER_BOOTSTRAP_SERVERS = ImmutableList.of("localhost:9092");
137 | private static final Map KAFKA_DEFAULT_STREAM_CONFIGS = ImmutableMap.of("replication.factor", "1");
138 |
139 | private static final String INPUT_STREAM_ID = "word-input";
140 | private static final String OUTPUT_STREAM_ID = "count-output";
141 |
142 | private static final String CLUSTER_NODES = "couchbase://127.0.0.1";
143 | private static final int COUCHBASE_PORT = 11210;
144 | private static final String BUCKET_NAME = "my-bucket";
145 | private static final String BUCKET_PASSWORD = "123456";
146 | private static final String TOTAL_COUNT_ID = "total-count";
147 |
148 | @Override
149 | public void describe(StreamApplicationDescriptor app) {
150 |
151 | KafkaSystemDescriptor kafkaSystemDescriptor = new KafkaSystemDescriptor(KAFKA_SYSTEM_NAME)
152 | .withConsumerZkConnect(KAFKA_CONSUMER_ZK_CONNECT)
153 | .withProducerBootstrapServers(KAFKA_PRODUCER_BOOTSTRAP_SERVERS)
154 | .withDefaultStreamConfigs(KAFKA_DEFAULT_STREAM_CONFIGS);
155 |
156 | KafkaInputDescriptor wordInputDescriptor =
157 | kafkaSystemDescriptor.getInputDescriptor(INPUT_STREAM_ID, new StringSerde());
158 |
159 | KafkaOutputDescriptor countOutputDescriptor =
160 | kafkaSystemDescriptor.getOutputDescriptor(OUTPUT_STREAM_ID, new StringSerde());
161 |
162 | MyCouchbaseTableWriteFunction writeFn = new MyCouchbaseTableWriteFunction(BUCKET_NAME, CLUSTER_NODES)
163 | .withBootstrapCarrierDirectPort(COUCHBASE_PORT)
164 | .withUsernameAndPassword(BUCKET_NAME, BUCKET_PASSWORD)
165 | .withTimeout(Duration.ofSeconds(5));
166 |
167 | TableRetryPolicy retryPolicy = new TableRetryPolicy()
168 | .withFixedBackoff(Duration.ofSeconds(1))
169 | .withStopAfterAttempts(3);
170 |
171 | RemoteTableDescriptor couchbaseTableDescriptor = new RemoteTableDescriptor("couchbase-table")
172 | .withReadFunction(new NoOpTableReadFunction())
173 | .withReadRateLimiterDisabled()
174 | .withWriteFunction(writeFn)
175 | .withWriteRetryPolicy(retryPolicy)
176 | .withWriteRateLimit(4);
177 |
178 | app.withDefaultSystem(kafkaSystemDescriptor);
179 | MessageStream wordStream = app.getInputStream(wordInputDescriptor);
180 | OutputStream countStream = app.getOutputStream(countOutputDescriptor);
181 | app.getTable(couchbaseTableDescriptor);
182 |
183 | wordStream
184 | .flatMap(m -> Arrays.asList(m.split(" ")))
185 | .filter(word -> word != null && word.length() > 0)
186 | .map(new MyCountFunction())
187 | .map(countString -> currentTime() + " " + countString)
188 | .sendTo(countStream);
189 | }
190 |
191 | static class MyCountFunction implements MapFunction {
192 |
193 | private MyCouchbaseTableWriteFunction writeFn;
194 |
195 | @Override
196 | public void init(Context context) {
197 | RemoteTable table = (RemoteTable) context.getTaskContext().getTable("couchbase-table");
198 | writeFn = (MyCouchbaseTableWriteFunction) table.getWriteFunction();
199 | }
200 |
201 | @Override
202 | public String apply(String word) {
203 | CompletableFuture countFuture = writeFn.incCounter(word);
204 | CompletableFuture totalCountFuture = writeFn.incCounter(TOTAL_COUNT_ID);
205 | return String.format("%s word=%s, count=%d, total-count=%d",
206 | currentTime(), word, countFuture.join(), totalCountFuture.join());
207 | }
208 | }
209 |
210 | static class MyCouchbaseTableWriteFunction extends CouchbaseTableWriteFunction {
211 |
212 | private final static int OP_COUNTER = 1;
213 |
214 | public MyCouchbaseTableWriteFunction(String bucketName, String... clusterNodes) {
215 | super(bucketName, JsonObject.class, clusterNodes);
216 | }
217 |
218 | @Override
219 | public CompletableFuture writeAsync(int opId, Object... args) {
220 | switch (opId) {
221 | case OP_COUNTER:
222 | Preconditions.checkArgument(2 == args.length,
223 | String.format("Two arguments (String and int) are expected for counter operation (opId=%d)", opId));
224 | String id = (String) args[0];
225 | int delta = (int) args[1];
226 | return asyncWriteHelper(
227 | bucket.async().counter(id, delta, 1, timeout.toMillis(), TimeUnit.MILLISECONDS),
228 | String.format("Failed to invoke counter with Id %s from bucket %s.", id, bucketName),
229 | false);
230 | default:
231 | throw new SamzaException("Unknown opId: " + opId);
232 | }
233 | }
234 |
235 | public CompletableFuture incCounter(String id) {
236 | return table.writeAsync(OP_COUNTER, id, 1);
237 | }
238 |
239 | }
240 |
241 | private static String currentTime() {
242 | return DATE_FORMAT.format(new Date());
243 | }
244 |
245 | }
246 |
--------------------------------------------------------------------------------