other) {
56 | localValue = localValue.add(other.getLocalValue());
57 | }
58 |
59 | @Override
60 | public void resetLocal() {
61 | this.localValue = BigDecimal.ZERO;
62 | }
63 |
64 | @Override
65 | public BigDecimalCounter clone() {
66 | BigDecimalCounter result = new BigDecimalCounter();
67 | result.localValue = localValue;
68 | return result;
69 | }
70 |
71 | // ------------------------------------------------------------------------
72 | // Utilities
73 | // ------------------------------------------------------------------------
74 |
75 | @Override
76 | public String toString() {
77 | return "BigDecimalCounter " + this.localValue;
78 | }
79 | }
80 |
--------------------------------------------------------------------------------
/flink-job/src/main/java/com/ververica/field/dynamicrules/accumulators/BigDecimalMaximum.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, 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.ververica.field.dynamicrules.accumulators;
20 |
21 | import java.math.BigDecimal;
22 | import org.apache.flink.annotation.PublicEvolving;
23 | import org.apache.flink.api.common.accumulators.Accumulator;
24 | import org.apache.flink.api.common.accumulators.SimpleAccumulator;
25 |
26 | /**
27 | * An accumulator that finds the maximum {@code BigDecimal} value.
28 | *
29 | * Supports numbers greater than Double.MIN_VALUE.
30 | */
31 | @PublicEvolving
32 | public class BigDecimalMaximum implements SimpleAccumulator {
33 |
34 | private static final long serialVersionUID = 1L;
35 |
36 | private BigDecimal max = BigDecimal.valueOf(Double.MIN_VALUE);
37 |
38 | private final BigDecimal limit = BigDecimal.valueOf(Double.MIN_VALUE);
39 |
40 | public BigDecimalMaximum() {}
41 |
42 | public BigDecimalMaximum(BigDecimal value) {
43 | this.max = value;
44 | }
45 |
46 | // ------------------------------------------------------------------------
47 | // Accumulator
48 | // ------------------------------------------------------------------------
49 |
50 | @Override
51 | public void add(BigDecimal value) {
52 | if (value.compareTo(limit) < 0) {
53 | throw new IllegalArgumentException(
54 | "BigDecimalMaximum accumulator only supports values greater than Double.MIN_VALUE");
55 | }
56 | this.max = max.max(value);
57 | }
58 |
59 | @Override
60 | public BigDecimal getLocalValue() {
61 | return this.max;
62 | }
63 |
64 | @Override
65 | public void merge(Accumulator other) {
66 | this.max = max.max(other.getLocalValue());
67 | }
68 |
69 | @Override
70 | public void resetLocal() {
71 | this.max = BigDecimal.valueOf(Double.MIN_VALUE);
72 | }
73 |
74 | @Override
75 | public BigDecimalMaximum clone() {
76 | BigDecimalMaximum clone = new BigDecimalMaximum();
77 | clone.max = this.max;
78 | return clone;
79 | }
80 |
81 | // ------------------------------------------------------------------------
82 | // Utilities
83 | // ------------------------------------------------------------------------
84 |
85 | @Override
86 | public String toString() {
87 | return "BigDecimal " + this.max;
88 | }
89 | }
90 |
--------------------------------------------------------------------------------
/flink-job/src/main/java/com/ververica/field/dynamicrules/accumulators/BigDecimalMinimum.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, 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.ververica.field.dynamicrules.accumulators;
20 |
21 | import java.math.BigDecimal;
22 | import org.apache.flink.annotation.PublicEvolving;
23 | import org.apache.flink.api.common.accumulators.Accumulator;
24 | import org.apache.flink.api.common.accumulators.SimpleAccumulator;
25 |
26 | /**
27 | * An accumulator that finds the minimum {@code BigDecimal} value.
28 | *
29 | * Supports numbers less than Double.MAX_VALUE.
30 | */
31 | @PublicEvolving
32 | public class BigDecimalMinimum implements SimpleAccumulator {
33 |
34 | private static final long serialVersionUID = 1L;
35 |
36 | private BigDecimal min = BigDecimal.valueOf(Double.MAX_VALUE);
37 |
38 | private final BigDecimal limit = BigDecimal.valueOf(Double.MAX_VALUE);
39 |
40 | public BigDecimalMinimum() {}
41 |
42 | public BigDecimalMinimum(BigDecimal value) {
43 | this.min = value;
44 | }
45 |
46 | // ------------------------------------------------------------------------
47 | // Accumulator
48 | // ------------------------------------------------------------------------
49 |
50 | @Override
51 | public void add(BigDecimal value) {
52 | if (value.compareTo(limit) > 0) {
53 | throw new IllegalArgumentException(
54 | "BigDecimalMinimum accumulator only supports values less than Double.MAX_VALUE");
55 | }
56 | this.min = min.min(value);
57 | }
58 |
59 | @Override
60 | public BigDecimal getLocalValue() {
61 | return this.min;
62 | }
63 |
64 | @Override
65 | public void merge(Accumulator other) {
66 | this.min = min.min(other.getLocalValue());
67 | }
68 |
69 | @Override
70 | public void resetLocal() {
71 | this.min = BigDecimal.valueOf(Double.MAX_VALUE);
72 | }
73 |
74 | @Override
75 | public BigDecimalMinimum clone() {
76 | BigDecimalMinimum clone = new BigDecimalMinimum();
77 | clone.min = this.min;
78 | return clone;
79 | }
80 |
81 | // ------------------------------------------------------------------------
82 | // Utilities
83 | // ------------------------------------------------------------------------
84 |
85 | @Override
86 | public String toString() {
87 | return "BigDecimal " + this.min;
88 | }
89 | }
90 |
--------------------------------------------------------------------------------
/flink-job/src/main/java/com/ververica/field/dynamicrules/functions/AverageAggregate.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, 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.ververica.field.dynamicrules.functions;
20 |
21 | import org.apache.flink.api.common.functions.AggregateFunction;
22 | import org.apache.flink.api.java.tuple.Tuple2;
23 |
24 | /**
25 | * The accumulator is used to keep a running sum and a count. The {@code getResult} method computes
26 | * the average.
27 | */
28 | public class AverageAggregate implements AggregateFunction, Double> {
29 | @Override
30 | public Tuple2 createAccumulator() {
31 | return new Tuple2<>(0L, 0L);
32 | }
33 |
34 | @Override
35 | public Tuple2 add(Long value, Tuple2 accumulator) {
36 | return new Tuple2<>(accumulator.f0 + value, accumulator.f1 + 1L);
37 | }
38 |
39 | @Override
40 | public Double getResult(Tuple2 accumulator) {
41 | return ((double) accumulator.f0) / accumulator.f1;
42 | }
43 |
44 | @Override
45 | public Tuple2 merge(Tuple2 a, Tuple2 b) {
46 | return new Tuple2<>(a.f0 + b.f0, a.f1 + b.f1);
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/flink-job/src/main/java/com/ververica/field/dynamicrules/functions/JsonDeserializer.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, 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.ververica.field.dynamicrules.functions;
20 |
21 | import com.ververica.field.dynamicrules.JsonMapper;
22 | import lombok.extern.slf4j.Slf4j;
23 | import org.apache.flink.api.common.functions.RichFlatMapFunction;
24 | import org.apache.flink.configuration.Configuration;
25 | import org.apache.flink.util.Collector;
26 |
27 | @Slf4j
28 | public class JsonDeserializer extends RichFlatMapFunction {
29 |
30 | private JsonMapper parser;
31 | private final Class targetClass;
32 |
33 | public JsonDeserializer(Class targetClass) {
34 | this.targetClass = targetClass;
35 | }
36 |
37 | @Override
38 | public void open(Configuration parameters) throws Exception {
39 | super.open(parameters);
40 | parser = new JsonMapper<>(targetClass);
41 | }
42 |
43 | @Override
44 | public void flatMap(String value, Collector out) throws Exception {
45 | log.info("{}", value);
46 | try {
47 | T parsed = parser.fromString(value);
48 | out.collect(parsed);
49 | } catch (Exception e) {
50 | log.warn("Failed parsing rule, dropping it:", e);
51 | }
52 | }
53 | }
54 |
--------------------------------------------------------------------------------
/flink-job/src/main/java/com/ververica/field/dynamicrules/functions/JsonGeneratorWrapper.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, 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.ververica.field.dynamicrules.functions;
20 |
21 | import com.ververica.field.sources.BaseGenerator;
22 | import java.util.SplittableRandom;
23 | import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException;
24 | import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
25 |
26 | public class JsonGeneratorWrapper extends BaseGenerator {
27 |
28 | private BaseGenerator wrappedGenerator;
29 | private static final ObjectMapper objectMapper = new ObjectMapper();
30 |
31 | public JsonGeneratorWrapper(BaseGenerator wrappedGenerator) {
32 | this.wrappedGenerator = wrappedGenerator;
33 | this.maxRecordsPerSecond = wrappedGenerator.getMaxRecordsPerSecond();
34 | }
35 |
36 | @Override
37 | public String randomEvent(SplittableRandom rnd, long id) {
38 | T transaction = wrappedGenerator.randomEvent(rnd, id);
39 | String json;
40 | try {
41 | json = objectMapper.writeValueAsString(transaction);
42 | } catch (JsonProcessingException e) {
43 | throw new RuntimeException(e);
44 | }
45 | return json;
46 | }
47 | }
48 |
--------------------------------------------------------------------------------
/flink-job/src/main/java/com/ververica/field/dynamicrules/functions/JsonSerializer.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, 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.ververica.field.dynamicrules.functions;
20 |
21 | import com.ververica.field.dynamicrules.JsonMapper;
22 | import lombok.extern.slf4j.Slf4j;
23 | import org.apache.flink.api.common.functions.RichFlatMapFunction;
24 | import org.apache.flink.configuration.Configuration;
25 | import org.apache.flink.util.Collector;
26 |
27 | @Slf4j
28 | public class JsonSerializer extends RichFlatMapFunction {
29 |
30 | private JsonMapper parser;
31 | private final Class targetClass;
32 |
33 | public JsonSerializer(Class sourceClass) {
34 | this.targetClass = sourceClass;
35 | }
36 |
37 | @Override
38 | public void open(Configuration parameters) throws Exception {
39 | super.open(parameters);
40 | parser = new JsonMapper<>(targetClass);
41 | }
42 |
43 | @Override
44 | public void flatMap(T value, Collector out) throws Exception {
45 | System.out.println(value);
46 | try {
47 | String serialized = parser.toString(value);
48 | out.collect(serialized);
49 | } catch (Exception e) {
50 | log.warn("Failed serializing to JSON dropping it:", e);
51 | }
52 | }
53 | }
54 |
--------------------------------------------------------------------------------
/flink-job/src/main/java/com/ververica/field/dynamicrules/functions/ProcessingUtils.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, 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.ververica.field.dynamicrules.functions;
20 |
21 | import com.ververica.field.dynamicrules.Rule;
22 | import java.util.HashSet;
23 | import java.util.Set;
24 | import org.apache.flink.api.common.state.BroadcastState;
25 | import org.apache.flink.api.common.state.MapState;
26 |
27 | class ProcessingUtils {
28 |
29 | static void handleRuleBroadcast(Rule rule, BroadcastState broadcastState)
30 | throws Exception {
31 | switch (rule.getRuleState()) {
32 | case ACTIVE:
33 | case PAUSE:
34 | broadcastState.put(rule.getRuleId(), rule);
35 | break;
36 | case DELETE:
37 | broadcastState.remove(rule.getRuleId());
38 | break;
39 | }
40 | }
41 |
42 | static Set addToStateValuesSet(MapState> mapState, K key, V value)
43 | throws Exception {
44 |
45 | Set valuesSet = mapState.get(key);
46 |
47 | if (valuesSet != null) {
48 | valuesSet.add(value);
49 | } else {
50 | valuesSet = new HashSet<>();
51 | valuesSet.add(value);
52 | }
53 | mapState.put(key, valuesSet);
54 | return valuesSet;
55 | }
56 | }
57 |
--------------------------------------------------------------------------------
/flink-job/src/main/java/com/ververica/field/dynamicrules/functions/RuleDeserializer.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, 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.ververica.field.dynamicrules.functions;
20 |
21 | import com.ververica.field.dynamicrules.Rule;
22 | import com.ververica.field.dynamicrules.Rule.RuleState;
23 | import com.ververica.field.dynamicrules.RuleParser;
24 | import lombok.extern.slf4j.Slf4j;
25 | import org.apache.flink.api.common.functions.RichFlatMapFunction;
26 | import org.apache.flink.configuration.Configuration;
27 | import org.apache.flink.util.Collector;
28 |
29 | @Slf4j
30 | public class RuleDeserializer extends RichFlatMapFunction {
31 |
32 | private RuleParser ruleParser;
33 |
34 | @Override
35 | public void open(Configuration parameters) throws Exception {
36 | super.open(parameters);
37 | ruleParser = new RuleParser();
38 | }
39 |
40 | @Override
41 | public void flatMap(String value, Collector out) throws Exception {
42 | log.info("{}", value);
43 | try {
44 | Rule rule = ruleParser.fromString(value);
45 | if (rule.getRuleState() != RuleState.CONTROL && rule.getRuleId() == null) {
46 | throw new NullPointerException("ruleId cannot be null: " + rule.toString());
47 | }
48 | out.collect(rule);
49 | } catch (Exception e) {
50 | log.warn("Failed parsing rule, dropping it:", e);
51 | }
52 | }
53 | }
54 |
--------------------------------------------------------------------------------
/flink-job/src/main/java/com/ververica/field/dynamicrules/functions/TimeStamper.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, 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.ververica.field.dynamicrules.functions;
20 |
21 | import com.ververica.field.dynamicrules.TimestampAssignable;
22 | import org.apache.flink.api.common.functions.RichFlatMapFunction;
23 | import org.apache.flink.util.Collector;
24 |
25 | public class TimeStamper> extends RichFlatMapFunction {
26 |
27 | @Override
28 | public void flatMap(T value, Collector out) throws Exception {
29 | value.assignIngestionTimestamp(System.currentTimeMillis());
30 | out.collect(value);
31 | }
32 | }
33 |
--------------------------------------------------------------------------------
/flink-job/src/main/java/com/ververica/field/dynamicrules/functions/TransactionsGenerator.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, 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.ververica.field.dynamicrules.functions;
20 |
21 | import com.ververica.field.dynamicrules.Transaction;
22 | import com.ververica.field.dynamicrules.Transaction.PaymentType;
23 | import com.ververica.field.sources.BaseGenerator;
24 | import java.math.BigDecimal;
25 | import java.util.SplittableRandom;
26 | import java.util.concurrent.ThreadLocalRandom;
27 |
28 | public class TransactionsGenerator extends BaseGenerator {
29 |
30 | private static long MAX_PAYEE_ID = 100000;
31 | private static long MAX_BENEFICIARY_ID = 100000;
32 |
33 | private static double MIN_PAYMENT_AMOUNT = 5d;
34 | private static double MAX_PAYMENT_AMOUNT = 20d;
35 |
36 | public TransactionsGenerator(int maxRecordsPerSecond) {
37 | super(maxRecordsPerSecond);
38 | }
39 |
40 | @Override
41 | public Transaction randomEvent(SplittableRandom rnd, long id) {
42 | long transactionId = rnd.nextLong(Long.MAX_VALUE);
43 | long payeeId = rnd.nextLong(MAX_PAYEE_ID);
44 | long beneficiaryId = rnd.nextLong(MAX_BENEFICIARY_ID);
45 | double paymentAmountDouble =
46 | ThreadLocalRandom.current().nextDouble(MIN_PAYMENT_AMOUNT, MAX_PAYMENT_AMOUNT);
47 | paymentAmountDouble = Math.floor(paymentAmountDouble * 100) / 100;
48 | BigDecimal paymentAmount = BigDecimal.valueOf(paymentAmountDouble);
49 |
50 | Transaction transaction =
51 | Transaction.builder()
52 | .transactionId(transactionId)
53 | .payeeId(payeeId)
54 | .beneficiaryId(beneficiaryId)
55 | .paymentAmount(paymentAmount)
56 | .paymentType(paymentType(transactionId))
57 | .eventTime(System.currentTimeMillis())
58 | .ingestionTimestamp(System.currentTimeMillis())
59 | .build();
60 |
61 | return transaction;
62 | }
63 |
64 | private PaymentType paymentType(long id) {
65 | int name = (int) (id % 2);
66 | switch (name) {
67 | case 0:
68 | return PaymentType.CRD;
69 | case 1:
70 | return PaymentType.CSH;
71 | default:
72 | throw new IllegalStateException("");
73 | }
74 | }
75 | }
76 |
--------------------------------------------------------------------------------
/flink-job/src/main/java/com/ververica/field/dynamicrules/sinks/AlertsSink.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, 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.ververica.field.dynamicrules.sinks;
20 |
21 | import static com.ververica.field.config.Parameters.ALERTS_SINK;
22 | import static com.ververica.field.config.Parameters.ALERTS_TOPIC;
23 | import static com.ververica.field.config.Parameters.GCP_PROJECT_NAME;
24 | import static com.ververica.field.config.Parameters.GCP_PUBSUB_ALERTS_SUBSCRIPTION;
25 |
26 | import com.ververica.field.config.Config;
27 | import com.ververica.field.dynamicrules.Alert;
28 | import com.ververica.field.dynamicrules.KafkaUtils;
29 | import com.ververica.field.dynamicrules.functions.JsonSerializer;
30 | import java.io.IOException;
31 | import java.util.Properties;
32 | import org.apache.flink.api.common.serialization.SimpleStringSchema;
33 | import org.apache.flink.streaming.api.datastream.DataStream;
34 | import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
35 | import org.apache.flink.streaming.api.functions.sink.SinkFunction;
36 | import org.apache.flink.streaming.connectors.gcp.pubsub.PubSubSink;
37 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011;
38 |
39 | public class AlertsSink {
40 |
41 | public static SinkFunction createAlertsSink(Config config) throws IOException {
42 |
43 | String sinkType = config.get(ALERTS_SINK);
44 | AlertsSink.Type alertsSinkType = AlertsSink.Type.valueOf(sinkType.toUpperCase());
45 |
46 | switch (alertsSinkType) {
47 | case KAFKA:
48 | Properties kafkaProps = KafkaUtils.initProducerProperties(config);
49 | String alertsTopic = config.get(ALERTS_TOPIC);
50 | return new FlinkKafkaProducer011<>(alertsTopic, new SimpleStringSchema(), kafkaProps);
51 | case PUBSUB:
52 | return PubSubSink.newBuilder()
53 | .withSerializationSchema(new SimpleStringSchema())
54 | .withProjectName(config.get(GCP_PROJECT_NAME))
55 | .withTopicName(config.get(GCP_PUBSUB_ALERTS_SUBSCRIPTION))
56 | .build();
57 | case STDOUT:
58 | return new PrintSinkFunction<>(true);
59 | default:
60 | throw new IllegalArgumentException(
61 | "Source \"" + alertsSinkType + "\" unknown. Known values are:" + Type.values());
62 | }
63 | }
64 |
65 | public static DataStream alertsStreamToJson(DataStream alerts) {
66 | return alerts.flatMap(new JsonSerializer<>(Alert.class)).name("Alerts Deserialization");
67 | }
68 |
69 | public enum Type {
70 | KAFKA("Alerts Sink (Kafka)"),
71 | PUBSUB("Alerts Sink (Pub/Sub)"),
72 | STDOUT("Alerts Sink (Std. Out)");
73 |
74 | private String name;
75 |
76 | Type(String name) {
77 | this.name = name;
78 | }
79 |
80 | public String getName() {
81 | return name;
82 | }
83 | }
84 | }
85 |
--------------------------------------------------------------------------------
/flink-job/src/main/java/com/ververica/field/dynamicrules/sinks/CurrentRulesSink.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, 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.ververica.field.dynamicrules.sinks;
20 |
21 | import static com.ververica.field.config.Parameters.GCP_PROJECT_NAME;
22 | import static com.ververica.field.config.Parameters.GCP_PUBSUB_RULES_SUBSCRIPTION;
23 | import static com.ververica.field.config.Parameters.RULES_EXPORT_SINK;
24 | import static com.ververica.field.config.Parameters.RULES_EXPORT_TOPIC;
25 |
26 | import com.ververica.field.config.Config;
27 | import com.ververica.field.dynamicrules.KafkaUtils;
28 | import com.ververica.field.dynamicrules.Rule;
29 | import com.ververica.field.dynamicrules.functions.JsonSerializer;
30 | import java.io.IOException;
31 | import java.util.Properties;
32 | import org.apache.flink.api.common.serialization.SimpleStringSchema;
33 | import org.apache.flink.streaming.api.datastream.DataStream;
34 | import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
35 | import org.apache.flink.streaming.api.functions.sink.SinkFunction;
36 | import org.apache.flink.streaming.connectors.gcp.pubsub.PubSubSink;
37 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011;
38 |
39 | public class CurrentRulesSink {
40 |
41 | public static SinkFunction createRulesSink(Config config) throws IOException {
42 |
43 | String sinkType = config.get(RULES_EXPORT_SINK);
44 | CurrentRulesSink.Type currentRulesSinkType =
45 | CurrentRulesSink.Type.valueOf(sinkType.toUpperCase());
46 |
47 | switch (currentRulesSinkType) {
48 | case KAFKA:
49 | Properties kafkaProps = KafkaUtils.initProducerProperties(config);
50 | String alertsTopic = config.get(RULES_EXPORT_TOPIC);
51 | return new FlinkKafkaProducer011<>(alertsTopic, new SimpleStringSchema(), kafkaProps);
52 | case PUBSUB:
53 | return PubSubSink.newBuilder()
54 | .withSerializationSchema(new SimpleStringSchema())
55 | .withProjectName(config.get(GCP_PROJECT_NAME))
56 | .withTopicName(config.get(GCP_PUBSUB_RULES_SUBSCRIPTION))
57 | .build();
58 | case STDOUT:
59 | return new PrintSinkFunction<>(true);
60 | default:
61 | throw new IllegalArgumentException(
62 | "Source \"" + currentRulesSinkType + "\" unknown. Known values are:" + Type.values());
63 | }
64 | }
65 |
66 | public static DataStream rulesStreamToJson(DataStream alerts) {
67 | return alerts.flatMap(new JsonSerializer<>(Rule.class)).name("Rules Deserialization");
68 | }
69 |
70 | public enum Type {
71 | KAFKA("Current Rules Sink (Kafka)"),
72 | PUBSUB("Current Rules Sink (Pub/Sub)"),
73 | STDOUT("Current Rules Sink (Std. Out)");
74 |
75 | private String name;
76 |
77 | Type(String name) {
78 | this.name = name;
79 | }
80 |
81 | public String getName() {
82 | return name;
83 | }
84 | }
85 | }
86 |
--------------------------------------------------------------------------------
/flink-job/src/main/java/com/ververica/field/dynamicrules/sinks/LatencySink.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, 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.ververica.field.dynamicrules.sinks;
20 |
21 | import static com.ververica.field.config.Parameters.GCP_PROJECT_NAME;
22 | import static com.ververica.field.config.Parameters.GCP_PUBSUB_LATENCY_SUBSCRIPTION;
23 | import static com.ververica.field.config.Parameters.LATENCY_SINK;
24 | import static com.ververica.field.config.Parameters.LATENCY_TOPIC;
25 |
26 | import com.ververica.field.config.Config;
27 | import com.ververica.field.dynamicrules.KafkaUtils;
28 | import java.io.IOException;
29 | import java.util.Properties;
30 | import org.apache.flink.api.common.serialization.SimpleStringSchema;
31 | import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
32 | import org.apache.flink.streaming.api.functions.sink.SinkFunction;
33 | import org.apache.flink.streaming.connectors.gcp.pubsub.PubSubSink;
34 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011;
35 |
36 | public class LatencySink {
37 |
38 | public static SinkFunction createLatencySink(Config config) throws IOException {
39 |
40 | String latencySink = config.get(LATENCY_SINK);
41 | LatencySink.Type latencySinkType = LatencySink.Type.valueOf(latencySink.toUpperCase());
42 |
43 | switch (latencySinkType) {
44 | case KAFKA:
45 | Properties kafkaProps = KafkaUtils.initProducerProperties(config);
46 | String latencyTopic = config.get(LATENCY_TOPIC);
47 | return new FlinkKafkaProducer011<>(latencyTopic, new SimpleStringSchema(), kafkaProps);
48 | case PUBSUB:
49 | return PubSubSink.newBuilder()
50 | .withSerializationSchema(new SimpleStringSchema())
51 | .withProjectName(config.get(GCP_PROJECT_NAME))
52 | .withTopicName(config.get(GCP_PUBSUB_LATENCY_SUBSCRIPTION))
53 | .build();
54 | case STDOUT:
55 | return new PrintSinkFunction<>(true);
56 | default:
57 | throw new IllegalArgumentException(
58 | "Source \"" + latencySinkType + "\" unknown. Known values are:" + Type.values());
59 | }
60 | }
61 |
62 | public enum Type {
63 | KAFKA("Latency Sink (Kafka)"),
64 | PUBSUB("Latency Sink (Pub/Sub)"),
65 | STDOUT("Latency Sink (Std. Out)");
66 |
67 | private String name;
68 |
69 | Type(String name) {
70 | this.name = name;
71 | }
72 |
73 | public String getName() {
74 | return name;
75 | }
76 | }
77 | }
78 |
--------------------------------------------------------------------------------
/flink-job/src/main/java/com/ververica/field/dynamicrules/sources/RulesSource.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, 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.ververica.field.dynamicrules.sources;
20 |
21 | import static com.ververica.field.config.Parameters.GCP_PROJECT_NAME;
22 | import static com.ververica.field.config.Parameters.GCP_PUBSUB_RULES_SUBSCRIPTION;
23 | import static com.ververica.field.config.Parameters.RULES_SOURCE;
24 | import static com.ververica.field.config.Parameters.RULES_TOPIC;
25 | import static com.ververica.field.config.Parameters.SOCKET_PORT;
26 |
27 | import com.ververica.field.config.Config;
28 | import com.ververica.field.dynamicrules.KafkaUtils;
29 | import com.ververica.field.dynamicrules.Rule;
30 | import com.ververica.field.dynamicrules.functions.RuleDeserializer;
31 | import java.io.IOException;
32 | import java.util.Properties;
33 | import java.util.concurrent.TimeUnit;
34 | import org.apache.flink.api.common.serialization.SimpleStringSchema;
35 | import org.apache.flink.streaming.api.datastream.DataStream;
36 | import org.apache.flink.streaming.api.functions.source.SocketTextStreamFunction;
37 | import org.apache.flink.streaming.api.functions.source.SourceFunction;
38 | import org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor;
39 | import org.apache.flink.streaming.api.windowing.time.Time;
40 | import org.apache.flink.streaming.connectors.gcp.pubsub.PubSubSource;
41 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011;
42 |
43 | public class RulesSource {
44 |
45 | private static final int RULES_STREAM_PARALLELISM = 1;
46 |
47 | public static SourceFunction createRulesSource(Config config) throws IOException {
48 |
49 | String sourceType = config.get(RULES_SOURCE);
50 | RulesSource.Type rulesSourceType = RulesSource.Type.valueOf(sourceType.toUpperCase());
51 |
52 | switch (rulesSourceType) {
53 | case KAFKA:
54 | Properties kafkaProps = KafkaUtils.initConsumerProperties(config);
55 | String rulesTopic = config.get(RULES_TOPIC);
56 | FlinkKafkaConsumer011 kafkaConsumer =
57 | new FlinkKafkaConsumer011<>(rulesTopic, new SimpleStringSchema(), kafkaProps);
58 | kafkaConsumer.setStartFromLatest();
59 | return kafkaConsumer;
60 | case PUBSUB:
61 | return PubSubSource.newBuilder()
62 | .withDeserializationSchema(new SimpleStringSchema())
63 | .withProjectName(config.get(GCP_PROJECT_NAME))
64 | .withSubscriptionName(config.get(GCP_PUBSUB_RULES_SUBSCRIPTION))
65 | .build();
66 | case SOCKET:
67 | return new SocketTextStreamFunction("localhost", config.get(SOCKET_PORT), "\n", -1);
68 | default:
69 | throw new IllegalArgumentException(
70 | "Source \"" + rulesSourceType + "\" unknown. Known values are:" + Type.values());
71 | }
72 | }
73 |
74 | public static DataStream stringsStreamToRules(DataStream ruleStrings) {
75 | return ruleStrings
76 | .flatMap(new RuleDeserializer())
77 | .name("Rule Deserialization")
78 | .setParallelism(RULES_STREAM_PARALLELISM)
79 | .assignTimestampsAndWatermarks(
80 | new BoundedOutOfOrdernessTimestampExtractor(Time.of(0, TimeUnit.MILLISECONDS)) {
81 | @Override
82 | public long extractTimestamp(Rule element) {
83 | // Prevents connected data+update stream watermark stalling.
84 | return Long.MAX_VALUE;
85 | }
86 | });
87 | }
88 |
89 | public enum Type {
90 | KAFKA("Rules Source (Kafka)"),
91 | PUBSUB("Rules Source (Pub/Sub)"),
92 | SOCKET("Rules Source (Socket)");
93 |
94 | private String name;
95 |
96 | Type(String name) {
97 | this.name = name;
98 | }
99 |
100 | public String getName() {
101 | return name;
102 | }
103 | }
104 | }
105 |
--------------------------------------------------------------------------------
/flink-job/src/main/java/com/ververica/field/dynamicrules/sources/TransactionsSource.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, 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.ververica.field.dynamicrules.sources;
20 |
21 | import static com.ververica.field.config.Parameters.DATA_TOPIC;
22 | import static com.ververica.field.config.Parameters.RECORDS_PER_SECOND;
23 | import static com.ververica.field.config.Parameters.TRANSACTIONS_SOURCE;
24 |
25 | import com.ververica.field.config.Config;
26 | import com.ververica.field.dynamicrules.KafkaUtils;
27 | import com.ververica.field.dynamicrules.Transaction;
28 | import com.ververica.field.dynamicrules.functions.JsonDeserializer;
29 | import com.ververica.field.dynamicrules.functions.JsonGeneratorWrapper;
30 | import com.ververica.field.dynamicrules.functions.TimeStamper;
31 | import com.ververica.field.dynamicrules.functions.TransactionsGenerator;
32 | import java.util.Properties;
33 | import org.apache.flink.api.common.serialization.SimpleStringSchema;
34 | import org.apache.flink.streaming.api.datastream.DataStream;
35 | import org.apache.flink.streaming.api.functions.source.SourceFunction;
36 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011;
37 |
38 | public class TransactionsSource {
39 |
40 | public static SourceFunction createTransactionsSource(Config config) {
41 |
42 | String sourceType = config.get(TRANSACTIONS_SOURCE);
43 | TransactionsSource.Type transactionsSourceType =
44 | TransactionsSource.Type.valueOf(sourceType.toUpperCase());
45 |
46 | int transactionsPerSecond = config.get(RECORDS_PER_SECOND);
47 |
48 | switch (transactionsSourceType) {
49 | case KAFKA:
50 | Properties kafkaProps = KafkaUtils.initConsumerProperties(config);
51 | String transactionsTopic = config.get(DATA_TOPIC);
52 | FlinkKafkaConsumer011 kafkaConsumer =
53 | new FlinkKafkaConsumer011<>(transactionsTopic, new SimpleStringSchema(), kafkaProps);
54 | kafkaConsumer.setStartFromLatest();
55 | return kafkaConsumer;
56 | default:
57 | return new JsonGeneratorWrapper<>(new TransactionsGenerator(transactionsPerSecond));
58 | }
59 | }
60 |
61 | public static DataStream stringsStreamToTransactions(
62 | DataStream transactionStrings) {
63 | return transactionStrings
64 | .flatMap(new JsonDeserializer(Transaction.class))
65 | .returns(Transaction.class)
66 | .flatMap(new TimeStamper())
67 | .returns(Transaction.class)
68 | .name("Transactions Deserialization");
69 | }
70 |
71 | public enum Type {
72 | GENERATOR("Transactions Source (generated locally)"),
73 | KAFKA("Transactions Source (Kafka)");
74 |
75 | private String name;
76 |
77 | Type(String name) {
78 | this.name = name;
79 | }
80 |
81 | public String getName() {
82 | return name;
83 | }
84 | }
85 | }
86 |
--------------------------------------------------------------------------------
/flink-job/src/main/java/com/ververica/field/sources/BaseGenerator.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, 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.ververica.field.sources;
20 |
21 | import static org.apache.flink.util.Preconditions.checkArgument;
22 |
23 | import java.util.SplittableRandom;
24 | import org.apache.flink.api.common.state.ListState;
25 | import org.apache.flink.api.common.state.ListStateDescriptor;
26 | import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
27 | import org.apache.flink.configuration.Configuration;
28 | import org.apache.flink.runtime.state.FunctionInitializationContext;
29 | import org.apache.flink.runtime.state.FunctionSnapshotContext;
30 | import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
31 | import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
32 |
33 | /** A simple random data generator with data rate throttling logic. */
34 | public abstract class BaseGenerator extends RichParallelSourceFunction
35 | implements CheckpointedFunction {
36 |
37 | private static final long serialVersionUID = 1L;
38 |
39 | protected int maxRecordsPerSecond;
40 |
41 | private volatile boolean running = true;
42 |
43 | private long id = -1;
44 |
45 | private transient ListState idState;
46 |
47 | protected BaseGenerator() {
48 | this.maxRecordsPerSecond = -1;
49 | }
50 |
51 | protected BaseGenerator(int maxRecordsPerSecond) {
52 | checkArgument(
53 | maxRecordsPerSecond == -1 || maxRecordsPerSecond > 0,
54 | "maxRecordsPerSecond must be positive or -1 (infinite)");
55 | this.maxRecordsPerSecond = maxRecordsPerSecond;
56 | }
57 |
58 | @Override
59 | public void open(Configuration parameters) throws Exception {
60 | if (id == -1) {
61 | id = getRuntimeContext().getIndexOfThisSubtask();
62 | }
63 | }
64 |
65 | @Override
66 | public final void run(SourceContext ctx) throws Exception {
67 | final int numberOfParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks();
68 | final Throttler throttler = new Throttler(maxRecordsPerSecond, numberOfParallelSubtasks);
69 | final SplittableRandom rnd = new SplittableRandom();
70 |
71 | final Object lock = ctx.getCheckpointLock();
72 |
73 | while (running) {
74 | T event = randomEvent(rnd, id);
75 |
76 | synchronized (lock) {
77 | ctx.collect(event);
78 | id += numberOfParallelSubtasks;
79 | }
80 |
81 | throttler.throttle();
82 | }
83 | }
84 |
85 | @Override
86 | public final void cancel() {
87 | running = false;
88 | }
89 |
90 | @Override
91 | public final void snapshotState(FunctionSnapshotContext context) throws Exception {
92 | idState.clear();
93 | idState.add(id);
94 | }
95 |
96 | @Override
97 | public void initializeState(FunctionInitializationContext context) throws Exception {
98 | idState =
99 | context
100 | .getOperatorStateStore()
101 | .getUnionListState(new ListStateDescriptor<>("ids", BasicTypeInfo.LONG_TYPE_INFO));
102 |
103 | if (context.isRestored()) {
104 | long max = Long.MIN_VALUE;
105 | for (Long value : idState.get()) {
106 | max = Math.max(max, value);
107 | }
108 |
109 | id = max + getRuntimeContext().getIndexOfThisSubtask();
110 | }
111 | }
112 |
113 | public abstract T randomEvent(SplittableRandom rnd, long id);
114 |
115 | public int getMaxRecordsPerSecond() {
116 | return maxRecordsPerSecond;
117 | }
118 | }
119 |
--------------------------------------------------------------------------------
/flink-job/src/main/java/com/ververica/field/sources/Throttler.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, 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.ververica.field.sources;
20 |
21 | import org.apache.flink.util.Preconditions;
22 |
23 | /** Utility to throttle a thread to a given number of executions (records) per second. */
24 | final class Throttler {
25 |
26 | private final long throttleBatchSize;
27 | private final long nanosPerBatch;
28 |
29 | private long endOfNextBatchNanos;
30 | private int currentBatch;
31 |
32 | Throttler(long maxRecordsPerSecond, int numberOfParallelSubtasks) {
33 | Preconditions.checkArgument(
34 | maxRecordsPerSecond == -1 || maxRecordsPerSecond > 0,
35 | "maxRecordsPerSecond must be positive or -1 (infinite)");
36 | Preconditions.checkArgument(
37 | numberOfParallelSubtasks > 0, "numberOfParallelSubtasks must be greater than 0");
38 |
39 | if (maxRecordsPerSecond == -1) {
40 | // unlimited speed
41 | throttleBatchSize = -1;
42 | nanosPerBatch = 0;
43 | endOfNextBatchNanos = System.nanoTime() + nanosPerBatch;
44 | currentBatch = 0;
45 | return;
46 | }
47 | final float ratePerSubtask = (float) maxRecordsPerSecond / numberOfParallelSubtasks;
48 |
49 | if (ratePerSubtask >= 10000) {
50 | // high rates: all throttling in intervals of 2ms
51 | throttleBatchSize = (int) ratePerSubtask / 500;
52 | nanosPerBatch = 2_000_000L;
53 | } else {
54 | throttleBatchSize = ((int) (ratePerSubtask / 20)) + 1;
55 | nanosPerBatch = ((int) (1_000_000_000L / ratePerSubtask)) * throttleBatchSize;
56 | }
57 | this.endOfNextBatchNanos = System.nanoTime() + nanosPerBatch;
58 | this.currentBatch = 0;
59 | }
60 |
61 | void throttle() throws InterruptedException {
62 | if (throttleBatchSize == -1) {
63 | return;
64 | }
65 | if (++currentBatch != throttleBatchSize) {
66 | return;
67 | }
68 | currentBatch = 0;
69 |
70 | final long now = System.nanoTime();
71 | final int millisRemaining = (int) ((endOfNextBatchNanos - now) / 1_000_000);
72 |
73 | if (millisRemaining > 0) {
74 | endOfNextBatchNanos += nanosPerBatch;
75 | Thread.sleep(millisRemaining);
76 | } else {
77 | endOfNextBatchNanos = now + nanosPerBatch;
78 | }
79 | }
80 | }
81 |
--------------------------------------------------------------------------------
/flink-job/src/main/resources/log4j2.properties:
--------------------------------------------------------------------------------
1 | appender.console.type=Console
2 | appender.console.name=STDOUT
3 | appender.console.layout.type=PatternLayout
4 | appender.console.layout.pattern=%d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n
5 |
6 | logger.console.name=org.apache.flink.fs.s3base.shaded.com.amazonaws.latency
7 | logger.console.level=WARN
8 |
9 | rootLogger.level=INFO
10 | rootLogger.appenderRef.console.ref=STDOUT
--------------------------------------------------------------------------------
/flink-job/src/main/resources/logback.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n
5 |
6 |
7 |
8 |
9 |
10 |
11 |
--------------------------------------------------------------------------------
/flink-job/src/test/java/com/ververica/field/config/ConfigTest.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, 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.ververica.field.config;
20 |
21 | import static com.ververica.field.config.Parameters.KAFKA_HOST;
22 | import static com.ververica.field.config.Parameters.KAFKA_PORT;
23 | import static org.junit.Assert.assertEquals;
24 |
25 | import org.junit.Test;
26 |
27 | public class ConfigTest {
28 |
29 | @Test
30 | public void testParameters() {
31 | String[] args = new String[] {"--kafka-host", "host-from-args"};
32 | Parameters parameters = Parameters.fromArgs(args);
33 | Config config = Config.fromParameters(parameters);
34 |
35 | final String kafkaHost = config.get(KAFKA_HOST);
36 | assertEquals("Wrong config parameter retrived", "host-from-args", kafkaHost);
37 | }
38 |
39 | @Test
40 | public void testParameterWithDefaults() {
41 | String[] args = new String[] {};
42 | Parameters parameters = Parameters.fromArgs(args);
43 | Config config = Config.fromParameters(parameters);
44 |
45 | final Integer kafkaPort = config.get(KAFKA_PORT);
46 | assertEquals("Wrong config parameter retrived", new Integer(9092), kafkaPort);
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/flink-job/src/test/java/com/ververica/field/dynamicrules/RuleParserTest.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, 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.ververica.field.dynamicrules;
20 |
21 | import static org.junit.Assert.assertEquals;
22 |
23 | import com.ververica.field.dynamicrules.Rule.AggregatorFunctionType;
24 | import com.ververica.field.dynamicrules.Rule.LimitOperatorType;
25 | import com.ververica.field.dynamicrules.Rule.RuleState;
26 | import java.math.BigDecimal;
27 | import java.util.Arrays;
28 | import java.util.List;
29 | import org.junit.Assert;
30 | import org.junit.Test;
31 |
32 | public class RuleParserTest {
33 |
34 | @SafeVarargs
35 | public static List lst(T... ts) {
36 | return Arrays.asList(ts);
37 | }
38 |
39 | @Test
40 | public void testRuleParsedPlain() throws Exception {
41 | String ruleString1 = "1,(active),(taxiId&driverId),,(totalFare),(sum),(>),(5),(20)";
42 |
43 | RuleParser ruleParser = new RuleParser();
44 | Rule rule1 = ruleParser.fromString(ruleString1);
45 |
46 | assertEquals("ID incorrect", 1, (int) rule1.getRuleId());
47 | Assert.assertEquals("Rule state incorrect", RuleState.ACTIVE, rule1.getRuleState());
48 | assertEquals("Key names incorrect", lst("taxiId", "driverId"), rule1.getGroupingKeyNames());
49 | assertEquals("Unique names incorrect", lst(), rule1.getUnique());
50 | assertEquals("Cumulative key incorrect", "totalFare", rule1.getAggregateFieldName());
51 | Assert.assertEquals(
52 | "Aggregator function incorrect",
53 | AggregatorFunctionType.SUM,
54 | rule1.getAggregatorFunctionType());
55 | Assert.assertEquals(
56 | "Limit operator incorrect", LimitOperatorType.GREATER, rule1.getLimitOperatorType());
57 | assertEquals("Limit incorrect", BigDecimal.valueOf(5), rule1.getLimit());
58 | assertEquals("Window incorrect", 20, (int) rule1.getWindowMinutes());
59 | }
60 |
61 | @Test
62 | public void testRuleParsedJson() throws Exception {
63 | String ruleString1 =
64 | "{\n"
65 | + " \"ruleId\": 1,\n"
66 | + " \"ruleState\": \"ACTIVE\",\n"
67 | + " \"groupingKeyNames\": [\"taxiId\", \"driverId\"],\n"
68 | + " \"unique\": [],\n"
69 | + " \"aggregateFieldName\": \"totalFare\",\n"
70 | + " \"aggregatorFunctionType\": \"SUM\",\n"
71 | + " \"limitOperatorType\": \"GREATER\",\n"
72 | + " \"limit\": 50,\n"
73 | + " \"windowMinutes\": 20\n"
74 | + "}";
75 |
76 | RuleParser ruleParser = new RuleParser();
77 | Rule rule1 = ruleParser.fromString(ruleString1);
78 |
79 | assertEquals("ID incorrect", 1, (int) rule1.getRuleId());
80 | Assert.assertEquals("Rule state incorrect", RuleState.ACTIVE, rule1.getRuleState());
81 | assertEquals("Key names incorrect", lst("taxiId", "driverId"), rule1.getGroupingKeyNames());
82 | assertEquals("Unique names incorrect", lst(), rule1.getUnique());
83 | assertEquals("Cumulative key incorrect", "totalFare", rule1.getAggregateFieldName());
84 | Assert.assertEquals(
85 | "Aggregator function incorrect",
86 | AggregatorFunctionType.SUM,
87 | rule1.getAggregatorFunctionType());
88 | Assert.assertEquals(
89 | "Limit operator incorrect", LimitOperatorType.GREATER, rule1.getLimitOperatorType());
90 | assertEquals("Limit incorrect", BigDecimal.valueOf(50), rule1.getLimit());
91 | assertEquals("Window incorrect", 20, (int) rule1.getWindowMinutes());
92 | }
93 | }
94 |
--------------------------------------------------------------------------------
/flink-job/src/test/java/com/ververica/field/dynamicrules/util/AssertUtils.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, 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.ververica.field.dynamicrules.util;
20 |
21 | import java.util.HashMap;
22 | import java.util.Map;
23 | import java.util.Map.Entry;
24 | import org.apache.flink.api.common.state.BroadcastState;
25 | import org.junit.Assert;
26 |
27 | public class AssertUtils {
28 |
29 | public static void assertEquals(
30 | BroadcastState broadcastState, Map expectedState, String message)
31 | throws Exception {
32 |
33 | Map broadcastStateMap = new HashMap<>();
34 | for (Entry entry : broadcastState.entries()) {
35 | broadcastStateMap.put(entry.getKey(), entry.getValue());
36 | }
37 | Assert.assertEquals(message, broadcastStateMap, expectedState);
38 | }
39 | }
40 |
--------------------------------------------------------------------------------
/webapp/.dockerignore:
--------------------------------------------------------------------------------
1 | .git
2 | node_modules
3 | build
4 | .idea
5 | #target
--------------------------------------------------------------------------------
/webapp/.eslintrc:
--------------------------------------------------------------------------------
1 | {
2 | "extends": ["react-app"]
3 | }
4 |
--------------------------------------------------------------------------------
/webapp/.gitignore:
--------------------------------------------------------------------------------
1 | # Compiled class file
2 | *.class
3 |
4 | # Log file
5 | *.log
6 |
7 | # BlueJ files
8 | *.ctxt
9 |
10 | # Mobile Tools for Java (J2ME)
11 | .mtj.tmp/
12 |
13 | # Package Files #
14 | *.jar
15 | *.war
16 | *.nar
17 | *.ear
18 | *.zip
19 | *.tar.gz
20 | *.rar
21 |
22 | # virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml
23 | hs_err_pid*
24 |
25 | .idea
26 | target
27 | *.iml
28 |
29 |
30 | # See https://help.github.com/articles/ignoring-files/ for more about ignoring files.
31 |
32 | # dependencies
33 | /node_modules
34 | /.pnp
35 | .pnp.js
36 |
37 | # testing
38 | /coverage
39 |
40 | # production
41 | /build
42 |
43 | # misc
44 | .DS_Store
45 | .env.local
46 | .env.development.local
47 | .env.test.local
48 | .env.production.local
49 |
50 | npm-debug.log*
51 | yarn-debug.log*
52 | yarn-error.log*
53 | node_modules/
54 |
55 | .vscode
56 | .project
57 | /bin
58 | .gradle
59 |
60 |
61 | # Temporary
62 | .npmrc
--------------------------------------------------------------------------------
/webapp/.mvn/wrapper/maven-wrapper.properties:
--------------------------------------------------------------------------------
1 | distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.6.1/apache-maven-3.6.1-bin.zip
2 | wrapperUrl=https://repo.maven.apache.org/maven2/io/takari/maven-wrapper/0.5.5/maven-wrapper-0.5.5.jar
3 |
--------------------------------------------------------------------------------
/webapp/.prettierrc:
--------------------------------------------------------------------------------
1 | {
2 | "printWidth": 120,
3 | "trailingComma": "es5"
4 | }
5 |
--------------------------------------------------------------------------------
/webapp/Dockerfile:
--------------------------------------------------------------------------------
1 | # Start with a base image containing Java runtime
2 | FROM openjdk:8-jdk-alpine
3 |
4 | # Add Maintainer Info
5 | LABEL maintainer="alexander@ververica.com"
6 |
7 | # Add a volume pointing to /tmp
8 | VOLUME /tmp
9 |
10 | # Make port 5656 available to the world outside this container
11 | EXPOSE 5656
12 |
13 | ADD target/demo-backend-*.jar demo-backend.jar
14 |
15 | # Run the jar file
16 | ENTRYPOINT ["java","-Djava.security.egd=file:/dev/./urandom","-Dspring.profiles.active=dev","-jar","/demo-backend.jar"]
--------------------------------------------------------------------------------
/webapp/README.md:
--------------------------------------------------------------------------------
1 | To start Kafka:
2 |
3 | ```
4 | cd demo-backend
5 | docker-compose --log-level CRITICAL up
6 | ```
7 |
8 | To start backend/host frontend:
9 |
10 | ```
11 | mvn clean install spring-boot:run
12 | ```
13 |
14 | React App:
15 | The react app is now set to start both the java app and the react dev server, simply run
16 |
17 | ```
18 | npm install (to get all the new dependencies)
19 | npm start
20 | ```
21 |
22 | Two URLs:
23 |
24 | ```
25 | http://localhost:5656/transactions
26 | http://localhost:5656/temerature
27 | ```
28 |
29 | Swagger is available under:
30 |
31 | ```
32 | http://localhost:5656/swagger-ui.html
33 | ```
34 |
35 | Example Rule JSON:
36 |
37 | ```
38 | {
39 | "ruleId":1,
40 | "ruleState":"ACTIVE",
41 | "groupingKeyNames":[
42 | "paymentType"
43 | ],
44 | "unique":[],
45 | "aggregateFieldName":"paymentAmount",
46 | "aggregatorFunctionType":"SUM",
47 | "limitOperatorType":"GREATER",
48 | "limit":50,
49 | "windowMinutes":20
50 | }
51 | ```
52 |
53 | where
54 |
55 | ```
56 | ruleState in ["ACTIVE", "PAUSE", "DELETE"]
57 | aggregateFunctionType in [SUM, AVG, MIN, MAX]
58 | limitOperatorType in [EQUAL("="), NOT_EQUAL("!="), GREATER_EQUAL(">="), LESS_EQUAL("<="), GREATER(">"),LESS("<")]
59 | ```
60 |
61 | H2 Console:
62 |
63 | ```
64 | URL: http://localhost:5656/h2-console/
65 | ```
66 |
67 | | Setting | Value |
68 | | ------------ | ------------------ |
69 | | Driver Class | org.h2.Driver |
70 | | JDBC URL | jdbc:h2:mem:testdb |
71 | | User Name | sa |
72 | | Password | |
73 |
--------------------------------------------------------------------------------
/webapp/cloud.Dockerfile:
--------------------------------------------------------------------------------
1 | # --- UI Build
2 | FROM node:10 as ui-build
3 | WORKDIR /home/node/app
4 |
5 | COPY package.json package-lock.json tsconfig.json ./
6 | COPY scripts scripts
7 | RUN npm ci --unsafe-perm
8 | COPY public public
9 | COPY src/app src/app
10 | COPY src/index.tsx src
11 | COPY src/react-app-env.d.ts src
12 | RUN npm run build
13 |
14 | # --- Maven Build
15 | FROM maven:3.6.2-jdk-8-openj9 as maven-build
16 | WORKDIR /home/maven/work
17 |
18 | COPY pom.xml .
19 | RUN mvn -B -e -C -T 1C org.apache.maven.plugins:maven-dependency-plugin:3.1.1:go-offline
20 | COPY . .
21 | COPY --from=ui-build /home/node/app/build /home/maven/work/target/classes/static/
22 | RUN mvn -B -e -o -T 1C verify
23 | RUN mv target/demo-fraud-webapp*.jar target/demo-fraud-webapp.jar
24 |
25 | # --- Main container
26 | FROM openjdk:8-jdk-alpine as main
27 |
28 | COPY --from=maven-build /home/maven/work/target/demo-fraud-webapp.jar .
29 | EXPOSE 5656
30 |
31 | ENTRYPOINT ["java","-Djava.security.egd=file:/dev/./urandom","-Dspring.profiles.active=cloud","-jar","demo-fraud-webapp.jar"]
32 |
--------------------------------------------------------------------------------
/webapp/cloudbuild.yaml:
--------------------------------------------------------------------------------
1 | steps:
2 | - name: 'gcr.io/cloud-builders/npm'
3 | entrypoint: npm
4 | args: ['install']
5 | - name: 'gcr.io/cloud-builders/npm'
6 | entrypoint: npm
7 | args: ['run', 'postinstall']
8 | - name: 'gcr.io/cloud-builders/npm'
9 | entrypoint: npm
10 | args: ['run', 'build']
11 | - name: 'ubuntu'
12 | args: ['bash', '-c', 'ls', '-la']
13 | - name: maven:3.6.1-jdk-8-slim
14 | entrypoint: 'mvn'
15 | args: ['--no-transfer-progress', '--batch-mode', 'package']
16 | - name: 'ubuntu'
17 | args: ['bash', '-c', 'ls', '-la']
18 | - name: 'gcr.io/cloud-builders/docker'
19 | args: ['build', '-t', 'gcr.io/$PROJECT_ID/demo-backend', '.']
20 |
21 |
22 | images:
23 | - 'gcr.io/$PROJECT_ID/demo-backend'
--------------------------------------------------------------------------------
/webapp/mvn/maven-wrapper.properties:
--------------------------------------------------------------------------------
1 | distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.6.1/apache-maven-3.6.1-bin.zip
2 | wrapperUrl=https://repo.maven.apache.org/maven2/io/takari/maven-wrapper/0.5.5/maven-wrapper-0.5.5.jar
3 |
--------------------------------------------------------------------------------
/webapp/package.json:
--------------------------------------------------------------------------------
1 | {
2 | "name": "front-end",
3 | "version": "0.1.0",
4 | "author": "Daryl Roberts ",
5 | "private": true,
6 | "dependencies": {
7 | "@fortawesome/fontawesome-svg-core": "^1.2.25",
8 | "@fortawesome/free-solid-svg-icons": "^5.11.2",
9 | "@fortawesome/react-fontawesome": "^0.1.7",
10 | "@types/uuid": "^3.4.5",
11 | "axios": "^0.21.1",
12 | "bootstrap": "^4.3.1",
13 | "bootswatch": "^4.3.1",
14 | "get-form-data": "^2.0.0",
15 | "leader-line": "^1.0.5",
16 | "lodash": "^4.17.15",
17 | "react": "^16.8.6",
18 | "react-dom": "^16.8.6",
19 | "react-rangeslider": "^2.2.0",
20 | "react-select": "^3.0.4",
21 | "react-stomp": "^4.1.1",
22 | "react-use": "^10.6.2",
23 | "react-virtualized": "^9.21.1",
24 | "reactstrap": "^8.0.1",
25 | "styled-components": "^4.3.2",
26 | "uuid": "^3.3.3"
27 | },
28 | "devDependencies": {
29 | "@types/classnames": "^2.2.9",
30 | "@types/lodash": "^4.14.137",
31 | "@types/react": "^16.9.2",
32 | "@types/react-dom": "^16.8.5",
33 | "@types/react-select": "^3.0.2",
34 | "@types/react-rangeslider": "^2.2.1",
35 | "@types/react-virtualized": "^9.21.4",
36 | "@types/reactstrap": "^8.0.1",
37 | "@types/styled-components": "^4.1.18",
38 | "concurrently": "^4.1.2",
39 | "http-proxy-middleware": "^0.19.1",
40 | "node-sass": "^4.12.0",
41 | "react-scripts": "3.0.1",
42 | "tslint": "^5.19.0",
43 | "tslint-config-prettier": "^1.18.0",
44 | "tslint-react": "^4.0.0",
45 | "typescript": "^3.5.3"
46 | },
47 | "eslintConfig": {
48 | "extends": "react-app"
49 | },
50 | "browserslist": {
51 | "production": [
52 | ">0.2%",
53 | "not dead",
54 | "not op_mini all"
55 | ],
56 | "development": [
57 | "last 1 chrome version",
58 | "last 1 firefox version",
59 | "last 1 safari version"
60 | ]
61 | },
62 | "scripts": {
63 | "build": "react-scripts build",
64 | "client": "react-scripts start",
65 | "postinstall": "./scripts/fix-leader-line.sh",
66 | "server": "mvn clean install spring-boot:run",
67 | "start-kafka": "docker-compose --log-level CRITICAL up -d",
68 | "start": "concurrently --names 'client,server' 'npm run client' 'npm run server'",
69 | "stop-kafka": "docker-compose down -v",
70 | "test": "react-scripts test"
71 | }
72 | }
73 |
--------------------------------------------------------------------------------
/webapp/public/favicon.ico:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/afedulov/fraud-detection-demo/1bc0450b1a86440c276d49e78aae8351883eecf6/webapp/public/favicon.ico
--------------------------------------------------------------------------------
/webapp/public/index.html:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 |
5 |
6 |
7 |
8 |
17 | React App
18 |
19 |
20 |
21 |
22 |
32 |
33 |
34 |
--------------------------------------------------------------------------------
/webapp/scripts/fix-leader-line.sh:
--------------------------------------------------------------------------------
1 | #!/bin/bash
2 |
3 | if grep -q 'module.exports = LeaderLine' ./node_modules/leader-line/leader-line.min.js; then
4 | echo 'Leader Line already patched'
5 | else
6 | echo "\
7 | if (module && module.exports) { module.exports = LeaderLine }\
8 | " >> ./node_modules/leader-line/leader-line.min.js
9 | echo "Fixed LeaderLine!"
10 | fi;
11 |
12 |
13 |
--------------------------------------------------------------------------------
/webapp/src/app/assets/app.scss:
--------------------------------------------------------------------------------
1 | @import "./theme";
2 |
3 | body {
4 | // width: 1920px;
5 | // height: 1080px;
6 | position: relative;
7 | overflow: hidden;
8 | }
9 |
10 | .react-select {
11 | & &__control {
12 | @extend .form-control-sm;
13 | min-height: 0;
14 | height: auto;
15 | border-color: $gray-400;
16 |
17 | &--is-focused {
18 | border-color: $input-focus-border-color;
19 | box-shadow: 0 0 0 0.2rem rgba($primary, 0.25);
20 | }
21 |
22 | &:hover {
23 | border-color: $input-focus-border-color;
24 | }
25 | }
26 |
27 | & &__value-container {
28 | align-items: center;
29 | padding: 0;
30 | line-height: 1;
31 |
32 | & [class*="Input"] {
33 | margin-top: 0;
34 | margin-bottom: 0;
35 | padding: 0;
36 | }
37 | }
38 |
39 | & &__indicator {
40 | padding: 0.5px;
41 |
42 | &-separator {
43 | margin-left: 6px;
44 | margin-top: 0;
45 | margin-bottom: 0;
46 | }
47 | }
48 |
49 | & &__clear-indicator {
50 | &:hover {
51 | color: $primary;
52 | }
53 | }
54 |
55 | & &__dropdown-indicator {
56 | color: $gray-700;
57 | margin-right: -1px;
58 | margin-left: 6px;
59 |
60 | &:hover {
61 | color: $primary;
62 | }
63 | }
64 |
65 | & &__option {
66 | padding-top: 1px;
67 | padding-bottom: 1px;
68 | }
69 | }
70 |
71 | .ReactVirtualized__List {
72 | &:focus {
73 | outline: 0;
74 | }
75 |
76 | &::-webkit-scrollbar {
77 | display: none;
78 | }
79 | }
80 |
--------------------------------------------------------------------------------
/webapp/src/app/assets/flink_squirrel_200_color.png:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/afedulov/fraud-detection-demo/1bc0450b1a86440c276d49e78aae8351883eecf6/webapp/src/app/assets/flink_squirrel_200_color.png
--------------------------------------------------------------------------------
/webapp/src/app/assets/theme.scss:
--------------------------------------------------------------------------------
1 | $enable-rounded: false;
2 |
3 | // Flink Colors
4 | // $warning: #eea43c;
5 | // $primary: #b479eb;
6 |
7 | @import "~bootstrap/scss/bootstrap";
8 |
--------------------------------------------------------------------------------
/webapp/src/app/components/Alerts.tsx:
--------------------------------------------------------------------------------
1 | import React, { FC } from "react";
2 | import { Button, CardBody, CardHeader, Table, CardFooter, Badge } from "reactstrap";
3 | import styled from "styled-components/macro";
4 | import { faArrowRight } from "@fortawesome/free-solid-svg-icons";
5 | import { FontAwesomeIcon } from "@fortawesome/react-fontawesome";
6 |
7 | import { Alert } from "../interfaces";
8 | import { CenteredContainer } from "./CenteredContainer";
9 | import { ScrollingCol } from "./App";
10 | import { Payment, Payee, Details, Beneficiary, paymentTypeMap } from "./Transactions";
11 | import { Line } from "app/utils/useLines";
12 |
13 | const AlertTable = styled(Table)`
14 | && {
15 | width: calc(100% + 1px);
16 | border: 0;
17 | margin: 0;
18 |
19 | td {
20 | vertical-align: middle !important;
21 |
22 | &:first-child {
23 | border-left: 0;
24 | }
25 |
26 | &:last-child {
27 | border-right: 0;
28 | }
29 | }
30 |
31 | tr:first-child {
32 | td {
33 | border-top: 0;
34 | }
35 | }
36 | }
37 | `;
38 |
39 | export const Alerts: FC = props => {
40 | const tooManyAlerts = props.alerts.length > 4;
41 |
42 | const handleScroll = () => {
43 | props.lines.forEach(line => line.line.position());
44 | };
45 |
46 | return (
47 |
48 | {props.alerts.map((alert, idx) => {
49 | const t = alert.triggeringEvent;
50 | return (
51 |
58 |
59 | Alert
60 |
63 |
64 |
65 |
66 |
67 |
68 | Transaction |
69 | {alert.triggeringEvent.transactionId} |
70 |
71 |
72 |
73 |
74 | {t.payeeId}
75 |
76 |
77 | ${parseFloat(t.paymentAmount.toString()).toFixed(2)}
78 |
79 |
80 | {t.beneficiaryId}
81 |
82 | |
83 |
84 |
85 | Rule |
86 | {alert.ruleId} |
87 |
88 |
89 | Amount |
90 | {alert.triggeringValue} |
91 |
92 |
93 | Of |
94 | {alert.violatedRule.aggregateFieldName} |
95 |
96 |
97 |
98 |
99 |
100 | Alert for Rule {alert.ruleId} caused by Transaction{" "}
101 | {alert.triggeringEvent.transactionId} with Amount {alert.triggeringValue} of{" "}
102 | {alert.violatedRule.aggregateFieldName}.
103 |
104 |
105 | );
106 | })}
107 |
108 | );
109 | };
110 |
111 | interface Props {
112 | alerts: Alert[];
113 | clearAlert: any;
114 | lines: Line[];
115 | // handleScroll: () => void;
116 | }
117 |
--------------------------------------------------------------------------------
/webapp/src/app/components/CenteredContainer.tsx:
--------------------------------------------------------------------------------
1 | import React, { forwardRef, ReactNode, CSSProperties } from "react";
2 | import { Card } from "reactstrap";
3 | import cx from "classnames";
4 |
5 | export const CenteredContainer = forwardRef((props: Props, ref) => {
6 | return (
7 |
12 | {props.children}
13 |
14 | );
15 | });
16 |
17 | interface Props {
18 | tooManyItems: boolean;
19 | children: ReactNode;
20 | className?: string;
21 | style?: CSSProperties;
22 | }
23 |
--------------------------------------------------------------------------------
/webapp/src/app/components/FieldGroup.tsx:
--------------------------------------------------------------------------------
1 | import { IconDefinition } from "@fortawesome/free-solid-svg-icons";
2 | import { FontAwesomeIcon } from "@fortawesome/react-fontawesome";
3 | import React, { FC } from "react";
4 | import { Col, FormGroup, Label } from "reactstrap";
5 | import styled from "styled-components";
6 |
7 | const LabelColumn = styled(Label)`
8 | text-align: right;
9 | white-space: nowrap;
10 | overflow: hidden;
11 | text-overflow: ellipsis;
12 | flex-basis: 33%;
13 | flex: 1 1 auto;
14 | `;
15 |
16 | const InputColumn = styled(Col)`
17 | flex-basis: 67%;
18 | flex: 1 1 auto;
19 | `;
20 |
21 | export const FieldGroup: FC = props => (
22 |
23 |
24 |
25 | {props.label}
26 |
27 | {props.children}
28 |
29 | );
30 |
31 | interface Props {
32 | label: string;
33 | icon: IconDefinition;
34 | }
35 |
--------------------------------------------------------------------------------
/webapp/src/app/components/Header.tsx:
--------------------------------------------------------------------------------
1 | import logoImage from "app/assets/flink_squirrel_200_color.png";
2 | import React, { FC, useState, Dispatch, SetStateAction } from "react";
3 | import { Button, ButtonGroup, Col, Navbar, NavbarBrand } from "reactstrap";
4 | import styled from "styled-components/macro";
5 | import { AddRuleModal } from "./AddRuleModal";
6 | import { Rule } from "app/interfaces";
7 |
8 | const AppNavbar = styled(Navbar)`
9 | && {
10 | z-index: 1;
11 | justify-content: flex-start;
12 | padding: 0;
13 | }
14 | `;
15 |
16 | const Logo = styled.img`
17 | max-height: 40px;
18 | `;
19 |
20 | const TransactionsCol = styled(Col)`
21 | border-right: 1px solid rgba(255, 255, 255, 0.125);
22 | display: flex;
23 | align-items: center;
24 | justify-content: space-between;
25 | padding: 0.5em 15px;
26 | `;
27 |
28 | export const Header: FC = props => {
29 | const [modalOpen, setModalOpen] = useState(false);
30 | const openRuleModal = () => setModalOpen(true);
31 | const closeRuleModal = () => setModalOpen(false);
32 | const toggleRuleModal = () => setModalOpen(state => !state);
33 |
34 | const startTransactions = () => fetch("/api/startTransactionsGeneration").then();
35 | const stopTransactions = () => fetch("/api/stopTransactionsGeneration").then();
36 |
37 | const syncRules = () => fetch("/api/syncRules").then();
38 | const clearState = () => fetch("/api/clearState").then();
39 | const pushToFlink = () => fetch("/api/rules/pushToFlink").then();
40 |
41 | return (
42 | <>
43 |
44 |
45 | Live Transactions
46 |
47 |
50 |
53 |
54 |
55 |
56 |
57 |
60 |
61 |
64 |
65 |
68 |
69 |
72 |
73 |
74 |
75 | Apache Flink - Fraud Detection Demo
76 |
77 |
78 |
79 |
80 | >
81 | );
82 | };
83 |
84 | interface Props {
85 | setRules: Dispatch>;
86 | }
87 |
--------------------------------------------------------------------------------
/webapp/src/app/components/index.ts:
--------------------------------------------------------------------------------
1 | export { AddRuleModal } from "./AddRuleModal";
2 | export { App } from "./App";
3 | export { Header } from "./Header";
4 | export { Alerts } from "./Alerts";
5 | export { Rules } from "./Rules";
6 | export { Transactions } from "./Transactions";
7 |
--------------------------------------------------------------------------------
/webapp/src/app/interfaces/Alert.ts:
--------------------------------------------------------------------------------
1 | import { Transaction } from "./Transaction";
2 | import { RefObject } from "react";
3 | import { RulePayload } from "./Rule";
4 |
5 | export interface Alert {
6 | alertId: string;
7 | ruleId: number;
8 | violatedRule: RulePayload;
9 | triggeringValue: number;
10 | triggeringEvent: Transaction;
11 | ref: RefObject;
12 | }
13 |
--------------------------------------------------------------------------------
/webapp/src/app/interfaces/Rule.ts:
--------------------------------------------------------------------------------
1 | import { RefObject } from "react";
2 |
3 | export interface Rule {
4 | id: number;
5 | rulePayload: string;
6 | ref: RefObject;
7 | }
8 |
9 | export interface RulePayload {
10 | aggregateFieldName: string;
11 | aggregatorFunctionType: string;
12 | groupingKeyNames: string[];
13 | limit: number;
14 | limitOperatorType: string;
15 | windowMinutes: number;
16 | ruleState: string;
17 | }
18 |
--------------------------------------------------------------------------------
/webapp/src/app/interfaces/Transaction.ts:
--------------------------------------------------------------------------------
1 | // import { RefObject } from "react";
2 |
3 | // MSG
4 | // beneficiaryId: 42694
5 | // eventTime: 1565965071385
6 | // payeeId: 20908
7 | // paymentAmount: 13.54
8 | // paymentType: "CRD"
9 | // transactionId: 5954524216210268000
10 |
11 | export interface Transaction {
12 | beneficiaryId: number;
13 | eventTime: number;
14 | payeeId: number;
15 | paymentAmount: number;
16 | paymentType: string;
17 | transactionId: number;
18 | }
19 |
--------------------------------------------------------------------------------
/webapp/src/app/interfaces/index.ts:
--------------------------------------------------------------------------------
1 | export * from "./Rule";
2 | export * from "./Transaction";
3 | export * from "./Alert";
4 |
--------------------------------------------------------------------------------
/webapp/src/app/utils/index.ts:
--------------------------------------------------------------------------------
1 | export { useLines } from "./useLines";
2 |
--------------------------------------------------------------------------------
/webapp/src/app/utils/useLines.ts:
--------------------------------------------------------------------------------
1 | import { Alert, Rule } from "app/interfaces";
2 | import LeaderLine from "leader-line";
3 | import { flattenDeep } from "lodash/fp";
4 | import { RefObject, useCallback, useEffect, useState } from "react";
5 |
6 | export const useLines: UseLines = (transactionsRef, rules, alerts) => {
7 | const [lines, setLines] = useState([]);
8 |
9 | const updateLines = useCallback(() => {
10 | lines.forEach(line => {
11 | try {
12 | line.line.position();
13 | } catch {
14 | // nothing
15 | }
16 | });
17 | }, [lines]);
18 |
19 | useEffect(() => {
20 | const newLines = flattenDeep(
21 | rules.map(rule => {
22 | const hasAlert = alerts.some(alert => alert.ruleId === rule.id);
23 |
24 | const inputLine = new LeaderLine(transactionsRef.current, rule.ref.current, {
25 | color: hasAlert ? "#dc3545" : undefined,
26 | dash: { animation: true },
27 | endSocket: "left",
28 | startSocket: "right",
29 | }) as Line;
30 |
31 | const outputLines = alerts.reduce((acc, alert) => {
32 | if (alert.ruleId === rule.id) {
33 | return [
34 | ...acc,
35 | new LeaderLine(rule.ref.current, alert.ref.current, {
36 | color: "#fff",
37 | endPlugOutline: true,
38 | endSocket: "left",
39 | outline: true,
40 | outlineColor: "#dc3545",
41 | startSocket: "right",
42 | }) as Line,
43 | ];
44 | }
45 | return acc;
46 | }, []);
47 |
48 | return [inputLine, ...outputLines];
49 | })
50 | );
51 |
52 | setLines(newLines);
53 |
54 | return () => {
55 | newLines.forEach(line => line.line.remove());
56 | };
57 | }, [transactionsRef, rules, alerts]);
58 |
59 | return { lines, handleScroll: updateLines };
60 | };
61 |
62 | type UseLines = (
63 | transactionsRef: RefObject,
64 | rules: Rule[],
65 | alerts: Alert[]
66 | ) => {
67 | lines: Line[];
68 | handleScroll: () => void;
69 | };
70 |
71 | export interface Line {
72 | line: {
73 | color: string;
74 | position: () => void;
75 | remove: () => void;
76 | };
77 | ruleId: number;
78 | }
79 |
--------------------------------------------------------------------------------
/webapp/src/index.tsx:
--------------------------------------------------------------------------------
1 | import React from "react";
2 | import ReactDOM from "react-dom";
3 | import { App } from "./app/components";
4 | import "react-rangeslider/umd/rangeslider.min.css";
5 |
6 | ReactDOM.render(, document.getElementById("root"));
7 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/Main.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend;
19 |
20 | import org.springframework.boot.SpringApplication;
21 | import org.springframework.boot.autoconfigure.EnableAutoConfiguration;
22 | import org.springframework.boot.autoconfigure.SpringBootApplication;
23 | import org.springframework.context.annotation.ComponentScan;
24 |
25 | @SpringBootApplication
26 | @EnableAutoConfiguration
27 | @ComponentScan("com.ververica")
28 | public class Main {
29 | public static void main(String[] args) {
30 | SpringApplication.run(Main.class, args);
31 | }
32 | }
33 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/configurations/KafkaConsumerConfig.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.configurations;
19 |
20 | import java.util.HashMap;
21 | import java.util.Map;
22 | import org.apache.kafka.clients.consumer.ConsumerConfig;
23 | import org.apache.kafka.common.serialization.StringDeserializer;
24 | import org.springframework.beans.factory.annotation.Autowired;
25 | import org.springframework.beans.factory.annotation.Value;
26 | import org.springframework.context.annotation.Bean;
27 | import org.springframework.context.annotation.Configuration;
28 | import org.springframework.kafka.annotation.EnableKafka;
29 | import org.springframework.kafka.config.ConcurrentKafkaListenerContainerFactory;
30 | import org.springframework.kafka.config.KafkaListenerContainerFactory;
31 | import org.springframework.kafka.core.ConsumerFactory;
32 | import org.springframework.kafka.core.DefaultKafkaConsumerFactory;
33 | import org.springframework.kafka.listener.ConcurrentMessageListenerContainer;
34 |
35 | @EnableKafka
36 | @Configuration
37 | public class KafkaConsumerConfig {
38 |
39 | @Autowired private PropertyLogger propertyLogger;
40 |
41 | @Value("${kafka.bootstrap-servers}")
42 | public String bootstrapServer;
43 |
44 | @Bean
45 | public Map consumerConfigs() {
46 | Map props = new HashMap<>();
47 | props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
48 | props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
49 | props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
50 | props.put(ConsumerConfig.GROUP_ID_CONFIG, "temp-groupid.group");
51 | props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest");
52 |
53 | return props;
54 | }
55 |
56 | @Bean
57 | public ConsumerFactory consumerFactory() {
58 | return new DefaultKafkaConsumerFactory<>(consumerConfigs());
59 | }
60 |
61 | @Bean
62 | public KafkaListenerContainerFactory>
63 | kafkaListenerContainerFactory() {
64 | ConcurrentKafkaListenerContainerFactory factory =
65 | new ConcurrentKafkaListenerContainerFactory();
66 | factory.setConsumerFactory(consumerFactory());
67 | return factory;
68 | }
69 | }
70 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/configurations/KafkaProducerConfig.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.configurations;
19 |
20 | import java.util.HashMap;
21 | import java.util.Map;
22 | import org.apache.kafka.clients.producer.ProducerConfig;
23 | import org.apache.kafka.common.serialization.StringSerializer;
24 | import org.springframework.beans.factory.annotation.Value;
25 | import org.springframework.context.annotation.Bean;
26 | import org.springframework.context.annotation.Configuration;
27 | import org.springframework.kafka.core.DefaultKafkaProducerFactory;
28 | import org.springframework.kafka.core.KafkaTemplate;
29 | import org.springframework.kafka.core.ProducerFactory;
30 | import org.springframework.kafka.support.serializer.JsonSerializer;
31 |
32 | @Configuration
33 | public class KafkaProducerConfig {
34 |
35 | @Value("${kafka.bootstrap-servers}")
36 | private String bootstrapServers;
37 |
38 | @Bean
39 | public Map producerConfigsJson() {
40 | Map props = new HashMap<>();
41 | props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
42 | props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
43 | props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, JsonSerializer.class);
44 | return props;
45 | }
46 |
47 | @Bean
48 | public Map producerConfigsString() {
49 | Map props = new HashMap<>();
50 | props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
51 | props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
52 | props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
53 | return props;
54 | }
55 |
56 | // Transactions
57 | @Bean
58 | public ProducerFactory producerFactoryForJson() {
59 | return new DefaultKafkaProducerFactory<>(producerConfigsJson());
60 | }
61 |
62 | @Bean
63 | public KafkaTemplate kafkaTemplateForJson() {
64 | return new KafkaTemplate<>(producerFactoryForJson());
65 | }
66 |
67 | // Strings
68 | @Bean
69 | public ProducerFactory producerFactoryForString() {
70 | return new DefaultKafkaProducerFactory<>(producerConfigsString());
71 | }
72 |
73 | @Bean
74 | public KafkaTemplate kafkaTemplateForString() {
75 | return new KafkaTemplate<>(producerFactoryForString());
76 | }
77 | }
78 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/configurations/PropertyLogger.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.configurations;
19 |
20 | import java.util.Arrays;
21 | import java.util.stream.StreamSupport;
22 | import org.slf4j.Logger;
23 | import org.slf4j.LoggerFactory;
24 | import org.springframework.beans.factory.annotation.Autowired;
25 | import org.springframework.context.ApplicationContext;
26 | import org.springframework.context.event.ContextRefreshedEvent;
27 | import org.springframework.context.event.EventListener;
28 | import org.springframework.core.env.AbstractEnvironment;
29 | import org.springframework.core.env.EnumerablePropertySource;
30 | import org.springframework.core.env.Environment;
31 | import org.springframework.core.env.MutablePropertySources;
32 | import org.springframework.stereotype.Component;
33 |
34 | @Component
35 | public class PropertyLogger {
36 |
37 | @Autowired
38 | public PropertyLogger(ApplicationContext context) {
39 | logProperties(context);
40 | }
41 |
42 | private static final Logger LOGGER = LoggerFactory.getLogger(PropertyLogger.class);
43 |
44 | @EventListener
45 | public void handleContextRefresh(ContextRefreshedEvent event) {
46 | logProperties(event.getApplicationContext());
47 | }
48 |
49 | public void logProperties(ApplicationContext context) {
50 | final Environment env = context.getEnvironment();
51 | LOGGER.info("====== Environment and configuration ======");
52 | LOGGER.info("Active profiles: {}", Arrays.toString(env.getActiveProfiles()));
53 | final MutablePropertySources sources = ((AbstractEnvironment) env).getPropertySources();
54 | StreamSupport.stream(sources.spliterator(), false)
55 | .filter(ps -> ps instanceof EnumerablePropertySource)
56 | .map(ps -> ((EnumerablePropertySource) ps).getPropertyNames())
57 | .flatMap(Arrays::stream)
58 | .distinct()
59 | .filter(
60 | prop ->
61 | !(prop.contains("credentials")
62 | || prop.contains("password")
63 | || prop.contains("java.class.path")
64 | || prop.contains("sun.boot.class.path")))
65 | .forEach(prop -> LOGGER.info("{}: {}", prop, env.getProperty(prop)));
66 | LOGGER.info("===========================================");
67 | }
68 | }
69 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/configurations/SwaggerConfig.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.configurations;
19 |
20 | import org.springframework.context.annotation.Bean;
21 | import org.springframework.context.annotation.Configuration;
22 | import springfox.documentation.builders.ApiInfoBuilder;
23 | import springfox.documentation.builders.PathSelectors;
24 | import springfox.documentation.builders.RequestHandlerSelectors;
25 | import springfox.documentation.service.ApiInfo;
26 | import springfox.documentation.spi.DocumentationType;
27 | import springfox.documentation.spring.web.plugins.Docket;
28 | import springfox.documentation.swagger2.annotations.EnableSwagger2;
29 |
30 | @Configuration
31 | @EnableSwagger2
32 | public class SwaggerConfig {
33 | @Bean
34 | public Docket api() {
35 | return new Docket(DocumentationType.SWAGGER_2)
36 | .select()
37 | .apis(RequestHandlerSelectors.basePackage("com.ververica"))
38 | .paths(PathSelectors.regex("/.*"))
39 | .build()
40 | .apiInfo(apiEndPointsInfo());
41 | }
42 |
43 | private ApiInfo apiEndPointsInfo() {
44 | return new ApiInfoBuilder()
45 | .title("Ververica Demo REST API")
46 | .description("Ververica Demo Management REST API")
47 | .license("Apache 2.0")
48 | .licenseUrl("http://www.apache.org/licenses/LICENSE-2.0.html")
49 | .version("1.0.0")
50 | .build();
51 | }
52 | }
53 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/configurations/WebSocketConfig.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.configurations;
19 |
20 | import org.springframework.context.annotation.Configuration;
21 | import org.springframework.messaging.simp.config.MessageBrokerRegistry;
22 | import org.springframework.web.socket.config.annotation.EnableWebSocketMessageBroker;
23 | import org.springframework.web.socket.config.annotation.StompEndpointRegistry;
24 | import org.springframework.web.socket.config.annotation.WebSocketMessageBrokerConfigurer;
25 |
26 | @Configuration
27 | @EnableWebSocketMessageBroker
28 | public class WebSocketConfig implements WebSocketMessageBrokerConfigurer {
29 |
30 | @Override
31 | public void registerStompEndpoints(StompEndpointRegistry registry) {
32 | registry.addEndpoint("/ws/backend").withSockJS();
33 | }
34 |
35 | @Override
36 | public void configureMessageBroker(MessageBrokerRegistry registry) {
37 | registry.enableSimpleBroker("/topic");
38 | }
39 | }
40 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/controllers/AlertsController.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.controllers;
19 |
20 | import com.fasterxml.jackson.core.JsonProcessingException;
21 | import com.fasterxml.jackson.databind.ObjectMapper;
22 | import com.ververica.demo.backend.datasource.Transaction;
23 | import com.ververica.demo.backend.entities.Rule;
24 | import com.ververica.demo.backend.exceptions.RuleNotFoundException;
25 | import com.ververica.demo.backend.model.Alert;
26 | import com.ververica.demo.backend.repositories.RuleRepository;
27 | import com.ververica.demo.backend.services.KafkaTransactionsPusher;
28 | import java.math.BigDecimal;
29 | import org.springframework.beans.factory.annotation.Autowired;
30 | import org.springframework.beans.factory.annotation.Value;
31 | import org.springframework.messaging.simp.SimpMessagingTemplate;
32 | import org.springframework.web.bind.annotation.GetMapping;
33 | import org.springframework.web.bind.annotation.PathVariable;
34 | import org.springframework.web.bind.annotation.RequestMapping;
35 | import org.springframework.web.bind.annotation.RestController;
36 |
37 | @RestController
38 | @RequestMapping("/api")
39 | public class AlertsController {
40 |
41 | private final RuleRepository repository;
42 | private final KafkaTransactionsPusher transactionsPusher;
43 | private SimpMessagingTemplate simpSender;
44 |
45 | @Value("${web-socket.topic.alerts}")
46 | private String alertsWebSocketTopic;
47 |
48 | @Autowired
49 | public AlertsController(
50 | RuleRepository repository,
51 | KafkaTransactionsPusher transactionsPusher,
52 | SimpMessagingTemplate simpSender) {
53 | this.repository = repository;
54 | this.transactionsPusher = transactionsPusher;
55 | this.simpSender = simpSender;
56 | }
57 |
58 | ObjectMapper mapper = new ObjectMapper();
59 |
60 | @GetMapping("/rules/{id}/alert")
61 | Alert mockAlert(@PathVariable Integer id) throws JsonProcessingException {
62 | Rule rule = repository.findById(id).orElseThrow(() -> new RuleNotFoundException(id));
63 | Transaction triggeringEvent = transactionsPusher.getLastTransaction();
64 | String violatedRule = rule.getRulePayload();
65 | BigDecimal triggeringValue = triggeringEvent.getPaymentAmount().multiply(new BigDecimal(10));
66 |
67 | Alert alert = new Alert(rule.getId(), violatedRule, triggeringEvent, triggeringValue);
68 |
69 | String result = mapper.writeValueAsString(alert);
70 |
71 | simpSender.convertAndSend(alertsWebSocketTopic, result);
72 |
73 | return alert;
74 | }
75 | }
76 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/controllers/DataGenerationController.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.controllers;
19 |
20 | import com.ververica.demo.backend.datasource.DemoTransactionsGenerator;
21 | import com.ververica.demo.backend.datasource.TransactionsGenerator;
22 | import com.ververica.demo.backend.services.KafkaTransactionsPusher;
23 | import java.util.concurrent.ExecutorService;
24 | import java.util.concurrent.Executors;
25 | import lombok.extern.slf4j.Slf4j;
26 | import org.springframework.beans.factory.annotation.Autowired;
27 | import org.springframework.beans.factory.annotation.Value;
28 | import org.springframework.kafka.config.KafkaListenerEndpointRegistry;
29 | import org.springframework.kafka.listener.MessageListenerContainer;
30 | import org.springframework.web.bind.annotation.GetMapping;
31 | import org.springframework.web.bind.annotation.PathVariable;
32 | import org.springframework.web.bind.annotation.RestController;
33 |
34 | @RestController
35 | @Slf4j
36 | public class DataGenerationController {
37 |
38 | private TransactionsGenerator transactionsGenerator;
39 | private KafkaListenerEndpointRegistry kafkaListenerEndpointRegistry;
40 |
41 | private ExecutorService executor = Executors.newSingleThreadExecutor();
42 | private boolean generatingTransactions = false;
43 | private boolean listenerContainerRunning = true;
44 |
45 | @Value("${kafka.listeners.transactions.id}")
46 | private String transactionListenerId;
47 |
48 | @Value("${transactionsRateDisplayLimit}")
49 | private int transactionsRateDisplayLimit;
50 |
51 | @Autowired
52 | public DataGenerationController(
53 | KafkaTransactionsPusher transactionsPusher,
54 | KafkaListenerEndpointRegistry kafkaListenerEndpointRegistry) {
55 | transactionsGenerator = new DemoTransactionsGenerator(transactionsPusher, 1);
56 | this.kafkaListenerEndpointRegistry = kafkaListenerEndpointRegistry;
57 | }
58 |
59 | @GetMapping("/api/startTransactionsGeneration")
60 | public void startTransactionsGeneration() throws Exception {
61 | log.info("{}", "startTransactionsGeneration called");
62 | generateTransactions();
63 | }
64 |
65 | private void generateTransactions() {
66 | if (!generatingTransactions) {
67 | executor.submit(transactionsGenerator);
68 | generatingTransactions = true;
69 | }
70 | }
71 |
72 | @GetMapping("/api/stopTransactionsGeneration")
73 | public void stopTransactionsGeneration() {
74 | transactionsGenerator.cancel();
75 | generatingTransactions = false;
76 | log.info("{}", "stopTransactionsGeneration called");
77 | }
78 |
79 | @GetMapping("/api/generatorSpeed/{speed}")
80 | public void setGeneratorSpeed(@PathVariable Long speed) {
81 | log.info("Generator speed change request: " + speed);
82 | if (speed <= 0) {
83 | transactionsGenerator.cancel();
84 | generatingTransactions = false;
85 | return;
86 | } else {
87 | generateTransactions();
88 | }
89 |
90 | MessageListenerContainer listenerContainer =
91 | kafkaListenerEndpointRegistry.getListenerContainer(transactionListenerId);
92 | if (speed > transactionsRateDisplayLimit) {
93 | listenerContainer.stop();
94 | listenerContainerRunning = false;
95 | } else if (!listenerContainerRunning) {
96 | listenerContainer.start();
97 | }
98 |
99 | if (transactionsGenerator != null) {
100 | transactionsGenerator.adjustMaxRecordsPerSecond(speed);
101 | }
102 | }
103 | }
104 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/controllers/FlinkController.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.controllers;
19 |
20 | import com.fasterxml.jackson.core.JsonProcessingException;
21 | import com.fasterxml.jackson.databind.ObjectMapper;
22 | import com.ververica.demo.backend.entities.Rule;
23 | import com.ververica.demo.backend.model.RulePayload;
24 | import com.ververica.demo.backend.model.RulePayload.ControlType;
25 | import com.ververica.demo.backend.model.RulePayload.RuleState;
26 | import com.ververica.demo.backend.services.FlinkRulesService;
27 | import org.springframework.web.bind.annotation.GetMapping;
28 | import org.springframework.web.bind.annotation.RequestMapping;
29 | import org.springframework.web.bind.annotation.RestController;
30 |
31 | @RestController
32 | @RequestMapping("/api")
33 | public class FlinkController {
34 |
35 | private final FlinkRulesService flinkRulesService;
36 |
37 | // Currently rules channel is also (mis)used for control messages. This has to do with how control
38 | // channels are set up in Flink Job.
39 | FlinkController(FlinkRulesService flinkRulesService) {
40 | this.flinkRulesService = flinkRulesService;
41 | }
42 |
43 | private final ObjectMapper mapper = new ObjectMapper();
44 |
45 | @GetMapping("/syncRules")
46 | void syncRules() throws JsonProcessingException {
47 | Rule command = createControllCommand(ControlType.EXPORT_RULES_CURRENT);
48 | flinkRulesService.addRule(command);
49 | }
50 |
51 | @GetMapping("/clearState")
52 | void clearState() throws JsonProcessingException {
53 | Rule command = createControllCommand(ControlType.CLEAR_STATE_ALL);
54 | flinkRulesService.addRule(command);
55 | }
56 |
57 | private Rule createControllCommand(ControlType clearStateAll) throws JsonProcessingException {
58 | RulePayload payload = new RulePayload();
59 | payload.setRuleState(RuleState.CONTROL);
60 | payload.setControlType(clearStateAll);
61 | Rule rule = new Rule();
62 | rule.setRulePayload(mapper.writeValueAsString(payload));
63 | return rule;
64 | }
65 | }
66 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/controllers/RuleRestController.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.controllers;
19 |
20 | import com.fasterxml.jackson.core.JsonProcessingException;
21 | import com.fasterxml.jackson.databind.ObjectMapper;
22 | import com.ververica.demo.backend.entities.Rule;
23 | import com.ververica.demo.backend.exceptions.RuleNotFoundException;
24 | import com.ververica.demo.backend.model.RulePayload;
25 | import com.ververica.demo.backend.repositories.RuleRepository;
26 | import com.ververica.demo.backend.services.FlinkRulesService;
27 | import java.io.IOException;
28 | import java.util.List;
29 | import org.springframework.web.bind.annotation.*;
30 |
31 | @RestController
32 | @RequestMapping("/api")
33 | class RuleRestController {
34 |
35 | private final RuleRepository repository;
36 | private final FlinkRulesService flinkRulesService;
37 |
38 | RuleRestController(RuleRepository repository, FlinkRulesService flinkRulesService) {
39 | this.repository = repository;
40 | this.flinkRulesService = flinkRulesService;
41 | }
42 |
43 | private final ObjectMapper mapper = new ObjectMapper();
44 |
45 | @GetMapping("/rules")
46 | List all() {
47 | return repository.findAll();
48 | }
49 |
50 | @PostMapping("/rules")
51 | Rule newRule(@RequestBody Rule newRule) throws IOException {
52 | Rule savedRule = repository.save(newRule);
53 | Integer id = savedRule.getId();
54 | RulePayload payload = mapper.readValue(savedRule.getRulePayload(), RulePayload.class);
55 | payload.setRuleId(id);
56 | String payloadJson = mapper.writeValueAsString(payload);
57 | savedRule.setRulePayload(payloadJson);
58 | Rule result = repository.save(savedRule);
59 | flinkRulesService.addRule(result);
60 | return result;
61 | }
62 |
63 | @GetMapping("/rules/pushToFlink")
64 | void pushToFlink() {
65 | List rules = repository.findAll();
66 | for (Rule rule : rules) {
67 | flinkRulesService.addRule(rule);
68 | }
69 | }
70 |
71 | @GetMapping("/rules/{id}")
72 | Rule one(@PathVariable Integer id) {
73 | return repository.findById(id).orElseThrow(() -> new RuleNotFoundException(id));
74 | }
75 |
76 | @DeleteMapping("/rules/{id}")
77 | void deleteRule(@PathVariable Integer id) throws JsonProcessingException {
78 | repository.deleteById(id);
79 | flinkRulesService.deleteRule(id);
80 | }
81 |
82 | @DeleteMapping("/rules")
83 | void deleteAllRules() throws JsonProcessingException {
84 | List rules = repository.findAll();
85 | for (Rule rule : rules) {
86 | repository.deleteById(rule.getId());
87 | flinkRulesService.deleteRule(rule.getId());
88 | }
89 | }
90 | }
91 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/datasource/DemoTransactionsGenerator.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.datasource;
19 |
20 | import java.math.BigDecimal;
21 | import java.util.SplittableRandom;
22 | import java.util.function.Consumer;
23 | import lombok.extern.slf4j.Slf4j;
24 |
25 | @Slf4j
26 | public class DemoTransactionsGenerator extends TransactionsGenerator {
27 |
28 | private long lastPayeeIdBeneficiaryIdTriggered = System.currentTimeMillis();
29 | private long lastBeneficiaryIdTriggered = System.currentTimeMillis();
30 | private BigDecimal beneficiaryLimit = new BigDecimal(10000000);
31 | private BigDecimal payeeBeneficiaryLimit = new BigDecimal(20000000);
32 |
33 | public DemoTransactionsGenerator(Consumer consumer, int maxRecordsPerSecond) {
34 | super(consumer, maxRecordsPerSecond);
35 | }
36 |
37 | protected Transaction randomEvent(SplittableRandom rnd) {
38 | Transaction transaction = super.randomEvent(rnd);
39 | long now = System.currentTimeMillis();
40 | if (now - lastBeneficiaryIdTriggered > 8000 + rnd.nextInt(5000)) {
41 | transaction.setPaymentAmount(beneficiaryLimit.add(new BigDecimal(rnd.nextInt(1000000))));
42 | this.lastBeneficiaryIdTriggered = System.currentTimeMillis();
43 | }
44 | if (now - lastPayeeIdBeneficiaryIdTriggered > 12000 + rnd.nextInt(10000)) {
45 | transaction.setPaymentAmount(payeeBeneficiaryLimit.add(new BigDecimal(rnd.nextInt(1000000))));
46 | this.lastPayeeIdBeneficiaryIdTriggered = System.currentTimeMillis();
47 | }
48 | return transaction;
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/datasource/RulesBootstrapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.datasource;
19 |
20 | import com.ververica.demo.backend.entities.Rule;
21 | import com.ververica.demo.backend.repositories.RuleRepository;
22 | import com.ververica.demo.backend.services.FlinkRulesService;
23 | import java.util.List;
24 | import org.springframework.beans.factory.annotation.Autowired;
25 | import org.springframework.boot.ApplicationArguments;
26 | import org.springframework.boot.ApplicationRunner;
27 | import org.springframework.stereotype.Component;
28 |
29 | @Component
30 | public class RulesBootstrapper implements ApplicationRunner {
31 |
32 | private RuleRepository ruleRepository;
33 | private FlinkRulesService flinkRulesService;
34 |
35 | @Autowired
36 | public RulesBootstrapper(RuleRepository userRepository, FlinkRulesService flinkRulesService) {
37 | this.ruleRepository = userRepository;
38 | this.flinkRulesService = flinkRulesService;
39 | }
40 |
41 | public void run(ApplicationArguments args) {
42 | String payload1 =
43 | "{\"ruleId\":\"1\","
44 | + "\"aggregateFieldName\":\"paymentAmount\","
45 | + "\"aggregatorFunctionType\":\"SUM\","
46 | + "\"groupingKeyNames\":[\"payeeId\", \"beneficiaryId\"],"
47 | + "\"limit\":\"20000000\","
48 | + "\"limitOperatorType\":\"GREATER\","
49 | + "\"ruleState\":\"ACTIVE\","
50 | + "\"windowMinutes\":\"43200\"}";
51 |
52 | Rule rule1 = new Rule(payload1);
53 |
54 | String payload2 =
55 | "{\"ruleId\":\"2\","
56 | + "\"aggregateFieldName\":\"COUNT_FLINK\","
57 | + "\"aggregatorFunctionType\":\"SUM\","
58 | + "\"groupingKeyNames\":[\"paymentType\"],"
59 | + "\"limit\":\"300\","
60 | + "\"limitOperatorType\":\"LESS\","
61 | + "\"ruleState\":\"PAUSE\","
62 | + "\"windowMinutes\":\"1440\"}";
63 |
64 | Rule rule2 = new Rule(payload2);
65 |
66 | String payload3 =
67 | "{\"ruleId\":\"3\","
68 | + "\"aggregateFieldName\":\"paymentAmount\","
69 | + "\"aggregatorFunctionType\":\"SUM\","
70 | + "\"groupingKeyNames\":[\"beneficiaryId\"],"
71 | + "\"limit\":\"10000000\","
72 | + "\"limitOperatorType\":\"GREATER_EQUAL\","
73 | + "\"ruleState\":\"ACTIVE\","
74 | + "\"windowMinutes\":\"1440\"}";
75 |
76 | Rule rule3 = new Rule(payload3);
77 |
78 | String payload4 =
79 | "{\"ruleId\":\"4\","
80 | + "\"aggregateFieldName\":\"COUNT_WITH_RESET_FLINK\","
81 | + "\"aggregatorFunctionType\":\"SUM\","
82 | + "\"groupingKeyNames\":[\"paymentType\"],"
83 | + "\"limit\":\"100\","
84 | + "\"limitOperatorType\":\"GREATER_EQUAL\","
85 | + "\"ruleState\":\"ACTIVE\","
86 | + "\"windowMinutes\":\"1440\"}";
87 |
88 | Rule rule4 = new Rule(payload4);
89 |
90 | ruleRepository.save(rule1);
91 | ruleRepository.save(rule2);
92 | ruleRepository.save(rule3);
93 | ruleRepository.save(rule4);
94 |
95 | List rules = ruleRepository.findAll();
96 | rules.forEach(rule -> flinkRulesService.addRule(rule));
97 | }
98 | }
99 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/datasource/Throttler.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.datasource;
19 |
20 | import com.google.common.base.Preconditions;
21 |
22 | /** Utility to throttle a thread to a given number of executions (records) per second. */
23 | final class Throttler {
24 |
25 | private long throttleBatchSize;
26 | private long nanosPerBatch;
27 |
28 | private long endOfNextBatchNanos;
29 | private int currentBatch;
30 |
31 | Throttler(long maxRecordsPerSecond) {
32 | setup(maxRecordsPerSecond);
33 | }
34 |
35 | public void adjustMaxRecordsPerSecond(long maxRecordsPerSecond) {
36 | setup(maxRecordsPerSecond);
37 | }
38 |
39 | private synchronized void setup(long maxRecordsPerSecond) {
40 | Preconditions.checkArgument(
41 | maxRecordsPerSecond == -1 || maxRecordsPerSecond > 0,
42 | "maxRecordsPerSecond must be positive or -1 (infinite)");
43 |
44 | if (maxRecordsPerSecond == -1) {
45 | // unlimited speed
46 | throttleBatchSize = -1;
47 | nanosPerBatch = 0;
48 | endOfNextBatchNanos = System.nanoTime() + nanosPerBatch;
49 | currentBatch = 0;
50 | return;
51 | }
52 |
53 | if (maxRecordsPerSecond >= 10000) {
54 | // high rates: all throttling in intervals of 2ms
55 | throttleBatchSize = (int) maxRecordsPerSecond / 500;
56 | nanosPerBatch = 2_000_000L;
57 | } else {
58 | throttleBatchSize = ((int) (maxRecordsPerSecond / 20)) + 1;
59 | nanosPerBatch = ((int) (1_000_000_000L / maxRecordsPerSecond)) * throttleBatchSize;
60 | }
61 | this.endOfNextBatchNanos = System.nanoTime() + nanosPerBatch;
62 | this.currentBatch = 0;
63 | }
64 |
65 | synchronized void throttle() throws InterruptedException {
66 | if (throttleBatchSize == -1) {
67 | return;
68 | }
69 | if (++currentBatch != throttleBatchSize) {
70 | return;
71 | }
72 | currentBatch = 0;
73 |
74 | final long now = System.nanoTime();
75 | final int millisRemaining = (int) ((endOfNextBatchNanos - now) / 1_000_000);
76 |
77 | if (millisRemaining > 0) {
78 | endOfNextBatchNanos += nanosPerBatch;
79 | Thread.sleep(millisRemaining);
80 | } else {
81 | endOfNextBatchNanos = now + nanosPerBatch;
82 | }
83 | }
84 | }
85 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/datasource/Transaction.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.datasource;
19 |
20 | import java.math.BigDecimal;
21 | import java.time.ZoneOffset;
22 | import java.time.ZonedDateTime;
23 | import java.time.format.DateTimeFormatter;
24 | import java.util.Arrays;
25 | import java.util.Iterator;
26 | import java.util.List;
27 | import java.util.Locale;
28 | import lombok.AllArgsConstructor;
29 | import lombok.Builder;
30 | import lombok.Data;
31 | import lombok.NoArgsConstructor;
32 |
33 | @Data
34 | @Builder
35 | @NoArgsConstructor
36 | @AllArgsConstructor
37 | public class Transaction {
38 | public long transactionId;
39 | public long eventTime;
40 | public long payeeId;
41 | public long beneficiaryId;
42 | public BigDecimal paymentAmount;
43 | public PaymentType paymentType;
44 |
45 | private static transient DateTimeFormatter timeFormatter =
46 | DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss")
47 | .withLocale(Locale.US)
48 | .withZone(ZoneOffset.UTC);
49 |
50 | public enum PaymentType {
51 | CSH("CSH"),
52 | CRD("CRD");
53 |
54 | String representation;
55 |
56 | PaymentType(String repr) {
57 | this.representation = repr;
58 | }
59 |
60 | public static PaymentType fromString(String representation) {
61 | for (PaymentType b : PaymentType.values()) {
62 | if (b.representation.equals(representation)) {
63 | return b;
64 | }
65 | }
66 | return null;
67 | }
68 | }
69 |
70 | public static Transaction fromString(String line) {
71 | List tokens = Arrays.asList(line.split(","));
72 | int numArgs = 6;
73 | if (tokens.size() != numArgs) {
74 | throw new RuntimeException(
75 | "Invalid transaction: "
76 | + line
77 | + ". Required number of arguments: "
78 | + numArgs
79 | + " found "
80 | + tokens.size());
81 | }
82 |
83 | Transaction transaction = new Transaction();
84 |
85 | try {
86 | Iterator iter = tokens.iterator();
87 | transaction.transactionId = Long.parseLong(iter.next());
88 | transaction.eventTime =
89 | ZonedDateTime.parse(iter.next(), timeFormatter).toInstant().toEpochMilli();
90 | transaction.payeeId = Long.parseLong(iter.next());
91 | transaction.beneficiaryId = Long.parseLong(iter.next());
92 | transaction.paymentType = PaymentType.fromString(iter.next());
93 | transaction.paymentAmount = new BigDecimal(iter.next());
94 | } catch (NumberFormatException nfe) {
95 | throw new RuntimeException("Invalid record: " + line, nfe);
96 | }
97 |
98 | return transaction;
99 | }
100 | }
101 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/datasource/TransactionsGenerator.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.datasource;
19 |
20 | import com.ververica.demo.backend.datasource.Transaction.PaymentType;
21 | import java.math.BigDecimal;
22 | import java.util.SplittableRandom;
23 | import java.util.concurrent.ThreadLocalRandom;
24 | import java.util.function.Consumer;
25 | import lombok.extern.slf4j.Slf4j;
26 |
27 | @Slf4j
28 | public class TransactionsGenerator implements Runnable {
29 |
30 | private static long MAX_PAYEE_ID = 100000;
31 | private static long MAX_BENEFICIARY_ID = 100000;
32 |
33 | private static double MIN_PAYMENT_AMOUNT = 5d;
34 | private static double MAX_PAYMENT_AMOUNT = 20d;
35 | private final Throttler throttler;
36 |
37 | private volatile boolean running = true;
38 | private Integer maxRecordsPerSecond;
39 |
40 | private Consumer consumer;
41 |
42 | public TransactionsGenerator(Consumer consumer, int maxRecordsPerSecond) {
43 | this.consumer = consumer;
44 | this.maxRecordsPerSecond = maxRecordsPerSecond;
45 | this.throttler = new Throttler(maxRecordsPerSecond);
46 | }
47 |
48 | public void adjustMaxRecordsPerSecond(long maxRecordsPerSecond) {
49 | throttler.adjustMaxRecordsPerSecond(maxRecordsPerSecond);
50 | }
51 |
52 | protected Transaction randomEvent(SplittableRandom rnd) {
53 | long transactionId = rnd.nextLong(Long.MAX_VALUE);
54 | long payeeId = rnd.nextLong(MAX_PAYEE_ID);
55 | long beneficiaryId = rnd.nextLong(MAX_BENEFICIARY_ID);
56 | double paymentAmountDouble =
57 | ThreadLocalRandom.current().nextDouble(MIN_PAYMENT_AMOUNT, MAX_PAYMENT_AMOUNT);
58 | paymentAmountDouble = Math.floor(paymentAmountDouble * 100) / 100;
59 | BigDecimal paymentAmount = BigDecimal.valueOf(paymentAmountDouble);
60 |
61 | return Transaction.builder()
62 | .transactionId(transactionId)
63 | .payeeId(payeeId)
64 | .beneficiaryId(beneficiaryId)
65 | .paymentAmount(paymentAmount)
66 | .paymentType(paymentType(transactionId))
67 | .eventTime(System.currentTimeMillis())
68 | .build();
69 | }
70 |
71 | public Transaction generateOne() {
72 | return randomEvent(new SplittableRandom());
73 | }
74 |
75 | private static PaymentType paymentType(long id) {
76 | int name = (int) (id % 2);
77 | switch (name) {
78 | case 0:
79 | return PaymentType.CRD;
80 | case 1:
81 | return PaymentType.CSH;
82 | default:
83 | throw new IllegalStateException("");
84 | }
85 | }
86 |
87 | @Override
88 | public final void run() {
89 | running = true;
90 |
91 | final SplittableRandom rnd = new SplittableRandom();
92 |
93 | while (running) {
94 | Transaction event = randomEvent(rnd);
95 | log.debug("{}", event);
96 | consumer.accept(event);
97 | try {
98 | throttler.throttle();
99 | } catch (InterruptedException e) {
100 | throw new RuntimeException(e);
101 | }
102 | }
103 | log.info("Finished run()");
104 | }
105 |
106 | public final void cancel() {
107 | running = false;
108 | log.info("Cancelled");
109 | }
110 | }
111 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/entities/Rule.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.entities;
19 |
20 | import javax.persistence.Entity;
21 | import javax.persistence.GeneratedValue;
22 | import javax.persistence.GenerationType;
23 | import javax.persistence.Id;
24 | import lombok.AllArgsConstructor;
25 | import lombok.Data;
26 | import lombok.NoArgsConstructor;
27 |
28 | @Entity
29 | @Data
30 | @AllArgsConstructor
31 | @NoArgsConstructor
32 | public class Rule {
33 |
34 | public Rule(String rulePayload) {
35 | this.rulePayload = rulePayload;
36 | }
37 |
38 | @Id
39 | @GeneratedValue(strategy = GenerationType.IDENTITY)
40 | private Integer id;
41 |
42 | private String rulePayload;
43 | }
44 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/exceptions/RuleNotFoundException.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.exceptions;
19 |
20 | public class RuleNotFoundException extends RuntimeException {
21 |
22 | public RuleNotFoundException(Integer id) {
23 | super("Could not find employee " + id);
24 | }
25 | }
26 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/model/Alert.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.model;
19 |
20 | import com.ververica.demo.backend.datasource.Transaction;
21 | import java.math.BigDecimal;
22 | import lombok.AllArgsConstructor;
23 | import lombok.Data;
24 |
25 | @Data
26 | @AllArgsConstructor
27 | public class Alert {
28 | private Integer ruleId;
29 | private String rulePayload;
30 |
31 | Transaction triggeringEvent;
32 | BigDecimal triggeringValue;
33 | }
34 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/model/RulePayload.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.model;
19 |
20 | import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
21 | import java.math.BigDecimal;
22 | import java.util.List;
23 | import lombok.Data;
24 | import lombok.EqualsAndHashCode;
25 | import lombok.ToString;
26 |
27 | /** Rules representation. */
28 | @EqualsAndHashCode
29 | @ToString
30 | @Data
31 | @JsonIgnoreProperties(ignoreUnknown = true)
32 | public class RulePayload {
33 |
34 | private Integer ruleId;
35 | private RuleState ruleState;
36 | private List groupingKeyNames; // aggregation
37 | private List unique;
38 | private String aggregateFieldName;
39 | private AggregatorFunctionType aggregatorFunctionType;
40 | private LimitOperatorType limitOperatorType;
41 | private BigDecimal limit;
42 | private Integer windowMinutes;
43 | private ControlType controlType;
44 |
45 | /**
46 | * Evaluates this rule by comparing provided value with rules' limit based on limit operator type.
47 | *
48 | * @param comparisonValue value to be compared with the limit
49 | */
50 | public boolean apply(BigDecimal comparisonValue) {
51 | switch (limitOperatorType) {
52 | case EQUAL:
53 | return comparisonValue.compareTo(limit) == 0;
54 | case NOT_EQUAL:
55 | return comparisonValue.compareTo(limit) != 0;
56 | case GREATER:
57 | return comparisonValue.compareTo(limit) > 0;
58 | case LESS:
59 | return comparisonValue.compareTo(limit) < 0;
60 | case LESS_EQUAL:
61 | return comparisonValue.compareTo(limit) <= 0;
62 | case GREATER_EQUAL:
63 | return comparisonValue.compareTo(limit) >= 0;
64 | default:
65 | throw new RuntimeException("Unknown limit operator type: " + limitOperatorType);
66 | }
67 | }
68 |
69 | public enum AggregatorFunctionType {
70 | SUM,
71 | AVG,
72 | MIN,
73 | MAX
74 | }
75 |
76 | public enum LimitOperatorType {
77 | EQUAL("="),
78 | NOT_EQUAL("!="),
79 | GREATER_EQUAL(">="),
80 | LESS_EQUAL("<="),
81 | GREATER(">"),
82 | LESS("<");
83 |
84 | String operator;
85 |
86 | LimitOperatorType(String operator) {
87 | this.operator = operator;
88 | }
89 |
90 | public static LimitOperatorType fromString(String text) {
91 | for (LimitOperatorType b : LimitOperatorType.values()) {
92 | if (b.operator.equals(text)) {
93 | return b;
94 | }
95 | }
96 | return null;
97 | }
98 | }
99 |
100 | public enum RuleState {
101 | ACTIVE,
102 | PAUSE,
103 | DELETE,
104 | CONTROL
105 | }
106 |
107 | public enum ControlType {
108 | CLEAR_STATE_ALL,
109 | CLEAR_STATE_ALL_STOP,
110 | DELETE_RULES_ALL,
111 | EXPORT_RULES_CURRENT
112 | }
113 | }
114 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/repositories/RuleRepository.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.repositories;
19 |
20 | import com.ververica.demo.backend.entities.Rule;
21 | import java.util.List;
22 | import org.springframework.data.repository.CrudRepository;
23 |
24 | public interface RuleRepository extends CrudRepository {
25 |
26 | @Override
27 | List findAll();
28 | }
29 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/services/FlinkRulesService.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.services;
19 |
20 | import com.fasterxml.jackson.core.JsonProcessingException;
21 | import com.fasterxml.jackson.databind.ObjectMapper;
22 | import com.ververica.demo.backend.entities.Rule;
23 | import com.ververica.demo.backend.model.RulePayload;
24 | import com.ververica.demo.backend.model.RulePayload.RuleState;
25 | import org.springframework.beans.factory.annotation.Autowired;
26 | import org.springframework.beans.factory.annotation.Value;
27 | import org.springframework.kafka.core.KafkaTemplate;
28 | import org.springframework.stereotype.Service;
29 |
30 | @Service
31 | public class FlinkRulesService {
32 |
33 | private KafkaTemplate kafkaTemplate;
34 |
35 | @Value("${kafka.topic.rules}")
36 | private String topic;
37 |
38 | private final ObjectMapper mapper = new ObjectMapper();
39 |
40 | @Autowired
41 | public FlinkRulesService(KafkaTemplate kafkaTemplate) {
42 | this.kafkaTemplate = kafkaTemplate;
43 | }
44 |
45 | public void addRule(Rule rule) {
46 | String payload = rule.getRulePayload();
47 | kafkaTemplate.send(topic, payload);
48 | }
49 |
50 | public void deleteRule(int ruleId) throws JsonProcessingException {
51 | RulePayload payload = new RulePayload();
52 | payload.setRuleId(ruleId);
53 | payload.setRuleState(RuleState.DELETE);
54 | String payloadJson = mapper.writeValueAsString(payload);
55 | kafkaTemplate.send(topic, payloadJson);
56 | }
57 | }
58 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/services/KafkaAlertsPusher.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.services;
19 |
20 | import com.ververica.demo.backend.model.Alert;
21 | import java.util.function.Consumer;
22 | import lombok.extern.slf4j.Slf4j;
23 | import org.springframework.beans.factory.annotation.Autowired;
24 | import org.springframework.beans.factory.annotation.Value;
25 | import org.springframework.kafka.core.KafkaTemplate;
26 | import org.springframework.stereotype.Service;
27 |
28 | @Service
29 | @Slf4j
30 | public class KafkaAlertsPusher implements Consumer {
31 |
32 | private KafkaTemplate kafkaTemplate;
33 |
34 | @Value("${kafka.topic.alerts}")
35 | private String topic;
36 |
37 | @Autowired
38 | public KafkaAlertsPusher(KafkaTemplate kafkaTemplateForJson) {
39 | this.kafkaTemplate = kafkaTemplateForJson;
40 | }
41 |
42 | @Override
43 | public void accept(Alert alert) {
44 | log.info("{}", alert);
45 | kafkaTemplate.send(topic, alert);
46 | }
47 | }
48 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/services/KafkaConsumerService.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.services;
19 |
20 | import com.fasterxml.jackson.databind.ObjectMapper;
21 | import com.ververica.demo.backend.entities.Rule;
22 | import com.ververica.demo.backend.model.RulePayload;
23 | import com.ververica.demo.backend.repositories.RuleRepository;
24 | import java.io.IOException;
25 | import java.util.Optional;
26 | import lombok.extern.slf4j.Slf4j;
27 | import org.springframework.beans.factory.annotation.Autowired;
28 | import org.springframework.beans.factory.annotation.Value;
29 | import org.springframework.kafka.annotation.KafkaListener;
30 | import org.springframework.messaging.handler.annotation.Payload;
31 | import org.springframework.messaging.simp.SimpMessagingTemplate;
32 | import org.springframework.stereotype.Service;
33 |
34 | @Service
35 | @Slf4j
36 | public class KafkaConsumerService {
37 |
38 | private final SimpMessagingTemplate simpTemplate;
39 | private final RuleRepository ruleRepository;
40 | private final ObjectMapper mapper = new ObjectMapper();
41 |
42 | @Value("${web-socket.topic.alerts}")
43 | private String alertsWebSocketTopic;
44 |
45 | @Value("${web-socket.topic.latency}")
46 | private String latencyWebSocketTopic;
47 |
48 | @Autowired
49 | public KafkaConsumerService(SimpMessagingTemplate simpTemplate, RuleRepository ruleRepository) {
50 | this.simpTemplate = simpTemplate;
51 | this.ruleRepository = ruleRepository;
52 | }
53 |
54 | @KafkaListener(topics = "${kafka.topic.alerts}", groupId = "alerts")
55 | public void templateAlerts(@Payload String message) {
56 | log.debug("{}", message);
57 | simpTemplate.convertAndSend(alertsWebSocketTopic, message);
58 | }
59 |
60 | @KafkaListener(topics = "${kafka.topic.latency}", groupId = "latency")
61 | public void templateLatency(@Payload String message) {
62 | log.debug("{}", message);
63 | simpTemplate.convertAndSend(latencyWebSocketTopic, message);
64 | }
65 |
66 | @KafkaListener(topics = "${kafka.topic.current-rules}", groupId = "current-rules")
67 | public void templateCurrentFlinkRules(@Payload String message) throws IOException {
68 | log.info("{}", message);
69 | RulePayload payload = mapper.readValue(message, RulePayload.class);
70 | Integer payloadId = payload.getRuleId();
71 | Optional existingRule = ruleRepository.findById(payloadId);
72 | if (!existingRule.isPresent()) {
73 | ruleRepository.save(new Rule(payloadId, mapper.writeValueAsString(payload)));
74 | }
75 | }
76 | }
77 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/services/KafkaTransactionsConsumerService.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.services;
19 |
20 | import com.fasterxml.jackson.databind.ObjectMapper;
21 | import java.util.Map;
22 | import lombok.extern.slf4j.Slf4j;
23 | import org.apache.kafka.common.TopicPartition;
24 | import org.springframework.beans.factory.annotation.Autowired;
25 | import org.springframework.beans.factory.annotation.Value;
26 | import org.springframework.kafka.annotation.KafkaListener;
27 | import org.springframework.kafka.listener.ConsumerSeekAware;
28 | import org.springframework.messaging.handler.annotation.Payload;
29 | import org.springframework.messaging.simp.SimpMessagingTemplate;
30 | import org.springframework.stereotype.Service;
31 |
32 | @Service
33 | @Slf4j
34 | public class KafkaTransactionsConsumerService implements ConsumerSeekAware {
35 |
36 | private final SimpMessagingTemplate simpTemplate;
37 | private final ObjectMapper mapper = new ObjectMapper();
38 |
39 | @Value("${web-socket.topic.transactions}")
40 | private String transactionsWebSocketTopic;
41 |
42 | @Autowired
43 | public KafkaTransactionsConsumerService(SimpMessagingTemplate simpTemplate) {
44 | this.simpTemplate = simpTemplate;
45 | }
46 |
47 | @KafkaListener(
48 | id = "${kafka.listeners.transactions.id}",
49 | topics = "${kafka.topic.transactions}",
50 | groupId = "transactions")
51 | public void consumeTransactions(@Payload String message) {
52 | log.debug("{}", message);
53 | simpTemplate.convertAndSend(transactionsWebSocketTopic, message);
54 | }
55 |
56 | @Override
57 | public void registerSeekCallback(ConsumerSeekCallback callback) {}
58 |
59 | @Override
60 | public void onPartitionsAssigned(
61 | Map assignments, ConsumerSeekCallback callback) {
62 | assignments.forEach((t, o) -> callback.seekToEnd(t.topic(), t.partition()));
63 | }
64 |
65 | @Override
66 | public void onIdleContainer(
67 | Map assignments, ConsumerSeekCallback callback) {}
68 | }
69 |
--------------------------------------------------------------------------------
/webapp/src/main/java/com/ververica/demo/backend/services/KafkaTransactionsPusher.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend.services;
19 |
20 | import com.ververica.demo.backend.datasource.Transaction;
21 | import java.util.function.Consumer;
22 | import lombok.extern.slf4j.Slf4j;
23 | import org.springframework.beans.factory.annotation.Autowired;
24 | import org.springframework.beans.factory.annotation.Value;
25 | import org.springframework.kafka.core.KafkaTemplate;
26 | import org.springframework.stereotype.Service;
27 |
28 | @Service
29 | @Slf4j
30 | public class KafkaTransactionsPusher implements Consumer {
31 |
32 | private KafkaTemplate kafkaTemplate;
33 | private Transaction lastTransaction;
34 |
35 | @Value("${kafka.topic.transactions}")
36 | private String topic;
37 |
38 | @Autowired
39 | public KafkaTransactionsPusher(KafkaTemplate kafkaTemplateForJson) {
40 | this.kafkaTemplate = kafkaTemplateForJson;
41 | }
42 |
43 | @Override
44 | public void accept(Transaction transaction) {
45 | lastTransaction = transaction;
46 | log.debug("{}", transaction);
47 | kafkaTemplate.send(topic, transaction);
48 | }
49 |
50 | public Transaction getLastTransaction() {
51 | return lastTransaction;
52 | }
53 | }
54 |
--------------------------------------------------------------------------------
/webapp/src/main/resources/application-cloud.yaml:
--------------------------------------------------------------------------------
1 | kafka:
2 | bootstrap-servers: cp-cp-kafka.confluent-platform.svc:9092
3 |
4 | security.auth.enabled: true
5 |
6 | spring:
7 | kafka:
8 | producer:
9 | bootstrap-servers: cp-cp-kafka.confluent-platform.svc:9092
10 |
--------------------------------------------------------------------------------
/webapp/src/main/resources/application-dev.yaml:
--------------------------------------------------------------------------------
1 | kafka:
2 | # bootstrap-servers: kafka-cp-kafka-headless:9092
3 | bootstrap-servers: kafka-cp-kafka-headless:9092
4 |
5 | security.auth.enabled: true
6 |
7 | spring:
8 | kafka:
9 | producer:
10 | bootstrap-servers: kafka-cp-kafka-headless:9092
--------------------------------------------------------------------------------
/webapp/src/main/resources/application.yaml:
--------------------------------------------------------------------------------
1 | server:
2 | port: 5656
3 |
4 | security.auth.enabled: false
5 |
6 | transactionsRateDisplayLimit: 50
7 |
8 | kafka:
9 | topic:
10 | transactions: livetransactions
11 | rules: rules
12 | alerts: alerts
13 | latency: latency
14 | current-rules: current-rules
15 | listeners:
16 | transactions.id: transactions-listener
17 |
18 | bootstrap-servers: localhost:9092
19 |
20 | web-socket:
21 | topic:
22 | transactions: /topic/transactions
23 | alerts: /topic/alerts
24 | latency: /topic/latency
25 |
26 | management.endpoints.web.exposure.include: mappings, loggers
27 | spring:
28 | h2.console.enabled: true
29 | thymeleaf.cache: false
30 | autoconfigure.exclude: org.springframework.boot.autoconfigure.security.SecurityAutoConfiguration
31 | kafka:
32 | producer:
33 | bootstrap-servers: localhost:9092
34 | key-deserializer: org.apache.kafka.common.serialization.StringDeserializer
35 | value-deserializer: org.apache.kafka.common.serialization.StringDeserializer
--------------------------------------------------------------------------------
/webapp/src/react-app-env.d.ts:
--------------------------------------------------------------------------------
1 | ///
2 |
3 | declare module "leader-line";
4 | declare module "react-stomp";
5 | declare module "get-form-data";
6 |
--------------------------------------------------------------------------------
/webapp/src/setupProxy.js:
--------------------------------------------------------------------------------
1 | const proxy = require("http-proxy-middleware");
2 |
3 | module.exports = function(app) {
4 | app.use(proxy("/api", { target: "http://localhost:5656" }));
5 | app.use(proxy("/ws", { target: "ws://localhost:5656", ws: true }));
6 | };
7 |
--------------------------------------------------------------------------------
/webapp/src/test/java/com/ververica/demo/backend/MainTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.ververica.demo.backend;
19 |
20 | import org.junit.Ignore;
21 | import org.junit.Test;
22 | import org.junit.runner.RunWith;
23 | import org.springframework.boot.test.context.SpringBootTest;
24 | import org.springframework.test.context.junit4.SpringRunner;
25 |
26 | @Ignore
27 | @RunWith(SpringRunner.class)
28 | @SpringBootTest
29 | public class MainTest {
30 |
31 | @Test
32 | public void contextLoads() {}
33 | }
34 |
--------------------------------------------------------------------------------
/webapp/tsconfig.json:
--------------------------------------------------------------------------------
1 | {
2 | "compilerOptions": {
3 | "target": "es5",
4 | "lib": ["dom", "dom.iterable", "esnext"],
5 | "allowJs": true,
6 | "skipLibCheck": true,
7 | "esModuleInterop": true,
8 | "allowSyntheticDefaultImports": true,
9 | "strict": true,
10 | "forceConsistentCasingInFileNames": true,
11 | "module": "esnext",
12 | "moduleResolution": "node",
13 | "resolveJsonModule": true,
14 | "isolatedModules": true,
15 | "noEmit": true,
16 | "jsx": "preserve",
17 | "baseUrl": "src"
18 | },
19 | "include": ["src"]
20 | }
21 |
--------------------------------------------------------------------------------
/webapp/tslint.json:
--------------------------------------------------------------------------------
1 | {
2 | "extends": ["tslint:latest", "tslint-react", "tslint-config-prettier"],
3 | "rules": {
4 | "no-submodule-imports": false,
5 | "interface-name": false,
6 | "no-implicit-dependencies": false,
7 | "ordered-imports": false
8 | }
9 | }
10 |
--------------------------------------------------------------------------------
/webapp/webapp.Dockerfile:
--------------------------------------------------------------------------------
1 | # --- UI Build
2 | FROM node:10 as ui-build
3 | WORKDIR /home/node/app
4 |
5 | COPY package.json package-lock.json tsconfig.json ./
6 | COPY scripts scripts
7 | RUN npm ci --unsafe-perm
8 | COPY public public
9 | COPY src/app src/app
10 | COPY src/index.tsx src
11 | COPY src/react-app-env.d.ts src
12 | RUN npm run build
13 |
14 | # --- Maven Build
15 | FROM maven:3.6.2-jdk-8-openj9 as maven-build
16 | WORKDIR /home/maven/work
17 |
18 | COPY pom.xml .
19 | RUN mvn -B -e -C -T 1C org.apache.maven.plugins:maven-dependency-plugin:3.1.1:go-offline
20 | COPY . .
21 | COPY --from=ui-build /home/node/app/build /home/maven/work/target/classes/static/
22 | RUN mvn -B -e -o -T 1C verify
23 | RUN mv target/demo-fraud-webapp*.jar target/demo-fraud-webapp.jar
24 |
25 | # --- Main container
26 | FROM openjdk:8-jdk-alpine as main
27 |
28 | COPY --from=maven-build /home/maven/work/target/demo-fraud-webapp.jar .
29 | EXPOSE 5656
30 |
31 | ENTRYPOINT ["java","-Djava.security.egd=file:/dev/./urandom","-Dspring.profiles.active=dev","-jar","demo-fraud-webapp.jar"]
32 |
--------------------------------------------------------------------------------