├── .gitignore
├── LICENSE
├── README.md
├── bravo-test-utils
├── .gitignore
├── build.gradle
└── src
│ ├── main
│ └── java
│ │ └── com
│ │ └── king
│ │ └── bravo
│ │ └── testing
│ │ ├── BravoTestPipeline.java
│ │ ├── CollectingSink.java
│ │ ├── MiniClusterResourceFactory.java
│ │ ├── PipelineAction.java
│ │ └── actions
│ │ ├── CancelJob.java
│ │ ├── NextWatermark.java
│ │ ├── OneTimePipelineAction.java
│ │ ├── Process.java
│ │ ├── Sleep.java
│ │ ├── TestPipelineSource.java
│ │ ├── TriggerFailure.java
│ │ └── TriggerSavepoint.java
│ └── test
│ └── resources
│ └── logback-test.xml
├── bravo
├── .gitignore
├── build.gradle
└── src
│ ├── main
│ └── java
│ │ └── com
│ │ └── king
│ │ └── bravo
│ │ ├── reader
│ │ ├── AbstractListStateReader.java
│ │ ├── AbstractMapStateReader.java
│ │ ├── KeyedStateReader.java
│ │ ├── ListStateFlattenReader.java
│ │ ├── ListStateListReader.java
│ │ ├── MapStateKKVReader.java
│ │ ├── MapStateValueReader.java
│ │ ├── OperatorStateReader.java
│ │ ├── ValueStateKVReader.java
│ │ ├── ValueStateValueReader.java
│ │ └── inputformat
│ │ │ ├── KeyedStateInputSplit.java
│ │ │ ├── RocksDBCheckpointIterator.java
│ │ │ ├── RocksDBKeyedStateInputFormat.java
│ │ │ ├── RocksDBSavepointIterator.java
│ │ │ └── ValidOffsetsIterator.java
│ │ ├── types
│ │ └── KeyedStateRow.java
│ │ ├── utils
│ │ ├── Identifiers.java
│ │ ├── KeyGroupFlags.java
│ │ ├── RocksDBUtils.java
│ │ └── StateMetadataUtils.java
│ │ └── writer
│ │ ├── OperatorStateWriter.java
│ │ └── functions
│ │ ├── KeyGroupAndStateNameKey.java
│ │ ├── OperatorIndexForKeyGroupKey.java
│ │ ├── RocksDBSavepointWriter.java
│ │ ├── RowFilter.java
│ │ └── ValueStateToKeyedStateRow.java
│ └── test
│ ├── java
│ └── com
│ │ └── king
│ │ └── bravo
│ │ ├── BroadcastStateTransformationTest.java
│ │ ├── KeyedStateAddRemoveTest.java
│ │ ├── MapStateReadingTest.java
│ │ ├── RocksDBCheckpointReadingTest.java
│ │ ├── TtlStateTest.java
│ │ ├── ValueStateTransformationTest.java
│ │ ├── ValueStateTypeChangeTest.java
│ │ └── WindowStateReadingTest.java
│ └── resources
│ └── logback-test.xml
├── build.gradle
├── gradle
└── wrapper
│ ├── gradle-wrapper.jar
│ └── gradle-wrapper.properties
├── gradlew
├── gradlew.bat
└── settings.gradle
/.gitignore:
--------------------------------------------------------------------------------
1 | /.gradle/
2 | .classpath
3 | .project
4 | .settings/
5 | bin/
6 | build/
7 | gradle.properties
8 |
--------------------------------------------------------------------------------
/LICENSE:
--------------------------------------------------------------------------------
1 | Apache License
2 | Version 2.0, January 2004
3 | http://www.apache.org/licenses/
4 |
5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
6 |
7 | 1. Definitions.
8 |
9 | "License" shall mean the terms and conditions for use, reproduction,
10 | and distribution as defined by Sections 1 through 9 of this document.
11 |
12 | "Licensor" shall mean the copyright owner or entity authorized by
13 | the copyright owner that is granting the License.
14 |
15 | "Legal Entity" shall mean the union of the acting entity and all
16 | other entities that control, are controlled by, or are under common
17 | control with that entity. For the purposes of this definition,
18 | "control" means (i) the power, direct or indirect, to cause the
19 | direction or management of such entity, whether by contract or
20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the
21 | outstanding shares, or (iii) beneficial ownership of such entity.
22 |
23 | "You" (or "Your") shall mean an individual or Legal Entity
24 | exercising permissions granted by this License.
25 |
26 | "Source" form shall mean the preferred form for making modifications,
27 | including but not limited to software source code, documentation
28 | source, and configuration files.
29 |
30 | "Object" form shall mean any form resulting from mechanical
31 | transformation or translation of a Source form, including but
32 | not limited to compiled object code, generated documentation,
33 | and conversions to other media types.
34 |
35 | "Work" shall mean the work of authorship, whether in Source or
36 | Object form, made available under the License, as indicated by a
37 | copyright notice that is included in or attached to the work
38 | (an example is provided in the Appendix below).
39 |
40 | "Derivative Works" shall mean any work, whether in Source or Object
41 | form, that is based on (or derived from) the Work and for which the
42 | editorial revisions, annotations, elaborations, or other modifications
43 | represent, as a whole, an original work of authorship. For the purposes
44 | of this License, Derivative Works shall not include works that remain
45 | separable from, or merely link (or bind by name) to the interfaces of,
46 | the Work and Derivative Works thereof.
47 |
48 | "Contribution" shall mean any work of authorship, including
49 | the original version of the Work and any modifications or additions
50 | to that Work or Derivative Works thereof, that is intentionally
51 | submitted to Licensor for inclusion in the Work by the copyright owner
52 | or by an individual or Legal Entity authorized to submit on behalf of
53 | the copyright owner. For the purposes of this definition, "submitted"
54 | means any form of electronic, verbal, or written communication sent
55 | to the Licensor or its representatives, including but not limited to
56 | communication on electronic mailing lists, source code control systems,
57 | and issue tracking systems that are managed by, or on behalf of, the
58 | Licensor for the purpose of discussing and improving the Work, but
59 | excluding communication that is conspicuously marked or otherwise
60 | designated in writing by the copyright owner as "Not a Contribution."
61 |
62 | "Contributor" shall mean Licensor and any individual or Legal Entity
63 | on behalf of whom a Contribution has been received by Licensor and
64 | subsequently incorporated within the Work.
65 |
66 | 2. Grant of Copyright License. Subject to the terms and conditions of
67 | this License, each Contributor hereby grants to You a perpetual,
68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
69 | copyright license to reproduce, prepare Derivative Works of,
70 | publicly display, publicly perform, sublicense, and distribute the
71 | Work and such Derivative Works in Source or Object form.
72 |
73 | 3. Grant of Patent License. Subject to the terms and conditions of
74 | this License, each Contributor hereby grants to You a perpetual,
75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
76 | (except as stated in this section) patent license to make, have made,
77 | use, offer to sell, sell, import, and otherwise transfer the Work,
78 | where such license applies only to those patent claims licensable
79 | by such Contributor that are necessarily infringed by their
80 | Contribution(s) alone or by combination of their Contribution(s)
81 | with the Work to which such Contribution(s) was submitted. If You
82 | institute patent litigation against any entity (including a
83 | cross-claim or counterclaim in a lawsuit) alleging that the Work
84 | or a Contribution incorporated within the Work constitutes direct
85 | or contributory patent infringement, then any patent licenses
86 | granted to You under this License for that Work shall terminate
87 | as of the date such litigation is filed.
88 |
89 | 4. Redistribution. You may reproduce and distribute copies of the
90 | Work or Derivative Works thereof in any medium, with or without
91 | modifications, and in Source or Object form, provided that You
92 | meet the following conditions:
93 |
94 | (a) You must give any other recipients of the Work or
95 | Derivative Works a copy of this License; and
96 |
97 | (b) You must cause any modified files to carry prominent notices
98 | stating that You changed the files; and
99 |
100 | (c) You must retain, in the Source form of any Derivative Works
101 | that You distribute, all copyright, patent, trademark, and
102 | attribution notices from the Source form of the Work,
103 | excluding those notices that do not pertain to any part of
104 | the Derivative Works; and
105 |
106 | (d) If the Work includes a "NOTICE" text file as part of its
107 | distribution, then any Derivative Works that You distribute must
108 | include a readable copy of the attribution notices contained
109 | within such NOTICE file, excluding those notices that do not
110 | pertain to any part of the Derivative Works, in at least one
111 | of the following places: within a NOTICE text file distributed
112 | as part of the Derivative Works; within the Source form or
113 | documentation, if provided along with the Derivative Works; or,
114 | within a display generated by the Derivative Works, if and
115 | wherever such third-party notices normally appear. The contents
116 | of the NOTICE file are for informational purposes only and
117 | do not modify the License. You may add Your own attribution
118 | notices within Derivative Works that You distribute, alongside
119 | or as an addendum to the NOTICE text from the Work, provided
120 | that such additional attribution notices cannot be construed
121 | as modifying the License.
122 |
123 | You may add Your own copyright statement to Your modifications and
124 | may provide additional or different license terms and conditions
125 | for use, reproduction, or distribution of Your modifications, or
126 | for any such Derivative Works as a whole, provided Your use,
127 | reproduction, and distribution of the Work otherwise complies with
128 | the conditions stated in this License.
129 |
130 | 5. Submission of Contributions. Unless You explicitly state otherwise,
131 | any Contribution intentionally submitted for inclusion in the Work
132 | by You to the Licensor shall be under the terms and conditions of
133 | this License, without any additional terms or conditions.
134 | Notwithstanding the above, nothing herein shall supersede or modify
135 | the terms of any separate license agreement you may have executed
136 | with Licensor regarding such Contributions.
137 |
138 | 6. Trademarks. This License does not grant permission to use the trade
139 | names, trademarks, service marks, or product names of the Licensor,
140 | except as required for reasonable and customary use in describing the
141 | origin of the Work and reproducing the content of the NOTICE file.
142 |
143 | 7. Disclaimer of Warranty. Unless required by applicable law or
144 | agreed to in writing, Licensor provides the Work (and each
145 | Contributor provides its Contributions) on an "AS IS" BASIS,
146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
147 | implied, including, without limitation, any warranties or conditions
148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
149 | PARTICULAR PURPOSE. You are solely responsible for determining the
150 | appropriateness of using or redistributing the Work and assume any
151 | risks associated with Your exercise of permissions under this License.
152 |
153 | 8. Limitation of Liability. In no event and under no legal theory,
154 | whether in tort (including negligence), contract, or otherwise,
155 | unless required by applicable law (such as deliberate and grossly
156 | negligent acts) or agreed to in writing, shall any Contributor be
157 | liable to You for damages, including any direct, indirect, special,
158 | incidental, or consequential damages of any character arising as a
159 | result of this License or out of the use or inability to use the
160 | Work (including but not limited to damages for loss of goodwill,
161 | work stoppage, computer failure or malfunction, or any and all
162 | other commercial damages or losses), even if such Contributor
163 | has been advised of the possibility of such damages.
164 |
165 | 9. Accepting Warranty or Additional Liability. While redistributing
166 | the Work or Derivative Works thereof, You may choose to offer,
167 | and charge a fee for, acceptance of support, warranty, indemnity,
168 | or other liability obligations and/or rights consistent with this
169 | License. However, in accepting such obligations, You may act only
170 | on Your own behalf and on Your sole responsibility, not on behalf
171 | of any other Contributor, and only if You agree to indemnify,
172 | defend, and hold each Contributor harmless for any liability
173 | incurred by, or claims asserted against, such Contributor by reason
174 | of your accepting any such warranty or additional liability.
175 |
176 | END OF TERMS AND CONDITIONS
177 |
178 | APPENDIX: How to apply the Apache License to your work.
179 |
180 | To apply the Apache License to your work, attach the following
181 | boilerplate notice, with the fields enclosed by brackets "[]"
182 | replaced with your own identifying information. (Don't include
183 | the brackets!) The text should be enclosed in the appropriate
184 | comment syntax for the file format. We also recommend that a
185 | file or class name and description of purpose be included on the
186 | same "printed page" as the copyright notice for easier
187 | identification within third-party archives.
188 |
189 | Copyright [yyyy] [name of copyright owner]
190 |
191 | Licensed under the Apache License, Version 2.0 (the "License");
192 | you may not use this file except in compliance with the License.
193 | You may obtain a copy of the License at
194 |
195 | http://www.apache.org/licenses/LICENSE-2.0
196 |
197 | Unless required by applicable law or agreed to in writing, software
198 | distributed under the License is distributed on an "AS IS" BASIS,
199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
200 | See the License for the specific language governing permissions and
201 | limitations under the License.
202 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | # Bravo
2 |
3 | **Bravo** [](https://maven-badges.herokuapp.com/maven-central/com.king.bravo/bravo)
4 |
5 | ```
6 |
7 | com.king.bravo
8 | bravo
9 | LATEST
10 |
11 | ```
12 |
13 | **Bravo Test Utils** [](https://maven-badges.herokuapp.com/maven-central/com.king.bravo/bravo-test-utils)
14 |
15 | ```
16 |
17 | com.king.bravo
18 | bravo-test-utils
19 | LATEST
20 | test
21 |
22 | ```
23 |
24 | *Note: The Flink dependencies of Bravo are declared compileOnly (provided in Maven world) so that it wont conflict with your own Flink version. This means that we assume that you already have that dependencies in your project*
25 |
26 | *Note: The latest version of Bravo only runs on Flink 1.7.0, but is able to read savepoints/checkpoints that have been created by older Flink versions. At least 1.6.0 created savepoints have been read successfully.*
27 |
28 | ## Introduction
29 |
30 | Bravo is a convenient state reader and writer library leveraging the Flink’s
31 | batch processing capabilities. It supports processing and writing Flink streaming snapshots.
32 | At the moment it only supports processing RocksDB snapshots but this can be extended in the future for other state backends.
33 |
34 | Our goal is to cover a few basic features:
35 | - Converting keyed states to Flink DataSets for processing and analytics
36 | - Reading/Writing non-keyed operators states
37 | - Bootstrap keyed states from Flink DataSets and create new valid savepoints
38 | - Transform existing savepoints by replacing/changing/creating states
39 |
40 | Some example use-cases:
41 | - Point-in-time state analytics across all operators and keys
42 | - Bootstrap state of a streaming job from external resources such as reading from database/filesystem
43 | - Validate and potentially repair corrupted state of a streaming job
44 | - Change max parallelism of a job
45 |
46 | ## Disclaimer
47 |
48 | This is more of a proof of concept implementation, not necessarily something production ready.
49 |
50 | Who am I to tell you what code to run in prod, I have to agree, but please double check the code you are planning to use :)
51 |
52 | ## Building Bravo
53 |
54 | In order to build Bravo locally you only need to use the gradle wrapper already included in the project
55 |
56 | ```bash
57 | cd bravo
58 |
59 | # Build project and run all tests
60 | ./gradlew clean build
61 |
62 | # Build project and publish to local maven repo
63 | ./gradlew clean install
64 | ```
65 |
66 |
67 | ## Reading states
68 |
69 | ### Reading and processing states
70 |
71 | The `OperatorStateReader` provides DataSet input format that understands RocksDB savepoints and checkpoints and can extract keyed state rows from it. The input format creates input splits by operator subtask of the savepoint at the moment but we can change this to split by keygroups directly.
72 |
73 | The reader can also be used to provide in-memory [access to non-keyed states](#accessing-non-keyed-states).
74 |
75 | For example, this code snippet shows how to read keys & values of a _keyed value state_:
76 |
77 | ```java
78 | // First we start by taking a savepoint/checkpoint of our running job...
79 | // Now it's time to load the metadata
80 | Savepoint savepoint = StateMetadataUtils.loadSavepoint(savepointPath);
81 |
82 | ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
83 |
84 | // We create a KeyedStateReader for accessing the state of the operator with the UID "CountPerKey"
85 | OperatorStateReader reader = new OperatorStateReader(env, savepoint, "CountPerKey");
86 |
87 | // The reader now has access to all keyed states of the "CountPerKey" operator
88 | // We are going to read one specific value state named "Count"
89 | // The DataSet contains the key-value tuples from our state
90 | DataSet> countState = reader.readKeyedStates(
91 | KeyedStateReader.forValueStateKVPairs("Count", new TypeHint>() {}));
92 |
93 | // We can now work with the countState dataset and analyze it however we want :)
94 | ```
95 |
96 | The `KeyedStateReader` class provides a set of methods for creating readers for different types of keyed states.
97 |
98 | Some examples:
99 |
100 | ```
101 | KeyedStateReader.forValueStateKVPairs(...)
102 | KeyedStateReader.forValueStateValues(...)
103 | KeyedStateReader.forMapStateEntries(...)
104 | KeyedStateReader.forListStates(...)
105 | KeyedStateReader.forMapStateValues(...)
106 | KeyedStateReader.forWindowStateValues(...)
107 | ```
108 |
109 | For more complete code examples on the usage of the specific readers please look at some of the test cases, they are actually quite nice:
110 |
111 | https://github.com/king/bravo/blob/master/bravo/src/test/java/com/king/bravo/
112 |
113 | #### Accessing non-keyed states
114 |
115 | The reader assumes that the machine that we run the code has enough memory to restore the non-keyed states locally. (This is mostly a safe assumption with the current operator state design)
116 |
117 | ```java
118 |
119 | // We restore the OperatorStateBackend in memory
120 | OperatorStateBackend stateBackend = reader.createOperatorStateBackendFromSnapshot(0);
121 |
122 | // Now we can access the state just like from the function
123 | stateBackend.getListState(...)
124 | stateBackend.getBroadcastState(...)
125 |
126 | ```
127 | ## Creating new savepoints
128 |
129 | ### OperatorStateWriter
130 |
131 | As the name suggests the `OperatorStateWriter` class provides utilities to change (replace/transform) the state for a single operator. Once we have a new valid operator state we will use some utility methods to create a new Savepoint.
132 |
133 | Let's continue our reading example by modifying the state of the all the users, then creating a new valid savepoint.
134 | ```java
135 | DataSet> countState = //see above example
136 |
137 | // We want to change our state based on some external data...
138 | DataSet> countsToAdd = environment.fromElements(
139 | Tuple2.of(0, 100), Tuple2.of(3, 1000),
140 | Tuple2.of(1, 100), Tuple2.of(2, 1000));
141 |
142 | // These are the new count states we want to put back in the state
143 | DataSet> newCounts = countState
144 | .join(countsToAdd)
145 | .where(0)
146 | .equalTo(0)
147 | .map(new SumValues());
148 |
149 | // We create a statetransformer that will store new checkpoint state under the newCheckpointDir base directory
150 | OperatorStateWriter writer = new OperatorStateWriter(savepoint, "CountPerKey", newCheckpointDir);
151 |
152 | writer.addValueState("Count", newCounts);
153 |
154 | // Once we are happy with all the modifications it's time to write the states to the persistent store
155 | OperatorState newOpState = writer.writeAll();
156 |
157 | // Last thing we do is create a new meta file that points to a valid savepoint
158 | StateMetadataUtils.writeSavepointMetadata(newCheckpointDir, StateMetadataUtils.createNewSavepoint(savepoint, newOpState));
159 | ```
160 |
161 | We can also use the StateTransformer to transform and replace non-keyed states:
162 |
163 | ```java
164 | writer.transformNonKeyedState(BiConsumer transformer);
165 | ```
166 |
167 | ## Contact us!
168 |
169 | The original King authors of this code:
170 | - David Artiga (david.artiga@king.com)
171 | - Gyula Fora (gyula.fora@king.com)
172 |
173 | With any questions or suggestions feel free to reach out to us in email anytime!
174 |
--------------------------------------------------------------------------------
/bravo-test-utils/.gitignore:
--------------------------------------------------------------------------------
1 | /build/
2 |
--------------------------------------------------------------------------------
/bravo-test-utils/build.gradle:
--------------------------------------------------------------------------------
1 |
2 | dependencies {
3 | compile group: 'org.apache.flink', name: 'flink-clients_2.11', version: versions.flink
4 | compile group: 'org.apache.flink', name: 'flink-runtime_2.11', version: versions.flink
5 | compile group: 'org.apache.flink', name: 'flink-test-utils_2.11', version: versions.flink
6 | compile group: 'org.apache.flink', name: 'flink-statebackend-rocksdb_2.11', version: versions.flink
7 | compile group: 'org.apache.flink', name: 'flink-streaming-java_2.11', version: versions.flink
8 | }
9 |
--------------------------------------------------------------------------------
/bravo-test-utils/src/main/java/com/king/bravo/testing/BravoTestPipeline.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 | package com.king.bravo.testing;
19 |
20 | import com.king.bravo.testing.actions.CancelJob;
21 | import com.king.bravo.testing.actions.NextWatermark;
22 | import com.king.bravo.testing.actions.Process;
23 | import com.king.bravo.testing.actions.Sleep;
24 | import com.king.bravo.testing.actions.TestPipelineSource;
25 | import com.king.bravo.testing.actions.TriggerFailure;
26 | import com.king.bravo.testing.actions.TriggerSavepoint;
27 | import org.apache.flink.api.common.JobID;
28 | import org.apache.flink.client.program.ClusterClient;
29 | import org.apache.flink.client.program.ProgramInvocationException;
30 | import org.apache.flink.configuration.CheckpointingOptions;
31 | import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
32 | import org.apache.flink.core.fs.FileStatus;
33 | import org.apache.flink.core.fs.FileSystem;
34 | import org.apache.flink.core.fs.Path;
35 | import org.apache.flink.runtime.checkpoint.Checkpoints;
36 | import org.apache.flink.runtime.checkpoint.savepoint.Savepoint;
37 | import org.apache.flink.runtime.jobgraph.JobGraph;
38 | import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
39 | import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation;
40 | import org.apache.flink.runtime.state.StateBackend;
41 | import org.apache.flink.runtime.state.filesystem.AbstractFsCheckpointStorage;
42 | import org.apache.flink.streaming.api.CheckpointingMode;
43 | import org.apache.flink.streaming.api.TimeCharacteristic;
44 | import org.apache.flink.streaming.api.datastream.DataStream;
45 | import org.apache.flink.streaming.api.environment.CheckpointConfig.ExternalizedCheckpointCleanup;
46 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
47 | import org.apache.flink.streaming.api.windowing.time.Time;
48 | import org.apache.flink.test.util.MiniClusterResource;
49 | import org.apache.flink.util.TestLogger;
50 | import org.junit.Before;
51 | import org.junit.Rule;
52 | import org.junit.rules.TemporaryFolder;
53 | import org.slf4j.Logger;
54 | import org.slf4j.LoggerFactory;
55 |
56 | import java.io.DataInputStream;
57 | import java.io.IOException;
58 | import java.io.Serializable;
59 | import java.lang.reflect.Method;
60 | import java.util.Arrays;
61 | import java.util.LinkedList;
62 | import java.util.List;
63 | import java.util.function.Function;
64 |
65 | public abstract class BravoTestPipeline extends TestLogger implements Serializable {
66 | private static final long serialVersionUID = 1L;
67 |
68 | protected final Logger logger = LoggerFactory.getLogger(getClass());
69 |
70 | @Rule
71 | public final TemporaryFolder folder = new TemporaryFolder();
72 |
73 | public static JobGraph jobGraph;
74 | public static ClusterClient> client;
75 | public static JobID jobID;
76 | public static LinkedList actions = new LinkedList<>();
77 |
78 | @Before
79 | public void cleanOutputs() {
80 | CollectingSink.OUTPUT.clear();
81 | actions.clear();
82 | }
83 |
84 | public Path getLastCheckpointPath() throws IOException {
85 | FileStatus[] listStatus = FileSystem.getLocalFileSystem()
86 | .listStatus(new Path(getCheckpointDir(), jobID.toString()));
87 |
88 | return Arrays.stream(listStatus)
89 | .filter(s -> s.getPath().getName().startsWith("chk"))
90 | .sorted((s1, s2) -> -Integer.compare(
91 | Integer.parseInt(s1.getPath().getName().split("-")[1]),
92 | Integer.parseInt(s2.getPath().getName().split("-")[1])))
93 | .findFirst()
94 | .map(s -> s.getPath())
95 | .orElseThrow(() -> new IllegalStateException("Cannot find any checkpoints"));
96 | }
97 |
98 | public List runTestPipeline(Function, DataStream> pipelinerBuilder)
99 | throws Exception {
100 | return runTestPipeline(2, null, pipelinerBuilder);
101 | }
102 |
103 | public List restoreTestPipelineFromSnapshot(String savepoint,
104 | Function, DataStream> pipelinerBuilder) throws Exception {
105 | return runTestPipeline(2, savepoint, pipelinerBuilder);
106 | }
107 |
108 | public List restoreTestPipelineFromLastCheckpoint(
109 | Function, DataStream> pipelinerBuilder) throws Exception {
110 | return restoreTestPipelineFromSnapshot(getLastCheckpointPath().getPath(), pipelinerBuilder);
111 | }
112 |
113 | public List restoreTestPipelineFromLastSavepoint(
114 | Function, DataStream> pipelinerBuilder) throws Exception {
115 | if (TriggerSavepoint.lastSavepointPath == null) {
116 | throw new RuntimeException("triggerSavepoint must be called to obtain a valid savepoint");
117 | }
118 | return restoreTestPipelineFromSnapshot(TriggerSavepoint.lastSavepointPath, pipelinerBuilder);
119 | }
120 |
121 | private StreamExecutionEnvironment createJobGraph(int parallelism,
122 | Function, DataStream> pipelinerBuilder) throws Exception {
123 | final Path checkpointDir = getCheckpointDir();
124 | final Path savepointRootDir = getSavepointDir();
125 |
126 | checkpointDir.getFileSystem().mkdirs(checkpointDir);
127 | savepointRootDir.getFileSystem().mkdirs(savepointRootDir);
128 |
129 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
130 | env.getConfig().disableSysoutLogging();
131 | env.getCheckpointConfig().enableExternalizedCheckpoints(ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
132 | env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
133 | env.setBufferTimeout(0);
134 | env.setParallelism(parallelism);
135 | env.enableCheckpointing(500, CheckpointingMode.EXACTLY_ONCE);
136 |
137 | env.setStateBackend((StateBackend) new RocksDBStateBackend(checkpointDir.toString(), true));
138 |
139 | DataStream sourceData = env
140 | .addSource(new TestPipelineSource())
141 | .uid("TestSource")
142 | .name("TestSource")
143 | .setParallelism(1);
144 |
145 | pipelinerBuilder.apply(sourceData)
146 | .addSink(new CollectingSink()).name("Output").uid("Output")
147 | .setParallelism(1);
148 |
149 | return env;
150 | }
151 |
152 | private List runTestPipeline(int parallelism, String savepoint,
153 | Function, DataStream> pipelinerBuilder) throws Exception {
154 |
155 | if (!actions.isEmpty() && actions.getLast() instanceof CancelJob
156 | && ((CancelJob) actions.getLast()).isClusterActionTriggered()) {
157 | cancelJob();
158 | }
159 |
160 | jobGraph = createJobGraph(parallelism, pipelinerBuilder).getStreamGraph().getJobGraph();
161 | if (savepoint != null) {
162 | jobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepoint));
163 | }
164 | jobID = jobGraph.getJobID();
165 |
166 | MiniClusterResourceFactory clusterFactory = createCluster(1, 2);
167 | MiniClusterResource cluster = clusterFactory.get();
168 | cluster.before();
169 | client = cluster.getClusterClient();
170 |
171 | try {
172 | // client.setDetached(true);
173 | client.submitJob(jobGraph, BravoTestPipeline.class.getClassLoader());
174 | } catch (ProgramInvocationException pie) {
175 | if (!pie.getMessage().contains("Job was cancelled")
176 | && !pie.getCause().getMessage().contains("Job was cancelled")) {
177 | throw pie;
178 | }
179 | } finally {
180 | cluster.after();
181 | }
182 |
183 | return CollectingSink.OUTPUT;
184 | }
185 |
186 | protected Path getCheckpointDir() {
187 | return new Path("file://" + folder.getRoot().getAbsolutePath(), "checkpoints");
188 | }
189 |
190 | protected Path getSavepointDir() {
191 | return new Path("file://" + folder.getRoot().getAbsolutePath(), "savepoints");
192 | }
193 |
194 | protected Path getLastSavepointPath() {
195 | return new Path(TriggerSavepoint.lastSavepointPath);
196 | }
197 |
198 | protected Savepoint getLastCheckpoint() throws IOException {
199 | return loadSavepoint(getLastCheckpointPath().getPath());
200 | }
201 |
202 | protected Savepoint getLastSavepoint() throws IOException {
203 | return loadSavepoint(getLastSavepointPath().getPath());
204 | }
205 |
206 | public void process(String element) {
207 | actions.add(new Process(element, 0));
208 | }
209 |
210 | public void process(String element, long ts) {
211 | actions.add(new Process(element, ts));
212 | }
213 |
214 | public void triggerFailure() {
215 | actions.add(new TriggerFailure());
216 | }
217 |
218 | public void triggerSavepoint() {
219 | actions.add(new TriggerSavepoint());
220 | }
221 |
222 | public void cancelJob() {
223 | actions.add(new CancelJob());
224 | }
225 |
226 | public void processWatermark(long timestamp) {
227 | actions.add(new NextWatermark(timestamp));
228 | }
229 |
230 | public void sleep(long millis) {
231 | actions.add(new Sleep(millis));
232 | }
233 |
234 | public void sleep(Time time) {
235 | sleep(time.toMilliseconds());
236 | }
237 |
238 | public static Savepoint loadSavepoint(String checkpointPointer) {
239 | try {
240 | Method resolveCheckpointPointer = AbstractFsCheckpointStorage.class.getDeclaredMethod(
241 | "resolveCheckpointPointer",
242 | String.class);
243 | resolveCheckpointPointer.setAccessible(true);
244 | CompletedCheckpointStorageLocation loc = (CompletedCheckpointStorageLocation) resolveCheckpointPointer
245 | .invoke(null, checkpointPointer);
246 |
247 | return Checkpoints.loadCheckpointMetadata(new DataInputStream(loc.getMetadataHandle().openInputStream()),
248 | BravoTestPipeline.class.getClassLoader());
249 | } catch (Exception e) {
250 | throw new RuntimeException(e);
251 | }
252 |
253 | }
254 |
255 | private MiniClusterResourceFactory createCluster(final int numTaskManagers,
256 | final int numSlotsPerTaskManager) {
257 | org.apache.flink.configuration.Configuration config = new org.apache.flink.configuration.Configuration();
258 | config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, getCheckpointDir().toUri().toString());
259 | config.setInteger(CheckpointingOptions.FS_SMALL_FILE_THRESHOLD, 0);
260 | config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, getSavepointDir().toUri().toString());
261 |
262 | MiniClusterResourceFactory clusterFactory = new MiniClusterResourceFactory(numTaskManagers,
263 | numSlotsPerTaskManager, config);
264 | return clusterFactory;
265 | }
266 | }
267 |
--------------------------------------------------------------------------------
/bravo-test-utils/src/main/java/com/king/bravo/testing/CollectingSink.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 | package com.king.bravo.testing;
19 |
20 | import java.util.ArrayList;
21 | import java.util.List;
22 |
23 | import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
24 | import org.apache.flink.streaming.api.functions.sink.SinkFunction;
25 |
26 | import com.google.common.collect.Lists;
27 |
28 | public class CollectingSink implements SinkFunction, ListCheckpointed {
29 | private static final long serialVersionUID = 1L;
30 |
31 | public static List OUTPUT = new ArrayList<>();
32 |
33 | public CollectingSink() {}
34 |
35 | @Override
36 | public void invoke(String out) throws Exception {
37 | OUTPUT.add(out);
38 | }
39 |
40 | @Override
41 | public List snapshotState(long checkpointId, long timestamp) throws Exception {
42 | return Lists.newArrayList(OUTPUT.size());
43 | }
44 |
45 | @Override
46 | public void restoreState(List state) throws Exception {
47 | int lastValid = state.get(0);
48 | List newOutput = new ArrayList<>();
49 | for (int i = 0; i < lastValid; i++) {
50 | newOutput.add(OUTPUT.get(i));
51 | }
52 | OUTPUT = (List) newOutput;
53 | }
54 |
55 | }
56 |
--------------------------------------------------------------------------------
/bravo-test-utils/src/main/java/com/king/bravo/testing/MiniClusterResourceFactory.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 | package com.king.bravo.testing;
19 |
20 | import org.apache.flink.configuration.Configuration;
21 | import org.apache.flink.test.util.MiniClusterResource;
22 | import org.apache.flink.test.util.MiniClusterResourceConfiguration;
23 |
24 | public class MiniClusterResourceFactory {
25 | private final int numTaskManagers;
26 | private final int numSlotsPerTaskManager;
27 | private final Configuration config;
28 |
29 | public MiniClusterResourceFactory(int numTaskManagers, int numSlotsPerTaskManager, Configuration config) {
30 | this.numTaskManagers = numTaskManagers;
31 | this.numSlotsPerTaskManager = numSlotsPerTaskManager;
32 | this.config = config;
33 | }
34 |
35 | public MiniClusterResource get() {
36 | return new MiniClusterResource(
37 | new MiniClusterResourceConfiguration.Builder()
38 | .setConfiguration(config)
39 | .setNumberTaskManagers(numTaskManagers)
40 | .setNumberSlotsPerTaskManager(numSlotsPerTaskManager)
41 | .build());
42 | }
43 | }
44 |
--------------------------------------------------------------------------------
/bravo-test-utils/src/main/java/com/king/bravo/testing/PipelineAction.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 | package com.king.bravo.testing;
19 |
20 | import org.apache.flink.api.common.JobID;
21 | import org.apache.flink.client.program.ClusterClient;
22 | import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
23 |
24 | public interface PipelineAction {
25 |
26 | default void withCheckpointLock(SourceContext ctx)
27 | throws Exception {};
28 |
29 | default void executeClusterAction(ClusterClient> client, JobID id) throws Exception {}
30 |
31 | }
32 |
--------------------------------------------------------------------------------
/bravo-test-utils/src/main/java/com/king/bravo/testing/actions/CancelJob.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 | package com.king.bravo.testing.actions;
19 |
20 | import org.apache.flink.api.common.JobID;
21 | import org.apache.flink.client.program.ClusterClient;
22 |
23 | public class CancelJob extends OneTimePipelineAction {
24 |
25 | @Override
26 | public void onceExecuteClusterAction(ClusterClient> client, JobID id) throws Exception {
27 | client.cancel(id);
28 | }
29 | }
30 |
--------------------------------------------------------------------------------
/bravo-test-utils/src/main/java/com/king/bravo/testing/actions/NextWatermark.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 | package com.king.bravo.testing.actions;
19 |
20 | import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
21 | import org.apache.flink.streaming.api.watermark.Watermark;
22 |
23 | import com.king.bravo.testing.PipelineAction;
24 |
25 | public class NextWatermark implements PipelineAction {
26 | long watermark;
27 |
28 | public NextWatermark(long watermark) {
29 | this.watermark = watermark;
30 | }
31 |
32 | @Override
33 | public void withCheckpointLock(SourceContext ctx) throws Exception {
34 | ctx.emitWatermark(new Watermark(watermark));
35 | Thread.sleep(500);
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/bravo-test-utils/src/main/java/com/king/bravo/testing/actions/OneTimePipelineAction.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 | package com.king.bravo.testing.actions;
19 |
20 | import org.apache.flink.api.common.JobID;
21 | import org.apache.flink.client.program.ClusterClient;
22 | import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
23 |
24 | import com.king.bravo.testing.PipelineAction;
25 |
26 | public abstract class OneTimePipelineAction implements PipelineAction {
27 |
28 | private boolean checkpointLockTriggered = false;
29 | private boolean clusterActionTriggered = false;
30 |
31 | public final void withCheckpointLock(SourceContext ctx) {
32 | if (checkpointLockTriggered) {
33 | return;
34 | }
35 |
36 | checkpointLockTriggered = true;
37 | onceWithCheckpointLock(ctx);
38 | }
39 |
40 | protected void onceWithCheckpointLock(SourceContext ctx) {}
41 |
42 | public final void executeClusterAction(ClusterClient> client, JobID id) throws Exception {
43 |
44 | if (clusterActionTriggered) {
45 | return;
46 | }
47 |
48 | clusterActionTriggered = true;
49 | onceExecuteClusterAction(client, id);
50 | };
51 |
52 | protected void onceExecuteClusterAction(ClusterClient> client, JobID id) throws Exception {}
53 |
54 | public boolean isCheckpointLockTriggered() {
55 | return checkpointLockTriggered;
56 | }
57 |
58 | public boolean isClusterActionTriggered() {
59 | return clusterActionTriggered;
60 | }
61 |
62 | }
63 |
--------------------------------------------------------------------------------
/bravo-test-utils/src/main/java/com/king/bravo/testing/actions/Process.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 | package com.king.bravo.testing.actions;
19 |
20 | import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
21 |
22 | import com.king.bravo.testing.PipelineAction;
23 |
24 | public class Process implements PipelineAction {
25 |
26 | final String output;
27 | final long timestamp;
28 |
29 | public Process(String output, long timestamp) {
30 | this.output = output;
31 | this.timestamp = timestamp;
32 | }
33 |
34 | @Override
35 | public void withCheckpointLock(SourceContext ctx) throws Exception {
36 | ctx.collectWithTimestamp(output, timestamp);
37 | }
38 | }
39 |
--------------------------------------------------------------------------------
/bravo-test-utils/src/main/java/com/king/bravo/testing/actions/Sleep.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 | package com.king.bravo.testing.actions;
19 |
20 | import org.apache.flink.api.common.JobID;
21 | import org.apache.flink.client.program.ClusterClient;
22 |
23 | import com.king.bravo.testing.PipelineAction;
24 |
25 | public class Sleep implements PipelineAction {
26 |
27 | long sleepTime;
28 |
29 | public Sleep(long sleepTime) {
30 | this.sleepTime = sleepTime;
31 | }
32 |
33 | @Override
34 | public void executeClusterAction(ClusterClient> client, JobID id) throws Exception {
35 | Thread.sleep(sleepTime);
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/bravo-test-utils/src/main/java/com/king/bravo/testing/actions/TestPipelineSource.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 | package com.king.bravo.testing.actions;
19 |
20 | import java.io.Serializable;
21 | import java.util.Collections;
22 | import java.util.List;
23 |
24 | import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
25 | import org.apache.flink.streaming.api.functions.source.SourceFunction;
26 | import org.apache.flink.streaming.api.watermark.Watermark;
27 |
28 | import com.king.bravo.testing.BravoTestPipeline;
29 | import com.king.bravo.testing.PipelineAction;
30 |
31 | public class TestPipelineSource implements SourceFunction, ListCheckpointed {
32 |
33 | private static final long serialVersionUID = 1L;
34 |
35 | private static long DEFAULT_SLEEP = 100;
36 |
37 | private volatile boolean isRunning = false;
38 | private int index = 0;
39 |
40 | @Override
41 | public void run(SourceContext ctx) throws Exception {
42 |
43 | isRunning = true;
44 |
45 | while (index < BravoTestPipeline.actions.size()) {
46 | if (!isRunning) {
47 | return;
48 | }
49 |
50 | PipelineAction nextAction = BravoTestPipeline.actions.get(index);
51 | synchronized (ctx.getCheckpointLock()) {
52 | index++;
53 | nextAction.withCheckpointLock(ctx);
54 | }
55 | nextAction.executeClusterAction(BravoTestPipeline.client, BravoTestPipeline.jobID);
56 | Thread.sleep(DEFAULT_SLEEP);
57 | }
58 |
59 | Thread.sleep(DEFAULT_SLEEP);
60 | ctx.emitWatermark(new Watermark(Long.MAX_VALUE));
61 | }
62 |
63 | @Override
64 | public void cancel() {
65 | isRunning = false;
66 | }
67 |
68 | @Override
69 | public void restoreState(List s) throws Exception {
70 | index = (int) s.get(0);
71 | }
72 |
73 | @Override
74 | public List snapshotState(long arg0, long arg1) throws Exception {
75 | return Collections.singletonList(index);
76 | }
77 |
78 | }
79 |
--------------------------------------------------------------------------------
/bravo-test-utils/src/main/java/com/king/bravo/testing/actions/TriggerFailure.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 | package com.king.bravo.testing.actions;
19 |
20 | import org.apache.flink.api.common.JobID;
21 | import org.apache.flink.client.program.ClusterClient;
22 |
23 | public class TriggerFailure extends OneTimePipelineAction {
24 |
25 | @Override
26 | public void onceExecuteClusterAction(ClusterClient> client, JobID id) throws Exception {
27 | Thread.sleep(1000);
28 | throw new RuntimeException("Triggered failure");
29 | }
30 | }
31 |
--------------------------------------------------------------------------------
/bravo-test-utils/src/main/java/com/king/bravo/testing/actions/TriggerSavepoint.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 | package com.king.bravo.testing.actions;
19 |
20 | import org.apache.flink.api.common.JobID;
21 | import org.apache.flink.client.program.ClusterClient;
22 |
23 | public class TriggerSavepoint extends OneTimePipelineAction {
24 |
25 | public static String lastSavepointPath;
26 |
27 | @Override
28 | public void onceExecuteClusterAction(ClusterClient> client, JobID id) throws Exception {
29 | lastSavepointPath = client.triggerSavepoint(id, null).get();
30 | }
31 | }
32 |
--------------------------------------------------------------------------------
/bravo-test-utils/src/test/resources/logback-test.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 |
5 |
6 | %date [%5level] - %msg \(%F:%L\) [%thread]%n
7 |
8 |
9 |
10 |
11 |
15 |
16 |
17 |
18 |
19 |
20 |
21 |
--------------------------------------------------------------------------------
/bravo/.gitignore:
--------------------------------------------------------------------------------
1 | /build/
2 |
--------------------------------------------------------------------------------
/bravo/build.gradle:
--------------------------------------------------------------------------------
1 |
2 | dependencies {
3 | compile group: 'com.google.guava', name: 'guava', version: '18.0'
4 | compileOnly group: 'org.apache.flink', name: 'flink-streaming-java_2.11', version: versions.flink
5 | compileOnly group: 'org.apache.flink', name: 'flink-statebackend-rocksdb_2.11', version: versions.flink
6 | testCompile (project(':bravo-test-utils'))
7 | }
8 |
--------------------------------------------------------------------------------
/bravo/src/main/java/com/king/bravo/reader/AbstractListStateReader.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 | package com.king.bravo.reader;
19 |
20 | import org.apache.flink.api.common.typeinfo.TypeInformation;
21 | import org.apache.flink.api.common.typeutils.TypeSerializer;
22 | import org.apache.flink.api.common.typeutils.base.ListSerializer;
23 |
24 | public abstract class AbstractListStateReader extends KeyedStateReader {
25 |
26 | private static final long serialVersionUID = 1L;
27 |
28 | protected AbstractListStateReader(String stateName, TypeInformation keyType, TypeInformation valueType,
29 | TypeInformation outType) {
30 | super(stateName, keyType, valueType, outType);
31 | }
32 |
33 | public void configure(int maxParallelism, TypeSerializer> keySerializer, TypeSerializer> valueSerializer)
34 | throws Exception {
35 |
36 | if (valueSerializer instanceof ListSerializer>) {
37 | super.configure(maxParallelism, keySerializer,
38 | ((ListSerializer>) valueSerializer).getElementSerializer());
39 | } else {
40 | throw new RuntimeException("Doesnt seem to be a list state");
41 | }
42 | }
43 | }
44 |
--------------------------------------------------------------------------------
/bravo/src/main/java/com/king/bravo/reader/AbstractMapStateReader.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 | package com.king.bravo.reader;
19 |
20 | import org.apache.flink.api.common.typeinfo.TypeInformation;
21 | import org.apache.flink.api.common.typeutils.TypeSerializer;
22 | import org.apache.flink.api.common.typeutils.base.MapSerializer;
23 | import org.apache.flink.configuration.Configuration;
24 |
25 | public abstract class AbstractMapStateReader extends KeyedStateReader {
26 |
27 | private static final long serialVersionUID = 1L;
28 | protected TypeInformation> mapKeytype = null;
29 | protected TypeSerializer> mapKeySerializer = null;
30 |
31 | protected AbstractMapStateReader(String stateName, TypeInformation keyType, TypeInformation valueType,
32 | TypeInformation outType) {
33 | super(stateName, keyType, valueType, outType);
34 | }
35 |
36 | public KeyedStateReader withMapKeyDeserializer(TypeInformation> type) {
37 | this.mapKeytype = type;
38 | return this;
39 | }
40 |
41 | public KeyedStateReader withMapKeyDeserializer(TypeSerializer> serializer) {
42 | this.mapKeySerializer = serializer;
43 | return this;
44 | }
45 |
46 | @SuppressWarnings("unchecked")
47 | public void configure(int maxParallelism, TypeSerializer> keySerializer, TypeSerializer> valueSerializer)
48 | throws Exception {
49 |
50 | if (valueSerializer instanceof MapSerializer) {
51 | MapSerializer, ?> mapSerializer = (MapSerializer, ?>) valueSerializer;
52 | if (mapKeytype == null && mapKeySerializer == null) {
53 | mapKeySerializer = (TypeSerializer