ctx) throws Exception {
56 | ThreadLocalRandom rnd = ThreadLocalRandom.current();
57 | ParetoDistribution paretoDistribution = new ParetoDistribution(numItems, shape);
58 |
59 | LOG.info("Starting transaction generator for {} items and {} sleep", numItems, sleep);
60 |
61 | while (isRunning) {
62 | long nextItemId;
63 | do {
64 | nextItemId = sample(paretoDistribution);
65 | } while (nextItemId > numItems);
66 | String itemId = "item_" + nextItemId;
67 |
68 | int quantity = (int) (Math.round(rnd.nextGaussian() / 2 * 10) * 10) + 5;
69 | if (quantity == 0) {
70 | continue;
71 | }
72 | long transactionId = rnd.nextLong(Long.MAX_VALUE);
73 | synchronized (ctx.getCheckpointLock()) {
74 | ctx.collect(new ItemTransaction(transactionId, System.currentTimeMillis(), itemId, quantity));
75 | }
76 | if (sleep > 0) {
77 | Thread.sleep(sleep);
78 | }
79 | }
80 |
81 | }
82 |
83 | private long sample(ParetoDistribution paretoDistribution) {
84 | return (Math.round(paretoDistribution.sample() - paretoDistribution.getScale()) + 1);
85 | }
86 |
87 | @Override
88 | public void cancel() {
89 | isRunning = false;
90 | }
91 | }
92 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/main/java/com/cloudera/streaming/examples/flink/operators/MaxWatermarkGeneratorSupplier.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Cloudera, Inc. 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, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.cloudera.streaming.examples.flink.operators;
20 |
21 | import org.apache.flink.api.common.eventtime.Watermark;
22 | import org.apache.flink.api.common.eventtime.WatermarkGenerator;
23 | import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier;
24 | import org.apache.flink.api.common.eventtime.WatermarkOutput;
25 |
26 | /**
27 | * Watermark implementation that emits {@link Watermark#MAX_WATERMARK}, basically removing
28 | * this stream from watermark computation.
29 | *
30 | * Should only be used on streams that won't be aggregated to the window.
31 | */
32 | public class MaxWatermarkGeneratorSupplier implements WatermarkGeneratorSupplier {
33 |
34 | @Override
35 | public WatermarkGenerator createWatermarkGenerator(Context context) {
36 | return new WatermarkGenerator() {
37 | @Override
38 | public void onEvent(T t, long l, WatermarkOutput watermarkOutput) {
39 | watermarkOutput.emitWatermark(Watermark.MAX_WATERMARK);
40 | }
41 |
42 | @Override
43 | public void onPeriodicEmit(WatermarkOutput watermarkOutput) {
44 | watermarkOutput.emitWatermark(Watermark.MAX_WATERMARK);
45 | }
46 | };
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/main/java/com/cloudera/streaming/examples/flink/operators/QueryGeneratorSource.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Cloudera, Inc. 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, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.cloudera.streaming.examples.flink.operators;
20 |
21 | import org.apache.flink.api.java.utils.ParameterTool;
22 | import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
23 |
24 | import com.cloudera.streaming.examples.flink.types.ItemTransaction;
25 | import com.cloudera.streaming.examples.flink.types.Query;
26 | import org.slf4j.Logger;
27 | import org.slf4j.LoggerFactory;
28 |
29 | import java.util.concurrent.ThreadLocalRandom;
30 |
31 | /**
32 | * A simple data generator that generates {@link ItemTransaction} data with a skewed itemId distribution
33 | * to better simulate real world access patterns with hot items.
34 | */
35 | public class QueryGeneratorSource implements ParallelSourceFunction {
36 |
37 | public static final String NUM_ITEMS_KEY = "num.items";
38 | public static final String SLEEP_KEY = "sleep";
39 | public static final int DEFAULT_NUM_ITEMS = 1_000;
40 | private static final Logger LOG = LoggerFactory.getLogger(QueryGeneratorSource.class);
41 | private final int numItems;
42 | private final long sleep;
43 | private volatile boolean isRunning = true;
44 |
45 | public QueryGeneratorSource(ParameterTool params) {
46 | this.numItems = params.getInt(NUM_ITEMS_KEY, DEFAULT_NUM_ITEMS);
47 | this.sleep = Math.min(1000, params.getLong(SLEEP_KEY, 1) * 100);
48 | }
49 |
50 | @Override
51 | public void run(SourceContext ctx) throws Exception {
52 | ThreadLocalRandom rnd = ThreadLocalRandom.current();
53 |
54 | LOG.info("Starting query generator for {} items and {} sleep", numItems, sleep);
55 |
56 | while (isRunning) {
57 | String itemId = "item_" + (rnd.nextInt(numItems) + 1);
58 | synchronized (ctx.getCheckpointLock()) {
59 | ctx.collect(new Query(rnd.nextLong(Long.MAX_VALUE), System.currentTimeMillis(), itemId));
60 | }
61 | if (sleep > 0) {
62 | Thread.sleep(sleep);
63 | }
64 | }
65 |
66 | }
67 |
68 | @Override
69 | public void cancel() {
70 | isRunning = false;
71 | }
72 | }
73 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/main/java/com/cloudera/streaming/examples/flink/operators/SummaryAlertingCondition.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Cloudera, Inc. 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, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.cloudera.streaming.examples.flink.operators;
20 |
21 | import org.apache.flink.api.common.functions.FilterFunction;
22 | import org.apache.flink.api.java.utils.ParameterTool;
23 |
24 | import com.cloudera.streaming.examples.flink.types.TransactionSummary;
25 |
26 | /**
27 | * Raises an alert if the ratio of failed transactions is over a ratio, in a large enough sample.
28 | */
29 | public class SummaryAlertingCondition implements FilterFunction {
30 |
31 | public static final String REPORTING_NUMBER_KEY = "transaction.num.min";
32 | public static final String REPORTING_FAILURE_RATE_KEY = "transaction.failure.rate.min";
33 |
34 | private final int minNum;
35 | private final double minFailureRate;
36 |
37 | public SummaryAlertingCondition(ParameterTool params) {
38 | minNum = params.getInt(REPORTING_NUMBER_KEY, 100);
39 | minFailureRate = params.getDouble(REPORTING_FAILURE_RATE_KEY, 0.5);
40 | }
41 |
42 | @Override
43 | public boolean filter(TransactionSummary transactionSummary) {
44 | int total = transactionSummary.numSuccessfulTransactions + transactionSummary.numFailedTransactions;
45 | return total > minNum && (((double) transactionSummary.numFailedTransactions) / total) > minFailureRate;
46 | }
47 | }
48 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/main/java/com/cloudera/streaming/examples/flink/operators/TransactionProcessor.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Cloudera, Inc. 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, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.cloudera.streaming.examples.flink.operators;
20 |
21 | import org.apache.flink.api.common.state.ValueState;
22 | import org.apache.flink.api.common.state.ValueStateDescriptor;
23 | import org.apache.flink.configuration.Configuration;
24 | import org.apache.flink.metrics.Histogram;
25 | import org.apache.flink.streaming.api.functions.co.KeyedCoProcessFunction;
26 | import org.apache.flink.util.Collector;
27 |
28 | import com.cloudera.streaming.examples.flink.KafkaItemTransactionJob;
29 | import com.cloudera.streaming.examples.flink.types.ItemInfo;
30 | import com.cloudera.streaming.examples.flink.types.ItemTransaction;
31 | import com.cloudera.streaming.examples.flink.types.Query;
32 | import com.cloudera.streaming.examples.flink.types.QueryResult;
33 | import com.cloudera.streaming.examples.flink.types.TransactionResult;
34 | import com.cloudera.streaming.examples.flink.utils.ExponentialHistogram;
35 |
36 | /**
37 | * Core transaction and query processing logic. {@link #processElement1(ItemTransaction, Context, Collector)} receives
38 | * transactions and executes them if there is sufficient quantity already stored in the state.
39 | *
40 | * {@link #processElement2(Query, Context, Collector)} receives item queries that just returns the current info for the
41 | * queried item.
42 | *
43 | *
Both processing functions are keyed by the itemId field.
44 | *
45 | *
In addition to the core logic we added custom histogram metrics to track state access time for future optimizations.
46 | */
47 | public class TransactionProcessor extends KeyedCoProcessFunction {
48 |
49 | private transient ValueState itemState;
50 | private transient Histogram itemRead;
51 | private transient Histogram itemWrite;
52 |
53 | @Override
54 | public void processElement1(ItemTransaction transaction, Context ctx, Collector out) throws Exception {
55 | long startTime = System.nanoTime();
56 | ItemInfo info = itemState.value();
57 | itemRead.update(System.nanoTime() - startTime);
58 |
59 | if (info == null) {
60 | info = new ItemInfo(transaction.itemId, 0);
61 | }
62 | int newQuantity = info.quantity + transaction.quantity;
63 |
64 | boolean success = newQuantity >= 0;
65 | if (success) {
66 | info.quantity = newQuantity;
67 | startTime = System.nanoTime();
68 | itemState.update(info);
69 | itemWrite.update(System.nanoTime() - startTime);
70 | }
71 | out.collect(new TransactionResult(transaction, success));
72 | }
73 |
74 | @Override
75 | public void processElement2(Query query, Context ctx, Collector out) throws Exception {
76 | ItemInfo info = itemState.value();
77 | ctx.output(KafkaItemTransactionJob.QUERY_RESULT, new QueryResult(query.queryId, info != null ? info : new ItemInfo(query.itemId, 0)));
78 | }
79 |
80 | @Override
81 | public void open(Configuration parameters) {
82 | // We create state read/write time metrics for later performance tuning
83 | itemRead = getRuntimeContext().getMetricGroup().histogram("ItemRead", new ExponentialHistogram());
84 | itemWrite = getRuntimeContext().getMetricGroup().histogram("ItemWrite", new ExponentialHistogram());
85 |
86 | itemState = getRuntimeContext().getState(new ValueStateDescriptor<>("itemInfo", ItemInfo.class));
87 | }
88 | }
89 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/main/java/com/cloudera/streaming/examples/flink/operators/TransactionSummaryAggregator.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Cloudera, Inc. 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, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.cloudera.streaming.examples.flink.operators;
20 |
21 | import org.apache.flink.api.common.functions.AggregateFunction;
22 |
23 | import com.cloudera.streaming.examples.flink.types.TransactionResult;
24 | import com.cloudera.streaming.examples.flink.types.TransactionSummary;
25 |
26 | /**
27 | * Aggregate function for summarizing the transaction successfulness.
28 | */
29 | public class TransactionSummaryAggregator implements AggregateFunction {
30 |
31 | @Override
32 | public TransactionSummary createAccumulator() {
33 | return new TransactionSummary();
34 | }
35 |
36 | @Override
37 | public TransactionSummary add(TransactionResult tr, TransactionSummary acc) {
38 | acc.itemId = tr.transaction.itemId;
39 | if (tr.success) {
40 | acc.numSuccessfulTransactions++;
41 | } else {
42 | acc.numFailedTransactions++;
43 | }
44 | acc.totalVolume += Math.abs(tr.transaction.quantity);
45 | return acc;
46 | }
47 |
48 | @Override
49 | public TransactionSummary getResult(TransactionSummary acc) {
50 | return acc;
51 | }
52 |
53 | @Override
54 | public TransactionSummary merge(TransactionSummary ts1, TransactionSummary ts2) {
55 | return new TransactionSummary(ts2.itemId != null ? ts2.itemId : ts1.itemId,
56 | ts1.numFailedTransactions + ts2.numFailedTransactions,
57 | ts1.numSuccessfulTransactions + ts2.numSuccessfulTransactions,
58 | ts1.totalVolume + ts2.totalVolume);
59 | }
60 | }
61 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/main/java/com/cloudera/streaming/examples/flink/types/ItemInfo.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Cloudera, Inc. 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, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.cloudera.streaming.examples.flink.types;
20 |
21 | import java.util.Objects;
22 |
23 | /**
24 | * Item stock information.
25 | */
26 | public class ItemInfo {
27 |
28 | public String itemId;
29 |
30 | public int quantity;
31 |
32 | public String itemName = "UNKNOWN";
33 |
34 | public ItemInfo() {
35 | }
36 |
37 | public ItemInfo(String itemId, int quantity) {
38 | this.itemId = itemId;
39 | this.quantity = quantity;
40 | }
41 |
42 | public ItemInfo(String itemId, int quantity, String itemName) {
43 | this.itemId = itemId;
44 | this.quantity = quantity;
45 | this.itemName = itemName;
46 | }
47 |
48 | public void setItemName(String name) {
49 | this.itemName = name;
50 | }
51 |
52 | @Override
53 | public boolean equals(Object o) {
54 | if (this == o) {
55 | return true;
56 | }
57 | if (o == null || getClass() != o.getClass()) {
58 | return false;
59 | }
60 |
61 | ItemInfo itemInfo = (ItemInfo) o;
62 | return quantity == itemInfo.quantity &&
63 | Objects.equals(itemId, itemInfo.itemId) &&
64 | Objects.equals(itemName, itemInfo.itemName);
65 | }
66 |
67 | @Override
68 | public int hashCode() {
69 | return Objects.hash(itemId, quantity, itemName);
70 | }
71 |
72 | @Override
73 | public String toString() {
74 | return "ItemInfo{" +
75 | "itemId='" + itemId + '\'' +
76 | ", quantity=" + quantity +
77 | ", itemName='" + itemName + '\'' +
78 | '}';
79 | }
80 | }
81 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/main/java/com/cloudera/streaming/examples/flink/types/ItemTransaction.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Cloudera, Inc. 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, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.cloudera.streaming.examples.flink.types;
20 |
21 | import java.util.Objects;
22 |
23 | /**
24 | * Transaction request for an item.
25 | */
26 | public class ItemTransaction {
27 |
28 | public long transactionId;
29 |
30 | public long ts;
31 |
32 | public String itemId;
33 |
34 | public int quantity;
35 |
36 | public ItemTransaction() {}
37 |
38 | public ItemTransaction(long transactionId, long ts, String itemId, int quantity) {
39 | this.transactionId = transactionId;
40 | this.ts = ts;
41 | this.itemId = itemId;
42 | this.quantity = quantity;
43 | }
44 |
45 | @Override
46 | public boolean equals(Object o) {
47 | if (this == o) {
48 | return true;
49 | }
50 | if (o == null || getClass() != o.getClass()) {
51 | return false;
52 | }
53 |
54 | ItemTransaction that = (ItemTransaction) o;
55 | return transactionId == that.transactionId &&
56 | ts == that.ts &&
57 | quantity == that.quantity &&
58 | Objects.equals(itemId, that.itemId);
59 | }
60 |
61 | @Override
62 | public int hashCode() {
63 | return Objects.hash(transactionId, ts, itemId, quantity);
64 | }
65 |
66 | @Override
67 | public String toString() {
68 | return "ItemTransaction{" +
69 | "transactionId=" + transactionId +
70 | ", ts=" + ts +
71 | ", itemId='" + itemId + '\'' +
72 | ", quantity=" + quantity +
73 | '}';
74 | }
75 | }
76 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/main/java/com/cloudera/streaming/examples/flink/types/JsonKafkaSerializationSchema.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Cloudera, Inc. 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, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.cloudera.streaming.examples.flink.types;
20 |
21 | import org.apache.flink.api.common.serialization.DeserializationSchema;
22 | import org.apache.flink.api.common.serialization.SerializationSchema;
23 |
24 | import com.fasterxml.jackson.core.JsonProcessingException;
25 | import com.fasterxml.jackson.databind.ObjectMapper;
26 |
27 | /**
28 | * Common serialization logic for JSON schemas.
29 | */
30 | public abstract class JsonKafkaSerializationSchema implements SerializationSchema, DeserializationSchema {
31 |
32 | protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
33 |
34 | protected final String topic;
35 |
36 | protected JsonKafkaSerializationSchema(String topic) {
37 | this.topic = topic;
38 | }
39 |
40 | @Override
41 | public boolean isEndOfStream(T nextElement) {
42 | return false;
43 | }
44 |
45 | @Override
46 | public byte[] serialize(T obj) {
47 | try {
48 | return OBJECT_MAPPER.writeValueAsBytes(obj);
49 | } catch (JsonProcessingException e) {
50 | throw new RuntimeException(e);
51 | }
52 | }
53 | }
54 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/main/java/com/cloudera/streaming/examples/flink/types/Query.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Cloudera, Inc. 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, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.cloudera.streaming.examples.flink.types;
20 |
21 | import java.util.Objects;
22 |
23 | /**
24 | * Query for the actually available quantity of an item.
25 | */
26 | public class Query {
27 |
28 | public long queryId;
29 |
30 | public long ts = System.currentTimeMillis();
31 |
32 | public String itemId;
33 |
34 | public Query() {}
35 |
36 | public Query(long queryId, String itemId) {
37 | this(queryId, null, itemId);
38 | }
39 |
40 | public Query(long queryId, Long ts, String itemId) {
41 | this.queryId = queryId;
42 | this.itemId = itemId;
43 | if (ts != null) {
44 | this.ts = ts;
45 | }
46 | }
47 |
48 | @Override
49 | public boolean equals(Object o) {
50 | if (this == o) {
51 | return true;
52 | }
53 | if (o == null || getClass() != o.getClass()) {
54 | return false;
55 | }
56 |
57 | Query query = (Query) o;
58 | return queryId == query.queryId &&
59 | ts == query.ts &&
60 | Objects.equals(itemId, query.itemId);
61 | }
62 |
63 | @Override
64 | public int hashCode() {
65 | return Objects.hash(queryId, ts, itemId);
66 | }
67 |
68 | @Override
69 | public String toString() {
70 | return "Query{" +
71 | "queryId=" + queryId +
72 | ", ts=" + ts +
73 | ", itemId='" + itemId + '\'' +
74 | '}';
75 | }
76 | }
77 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/main/java/com/cloudera/streaming/examples/flink/types/QueryResult.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Cloudera, Inc. 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, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.cloudera.streaming.examples.flink.types;
20 |
21 | import java.util.Objects;
22 |
23 | /**
24 | * Results for an item quantity query.
25 | */
26 | public class QueryResult {
27 |
28 | public long queryId;
29 |
30 | public ItemInfo itemInfo;
31 |
32 | public QueryResult() {
33 | }
34 |
35 | public QueryResult(long queryId, ItemInfo itemInfo) {
36 | this.queryId = queryId;
37 | this.itemInfo = itemInfo;
38 | }
39 |
40 | @Override
41 | public boolean equals(Object o) {
42 | if (this == o) {
43 | return true;
44 | }
45 | if (o == null || getClass() != o.getClass()) {
46 | return false;
47 | }
48 |
49 | QueryResult that = (QueryResult) o;
50 | return queryId == that.queryId &&
51 | Objects.equals(itemInfo, that.itemInfo);
52 | }
53 |
54 | @Override
55 | public int hashCode() {
56 | return Objects.hash(queryId, itemInfo);
57 | }
58 |
59 | @Override
60 | public String toString() {
61 | return "QueryResult{" +
62 | "queryId=" + queryId +
63 | ", itemInfo=" + itemInfo +
64 | '}';
65 | }
66 | }
67 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/main/java/com/cloudera/streaming/examples/flink/types/QueryResultSchema.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Cloudera, Inc. 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, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.cloudera.streaming.examples.flink.types;
20 |
21 | import org.apache.flink.api.common.typeinfo.TypeHint;
22 | import org.apache.flink.api.common.typeinfo.TypeInformation;
23 |
24 | import java.io.IOException;
25 |
26 | /**
27 | * Query result serialization schema for running the example with kafka.
28 | */
29 | public class QueryResultSchema extends JsonKafkaSerializationSchema {
30 |
31 | public QueryResultSchema(String topic) {
32 | super(topic);
33 | }
34 |
35 | @Override
36 | public QueryResult deserialize(byte[] message) {
37 | try {
38 | return OBJECT_MAPPER.readValue(message, QueryResult.class);
39 | } catch (IOException e) {
40 | return null;
41 | }
42 | }
43 |
44 | @Override
45 | public TypeInformation getProducedType() {
46 | return new TypeHint() {
47 | }.getTypeInfo();
48 | }
49 | }
50 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/main/java/com/cloudera/streaming/examples/flink/types/QuerySchema.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Cloudera, Inc. 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, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.cloudera.streaming.examples.flink.types;
20 |
21 | import org.apache.flink.api.common.typeinfo.TypeHint;
22 | import org.apache.flink.api.common.typeinfo.TypeInformation;
23 |
24 | import java.io.IOException;
25 |
26 | /**
27 | * Query serialization schema for running the example with kafka.
28 | */
29 | public class QuerySchema extends JsonKafkaSerializationSchema {
30 |
31 | public QuerySchema(String topic) {
32 | super(topic);
33 | }
34 |
35 | @Override
36 | public Query deserialize(byte[] message) {
37 | try {
38 | return OBJECT_MAPPER.readValue(message, Query.class);
39 | } catch (IOException e) {
40 | return null;
41 | }
42 | }
43 |
44 | @Override
45 | public TypeInformation getProducedType() {
46 | return new TypeHint() {
47 | }.getTypeInfo();
48 | }
49 | }
50 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/main/java/com/cloudera/streaming/examples/flink/types/TransactionResult.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Cloudera, Inc. 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, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.cloudera.streaming.examples.flink.types;
20 |
21 | import java.util.Objects;
22 |
23 | /**
24 | * The outcome of an item transaction request.
25 | */
26 | public class TransactionResult {
27 |
28 | public ItemTransaction transaction;
29 |
30 | public boolean success;
31 |
32 | public TransactionResult() {}
33 |
34 | public TransactionResult(ItemTransaction transaction, boolean success) {
35 | this.transaction = transaction;
36 | this.success = success;
37 | }
38 |
39 | @Override
40 | public String toString() {
41 | return "ItemTransactionResult{" +
42 | "transaction=" + transaction +
43 | ", success=" + success +
44 | '}';
45 | }
46 |
47 | @Override
48 | public boolean equals(Object o) {
49 | if (this == o) {
50 | return true;
51 | }
52 | if (o == null || getClass() != o.getClass()) {
53 | return false;
54 | }
55 |
56 | TransactionResult that = (TransactionResult) o;
57 | return success == that.success &&
58 | Objects.equals(transaction, that.transaction);
59 | }
60 |
61 | @Override
62 | public int hashCode() {
63 | return Objects.hash(transaction, success);
64 | }
65 | }
66 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/main/java/com/cloudera/streaming/examples/flink/types/TransactionSchema.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Cloudera, Inc. 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, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.cloudera.streaming.examples.flink.types;
20 |
21 | import org.apache.flink.api.common.typeinfo.TypeHint;
22 | import org.apache.flink.api.common.typeinfo.TypeInformation;
23 |
24 | import java.io.IOException;
25 |
26 | /**
27 | * Transaction serialization schema for running the example with kafka.
28 | */
29 | public class TransactionSchema extends JsonKafkaSerializationSchema {
30 |
31 | public TransactionSchema(String topic) {
32 | super(topic);
33 | }
34 |
35 | @Override
36 | public ItemTransaction deserialize(byte[] message) {
37 | try {
38 | return OBJECT_MAPPER.readValue(message, ItemTransaction.class);
39 | } catch (IOException e) {
40 | return null;
41 | }
42 | }
43 |
44 | @Override
45 | public TypeInformation getProducedType() {
46 | return new TypeHint() {
47 | }.getTypeInfo();
48 | }
49 | }
50 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/main/java/com/cloudera/streaming/examples/flink/types/TransactionSummary.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Cloudera, Inc. 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, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.cloudera.streaming.examples.flink.types;
20 |
21 | import java.util.Objects;
22 |
23 | /**
24 | * Transaction successfulness summary.
25 | */
26 | public class TransactionSummary {
27 |
28 | public String itemId;
29 |
30 | public int numSuccessfulTransactions = 0;
31 |
32 | public int numFailedTransactions = 0;
33 |
34 | public long totalVolume = 0;
35 |
36 | public TransactionSummary() {
37 | }
38 |
39 | public TransactionSummary(String itemId, int numSuccessfulTransactions, int numFailedTransactions, long totalVolume) {
40 | this.itemId = itemId;
41 | this.numSuccessfulTransactions = numSuccessfulTransactions;
42 | this.numFailedTransactions = numFailedTransactions;
43 | this.totalVolume = totalVolume;
44 | }
45 |
46 | @Override
47 | public boolean equals(Object o) {
48 | if (this == o) {
49 | return true;
50 | }
51 | if (o == null || getClass() != o.getClass()) {
52 | return false;
53 | }
54 |
55 | TransactionSummary that = (TransactionSummary) o;
56 | return numSuccessfulTransactions == that.numSuccessfulTransactions &&
57 | numFailedTransactions == that.numFailedTransactions &&
58 | totalVolume == that.totalVolume &&
59 | Objects.equals(itemId, that.itemId);
60 | }
61 |
62 | @Override
63 | public int hashCode() {
64 | return Objects.hash(itemId, numSuccessfulTransactions, numFailedTransactions, totalVolume);
65 | }
66 |
67 | @Override
68 | public String toString() {
69 | return "TransactionSummary{" +
70 | "itemId='" + itemId + '\'' +
71 | ", numSuccessfulTransactions=" + numSuccessfulTransactions +
72 | ", numFailedTransactions=" + numFailedTransactions +
73 | ", totalVolume=" + totalVolume +
74 | '}';
75 | }
76 | }
77 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/main/java/com/cloudera/streaming/examples/flink/utils/ExponentialHistogram.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Cloudera, Inc. 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, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.cloudera.streaming.examples.flink.utils;
20 |
21 | import org.apache.flink.metrics.Histogram;
22 | import org.apache.flink.metrics.HistogramStatistics;
23 |
24 | import com.codahale.metrics.ExponentiallyDecayingReservoir;
25 |
26 | /**
27 | * Wrapper for exponential histogram at com.codahale.metrics.
28 | */
29 | public class ExponentialHistogram implements Histogram {
30 |
31 | private final com.codahale.metrics.Histogram dropwizardHistogram;
32 |
33 | public ExponentialHistogram(int size, double alpha) {
34 | this.dropwizardHistogram = new com.codahale.metrics.Histogram(
35 | new ExponentiallyDecayingReservoir(size, alpha));
36 | }
37 |
38 | public ExponentialHistogram() {
39 | this.dropwizardHistogram = new com.codahale.metrics.Histogram(
40 | new ExponentiallyDecayingReservoir());
41 | }
42 |
43 | @Override
44 | public void update(long value) {
45 | dropwizardHistogram.update(value);
46 |
47 | }
48 |
49 | @Override
50 | public long getCount() {
51 | return dropwizardHistogram.getCount();
52 | }
53 |
54 | @Override
55 | public HistogramStatistics getStatistics() {
56 | return new SlidingHistogramStatistics(dropwizardHistogram.getSnapshot());
57 | }
58 |
59 | /**
60 | * Wrapper for Snapshot at com.codahale.metrics.
61 | */
62 | public static final class SlidingHistogramStatistics extends HistogramStatistics {
63 |
64 | private final com.codahale.metrics.Snapshot snapshot;
65 |
66 | SlidingHistogramStatistics(com.codahale.metrics.Snapshot snapshot) {
67 | this.snapshot = snapshot;
68 | }
69 |
70 | @Override
71 | public double getQuantile(double quantile) {
72 | return snapshot.getValue(quantile);
73 | }
74 |
75 | @Override
76 | public long[] getValues() {
77 | return snapshot.getValues();
78 | }
79 |
80 | @Override
81 | public int size() {
82 | return snapshot.size();
83 | }
84 |
85 | @Override
86 | public double getMean() {
87 | return snapshot.getMean();
88 | }
89 |
90 | @Override
91 | public double getStdDev() {
92 | return snapshot.getStdDev();
93 | }
94 |
95 | @Override
96 | public long getMax() {
97 | return snapshot.getMax();
98 | }
99 |
100 | @Override
101 | public long getMin() {
102 | return snapshot.getMin();
103 | }
104 | }
105 |
106 | }
107 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/main/java/com/cloudera/streaming/examples/flink/utils/Utils.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Cloudera, Inc. 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, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.cloudera.streaming.examples.flink.utils;
20 |
21 | import org.apache.flink.api.java.utils.ParameterTool;
22 |
23 | import org.slf4j.Logger;
24 | import org.slf4j.LoggerFactory;
25 |
26 | import java.util.Properties;
27 |
28 | /**
29 | * Utility functions.
30 | */
31 | public class Utils {
32 |
33 | private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
34 |
35 | public static final String KAFKA_PREFIX = "kafka.";
36 |
37 | public static Properties readKafkaProperties(ParameterTool params) {
38 | Properties properties = new Properties();
39 | for (String key : params.getProperties().stringPropertyNames()) {
40 | if (key.startsWith(KAFKA_PREFIX)) {
41 | properties.setProperty(key.substring(KAFKA_PREFIX.length()), params.get(key));
42 | }
43 | }
44 |
45 | LOG.info("### Kafka parameters:");
46 | for (String key : properties.stringPropertyNames()) {
47 | LOG.info("Kafka param: {}={}", key, properties.get(key));
48 | }
49 | return properties;
50 | }
51 |
52 | private Utils() {
53 | throw new UnsupportedOperationException("Utils should not be instantiated!");
54 | }
55 | }
56 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/test/java/com/cloudera/streaming/examples/flink/SocketTransactionProcessorJob.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Cloudera, Inc. 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, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.cloudera.streaming.examples.flink;
20 |
21 | import org.apache.flink.api.common.functions.FlatMapFunction;
22 | import org.apache.flink.api.java.utils.ParameterTool;
23 | import org.apache.flink.streaming.api.datastream.DataStream;
24 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
25 | import org.apache.flink.util.Collector;
26 |
27 | import com.cloudera.streaming.examples.flink.operators.ItemTransactionGeneratorSource;
28 | import com.cloudera.streaming.examples.flink.types.ItemTransaction;
29 | import com.cloudera.streaming.examples.flink.types.Query;
30 | import com.cloudera.streaming.examples.flink.types.QueryResult;
31 | import com.cloudera.streaming.examples.flink.types.TransactionResult;
32 | import com.cloudera.streaming.examples.flink.types.TransactionSummary;
33 | import com.fasterxml.jackson.databind.ObjectMapper;
34 |
35 | /**
36 | * Simple socket based pipeline for testing the application locally. Before running start a socket connection:
37 | *
38 | * nc -lk 9999
39 | *
40 | *
Once the job started you can send queries in the form:
41 | *
42 | *
{"queryId":$queryId, "itemId":"$itemId"}
43 | *
44 | *
For example:
45 | *
46 | *
{"queryId":123, "itemId":"item_2"}
47 | */
48 | public class SocketTransactionProcessorJob extends ItemTransactionJob {
49 |
50 | public static void main(String[] args) throws Exception {
51 | new SocketTransactionProcessorJob()
52 | .createApplicationPipeline(ParameterTool.fromArgs(new String[]{"--minimum.summary.vol", "850", "--sleep", "1"}))
53 | .execute();
54 | }
55 |
56 | @Override
57 | public void writeQueryOutput(ParameterTool params, DataStream queryResultStream) {
58 | queryResultStream.printToErr();
59 | }
60 |
61 | @Override
62 | protected void writeTransactionResults(ParameterTool params, DataStream transactionResults) {
63 | // Ignore them for now
64 | }
65 |
66 | @Override
67 | protected void writeTransactionSummaries(ParameterTool params, DataStream transactionSummaryStream) {
68 | // transactionSummaryStream.print();
69 | }
70 |
71 | @Override
72 | public DataStream readQueryStream(ParameterTool params, StreamExecutionEnvironment env) {
73 | return env.socketTextStream("localhost", 9999).flatMap(new FlatMapFunction() {
74 | private final ObjectMapper om = new ObjectMapper();
75 |
76 | @Override
77 | public void flatMap(String s, Collector out) {
78 | try {
79 | out.collect(om.readValue(s, Query.class));
80 | } catch (Throwable ignored) {}
81 | }
82 | });
83 | }
84 |
85 | @Override
86 | public DataStream readTransactionStream(ParameterTool params, StreamExecutionEnvironment env) {
87 | return env.addSource(new ItemTransactionGeneratorSource(params));
88 | }
89 | }
90 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/test/java/com/cloudera/streaming/examples/flink/TransactionProcessorTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Cloudera, Inc. 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, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.cloudera.streaming.examples.flink;
20 |
21 | import org.apache.flink.api.common.typeinfo.TypeInformation;
22 | import org.apache.flink.api.java.utils.ParameterTool;
23 | import org.apache.flink.streaming.api.datastream.DataStream;
24 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
25 | import org.apache.flink.test.util.CollectingSink;
26 | import org.apache.flink.test.util.JobTester;
27 | import org.apache.flink.test.util.ManualSource;
28 |
29 | import com.cloudera.streaming.examples.flink.types.ItemInfo;
30 | import com.cloudera.streaming.examples.flink.types.ItemTransaction;
31 | import com.cloudera.streaming.examples.flink.types.Query;
32 | import com.cloudera.streaming.examples.flink.types.QueryResult;
33 | import com.cloudera.streaming.examples.flink.types.TransactionResult;
34 | import com.cloudera.streaming.examples.flink.types.TransactionSummary;
35 | import org.junit.jupiter.api.Test;
36 |
37 | import static org.junit.jupiter.api.Assertions.assertEquals;
38 | import static org.junit.jupiter.api.Assertions.assertTrue;
39 |
40 | /**
41 | * Simple unit test covering the core functions of the stateful example.
42 | */
43 | class TransactionProcessorTest extends ItemTransactionJob {
44 |
45 | private final CollectingSink transactionResultSink = new CollectingSink<>();
46 | private final CollectingSink queryResultSink = new CollectingSink<>();
47 |
48 | private ManualSource transactionSource;
49 | private ManualSource querySource;
50 |
51 | @Test
52 | void runTest() throws Exception {
53 | JobTester.startTest(createApplicationPipeline(ParameterTool.fromArgs(new String[]{})));
54 |
55 | ItemTransaction it1 = new ItemTransaction(1, 2, "item_1", 100);
56 | transactionSource.sendRecord(it1);
57 | assertEquals(new TransactionResult(it1, true), transactionResultSink.poll());
58 |
59 | querySource.sendRecord(new Query(0, "item_1"));
60 | assertEquals(new QueryResult(0, new ItemInfo("item_1", 100)), queryResultSink.poll());
61 |
62 | querySource.sendRecord(new Query(3, "item_2"));
63 | assertEquals(new QueryResult(3, new ItemInfo("item_2", 0)), queryResultSink.poll());
64 |
65 | JobTester.stopTest();
66 |
67 | assertTrue(transactionResultSink.isEmpty());
68 | assertTrue(queryResultSink.isEmpty());
69 | }
70 |
71 | @Override
72 | public void writeQueryOutput(ParameterTool params, DataStream queryResultStream) {
73 | queryResultStream.addSink(queryResultSink);
74 | }
75 |
76 | @Override
77 | protected void writeTransactionResults(ParameterTool params, DataStream transactionResults) {
78 | transactionResults.addSink(transactionResultSink);
79 | }
80 |
81 | @Override
82 | protected void writeTransactionSummaries(ParameterTool params, DataStream transactionSummaryStream) {
83 | //ignore
84 | }
85 |
86 | @Override
87 | public DataStream readQueryStream(ParameterTool params, StreamExecutionEnvironment env) {
88 | querySource = JobTester.createManualSource(env, TypeInformation.of(Query.class));
89 | return querySource.getDataStream();
90 | }
91 |
92 | @Override
93 | public DataStream readTransactionStream(ParameterTool params, StreamExecutionEnvironment env) {
94 | transactionSource = JobTester.createManualSource(env, TypeInformation.of(ItemTransaction.class));
95 | return transactionSource.getDataStream();
96 | }
97 | }
98 |
--------------------------------------------------------------------------------
/flink-stateful-tutorial/src/test/resources/log4j2-test.properties:
--------------------------------------------------------------------------------
1 | ################################################################################
2 | # Licensed to Cloudera, Inc. 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, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 |
19 | rootLogger.level = INFO
20 | rootLogger.appenderRef.test.ref = TestLogger
21 |
22 | appender.testlogger.name = TestLogger
23 | appender.testlogger.type = CONSOLE
24 | appender.testlogger.target = SYSTEM_OUT
25 | appender.testlogger.layout.type = PatternLayout
26 | appender.testlogger.layout.pattern = %d %p %C{1.} [%t] %m%n
27 |
--------------------------------------------------------------------------------
/pom.xml:
--------------------------------------------------------------------------------
1 |
19 |
22 | 4.0.0
23 |
24 | com.cloudera.flink
25 | flink-tutorials
26 | 1.20.1-csa1.15.0.0
27 | pom
28 | Flink Tutorials
29 |
30 |
31 | flink-simple-tutorial
32 | flink-stateful-tutorial
33 | flink-secure-tutorial
34 |
35 |
36 |
37 | UTF-8
38 | 2.12
39 |
40 | 1.20.1-csa1.15.0.0
41 | 1.2.0-csa1.15.0.0
42 | 3.4.0-csa1.15.0.0
43 |
44 |
45 | 2.14.3
46 | 5.10.1
47 | 2.22.0
48 | 4.2.22
49 | 8.0.33
50 |
51 |
52 | 1.9.2
53 | 1.0.0
54 | 2.17
55 | 3.4.1
56 | 3.2.2
57 |
58 |
59 |
60 |
61 |
62 |
63 | org.apache.flink
64 | flink-clients
65 | ${flink.version}
66 |
67 |
68 |
69 | org.apache.flink
70 | flink-connector-cloudera-registry
71 | ${flink.cloudera.connector.version}
72 |
73 |
74 | junit
75 | junit
76 |
77 |
78 |
79 |
80 |
81 | org.apache.flink
82 | flink-connector-base
83 | ${flink.version}
84 |
85 |
86 |
87 | org.apache.flink
88 | flink-connector-kafka
89 | ${flink.kafka.connector.version}
90 |
91 |
92 |
93 | org.apache.flink
94 | flink-connector-files
95 | ${flink.version}
96 |
97 |
98 |
99 | org.apache.flink
100 | flink-core
101 | ${flink.version}
102 |
103 |
104 |
105 | org.apache.flink
106 | flink-java
107 | ${flink.version}
108 |
109 |
110 |
111 | org.apache.flink
112 | flink-statebackend-rocksdb
113 | ${flink.version}
114 |
115 |
116 |
117 | org.apache.flink
118 | flink-streaming-java
119 | ${flink.version}
120 |
121 |
122 |
123 | org.apache.flink
124 | flink-test-utils
125 | ${flink.version}
126 |
127 |
128 | junit
129 | junit
130 |
131 |
132 | org.junit.vintage
133 | junit-vintage-engine
134 |
135 |
136 |
137 |
138 |
139 |
140 | org.apache.logging.log4j
141 | log4j-api
142 | ${log4j.version}
143 |
144 |
145 |
146 | org.apache.logging.log4j
147 | log4j-core
148 | ${log4j.version}
149 |
150 |
151 |
152 | org.apache.logging.log4j
153 | log4j-slf4j-impl
154 | ${log4j.version}
155 |
156 |
157 |
158 |
159 | org.junit.jupiter
160 | junit-jupiter-engine
161 | ${junit-jupiter.version}
162 | test
163 |
164 |
165 |
166 |
167 | com.fasterxml.jackson.core
168 | jackson-databind
169 | ${jackson.version}
170 |
171 |
172 |
173 | io.dropwizard.metrics
174 | metrics-core
175 | ${metrics.version}
176 |
177 |
178 |
179 | mysql
180 | mysql-connector-java
181 | ${mysql.version}
182 |
183 |
184 |
185 |
186 |
187 |
188 |
189 | org.apache.maven.plugins
190 | maven-surefire-plugin
191 |
192 |
193 |
194 |
195 |
196 |
197 | org.apache.avro
198 | avro-maven-plugin
199 | ${avro.plugin.version}
200 |
201 |
202 |
203 | org.apache.maven.plugins
204 | maven-shade-plugin
205 | ${maven.shade.plugin.version}
206 |
207 |
208 |
209 | org.apache.maven.plugins
210 | maven-surefire-plugin
211 | ${maven.surefire.plugin.version}
212 |
213 |
214 |
215 | org.apache.maven.plugins
216 | maven-checkstyle-plugin
217 | ${maven.checkstyle.plugin.version}
218 |
219 |
220 | com.puppycrawl.tools
221 | checkstyle
222 |
223 | 8.14
224 |
225 |
226 |
227 |
228 | validate
229 | validate
230 |
231 | check
232 |
233 |
234 |
235 |
236 | checkstyle/suppressions.xml
237 | true
238 |
239 | https://raw.githubusercontent.com/apache/flink/master/tools/maven/checkstyle.xml
240 |
241 | true
242 | true
243 |
244 |
245 |
246 |
247 |
248 | org.eclipse.m2e
249 | lifecycle-mapping
250 | ${lifecycle.mapping.version}
251 |
252 |
253 |
254 |
255 |
256 | org.apache.maven.plugins
257 | maven-shade-plugin
258 | [3.0.0,)
259 |
260 | shade
261 |
262 |
263 |
264 |
265 |
266 |
267 |
268 |
269 | org.apache.maven.plugins
270 | maven-compiler-plugin
271 | [3.1,)
272 |
273 | testCompile
274 | compile
275 |
276 |
277 |
278 |
279 |
280 |
281 |
282 |
283 |
284 |
285 |
286 |
287 |
288 |
289 |
290 |
291 | cloudera
292 | https://repository.cloudera.com/artifactory/libs-release-local/
293 |
294 |
295 |
296 | nexus-public-snapshot
297 | https://nexus-private.hortonworks.com/nexus/repository/PUBLIC-SNAPSHOT
298 |
299 | false
300 |
301 |
302 | true
303 |
304 |
305 |
306 |
307 | hortonworks
308 | https://repo.hortonworks.com/content/repositories/releases/
309 |
310 |
311 |
312 |
313 |
314 | in-qa-snapshot
315 | Cloudera Snapshot Artifactory
316 | https://nexus-private.hortonworks.com/nexus/repository/IN-QA-SNAPSHOT
317 |
318 |
319 |
320 |
321 |
--------------------------------------------------------------------------------