├── .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** [![Maven Central](https://maven-badges.herokuapp.com/maven-central/com.king.bravo/bravo/badge.svg)](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** [![Maven Central](https://maven-badges.herokuapp.com/maven-central/com.king.bravo/bravo-test-utils/badge.svg)](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) mapSerializer.getKeySerializer(); 54 | } 55 | 56 | super.configure(maxParallelism, keySerializer, mapSerializer.getValueSerializer()); 57 | } else { 58 | throw new RuntimeException("Doesnt seem like a map state"); 59 | } 60 | } 61 | 62 | @Override 63 | public void open(Configuration c) { 64 | super.open(c); 65 | if (mapKeySerializer == null) { 66 | mapKeySerializer = mapKeytype.createSerializer(getRuntimeContext().getExecutionConfig()); 67 | } 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/reader/KeyedStateReader.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 com.king.bravo.types.KeyedStateRow; 21 | import com.king.bravo.utils.StateMetadataUtils; 22 | import org.apache.commons.lang3.Validate; 23 | import org.apache.flink.api.common.ExecutionConfig; 24 | import org.apache.flink.api.common.functions.RichFlatMapFunction; 25 | import org.apache.flink.api.common.typeinfo.TypeHint; 26 | import org.apache.flink.api.common.typeinfo.TypeInformation; 27 | import org.apache.flink.api.common.typeutils.TypeSerializer; 28 | import org.apache.flink.api.java.tuple.Tuple2; 29 | import org.apache.flink.api.java.tuple.Tuple3; 30 | import org.apache.flink.api.java.typeutils.ResultTypeQueryable; 31 | import org.apache.flink.api.java.typeutils.TupleTypeInfo; 32 | import org.apache.flink.configuration.Configuration; 33 | import org.apache.flink.core.memory.DataInputViewStreamWrapper; 34 | import org.slf4j.Logger; 35 | import org.slf4j.LoggerFactory; 36 | 37 | import java.io.IOException; 38 | import java.util.List; 39 | 40 | public abstract class KeyedStateReader extends RichFlatMapFunction 41 | implements ResultTypeQueryable { 42 | 43 | private static final Logger LOGGER = LoggerFactory.getLogger(KeyedStateReader.class); 44 | private static final long serialVersionUID = 1L; 45 | 46 | // The name "window-contents" appears as a plain string in many places in Flink source code. There is no constant 47 | // that we could refer from Flink. 48 | private static final String WINDOW_STATE_NAME = "window-contents"; 49 | 50 | protected final String stateName; 51 | 52 | protected TypeSerializer keyDeserializer; 53 | protected TypeSerializer valueDeserializer; 54 | protected TypeInformation keyDeserializerType; 55 | protected TypeInformation valueDeserializerType; 56 | 57 | private final TypeInformation outKeyType; 58 | private final TypeInformation outValueType; 59 | private final TypeInformation outType; 60 | 61 | protected int keygroupPrefixBytes; 62 | 63 | protected boolean initialized = false; 64 | 65 | protected boolean ttlState = false; 66 | 67 | protected KeyedStateReader(String stateName, TypeInformation keyType, TypeInformation valueType, 68 | TypeInformation outType) { 69 | this.stateName = stateName; 70 | this.outType = outType; 71 | this.outValueType = valueType; 72 | this.outKeyType = keyType; 73 | } 74 | 75 | @Override 76 | public TypeInformation getProducedType() { 77 | if (!initialized) { 78 | throw new RuntimeException("Parser not initialized, use it with KeyedStateReader#parseKeyedStateRows"); 79 | } 80 | return outType; 81 | } 82 | 83 | /** 84 | * Internal method used to configure the reader based on the snapshot 85 | * information, should not be called by the user. 86 | */ 87 | @SuppressWarnings("unchecked") 88 | public void configure(int maxParallelism, TypeSerializer keySerializer, TypeSerializer valueSerializer) 89 | throws Exception { 90 | 91 | keygroupPrefixBytes = StateMetadataUtils.getKeyGroupPrefixBytes(maxParallelism); 92 | 93 | if (this.keyDeserializer == null && this.keyDeserializerType == null) { 94 | this.keyDeserializer = (TypeSerializer) keySerializer; 95 | } 96 | 97 | boolean ttlSerializer = StateMetadataUtils.isTtlState(valueSerializer); 98 | if (ttlSerializer) { 99 | ttlState = true; 100 | } 101 | 102 | if (this.valueDeserializer == null && this.valueDeserializerType == null) { 103 | this.valueDeserializer = ttlSerializer ? StateMetadataUtils.unwrapTtlSerializer(valueSerializer) 104 | : (TypeSerializer) valueSerializer; 105 | } 106 | initialized = true; 107 | } 108 | 109 | /** 110 | * Use the provided output types to deserialize the state. By default the 111 | * serializers from the state are used for this. 112 | * 113 | */ 114 | public KeyedStateReader withOutputTypesForDeserialization() { 115 | withKeyDeserializer(outKeyType); 116 | withValueDeserializer(outValueType); 117 | return this; 118 | } 119 | 120 | /** 121 | * Use the provided serializer to deserialize the key bytes. 122 | * 123 | * @param keyDeserializer 124 | */ 125 | public KeyedStateReader withKeyDeserializer(TypeInformation keyDeserializer) { 126 | this.keyDeserializerType = Validate.notNull(keyDeserializer); 127 | return this; 128 | } 129 | 130 | /** 131 | * Use the provided serializer to deserialize the value bytes. 132 | * 133 | * @param valueDeserializer 134 | */ 135 | public KeyedStateReader withValueDeserializer(TypeInformation valueDeserializer) { 136 | this.valueDeserializerType = Validate.notNull(valueDeserializer); 137 | return this; 138 | } 139 | 140 | /** 141 | * Use the provided serializer to deserialize the key bytes. 142 | * 143 | * @param keyDeserializer 144 | */ 145 | public KeyedStateReader withKeyDeserializer(TypeSerializer keyDeserializer) { 146 | this.keyDeserializer = Validate.notNull(keyDeserializer); 147 | return this; 148 | } 149 | 150 | /** 151 | * Use the provided serializer to deserialize the value bytes. 152 | * 153 | * @param valueDeserializer 154 | */ 155 | public KeyedStateReader withValueDeserializer(TypeSerializer valueDeserializer) { 156 | this.valueDeserializer = Validate.notNull(valueDeserializer); 157 | return this; 158 | } 159 | 160 | @Override 161 | public void open(Configuration c) { 162 | ExecutionConfig executionConfig = getRuntimeContext().getExecutionConfig(); 163 | 164 | if (keyDeserializer == null) { 165 | keyDeserializer = keyDeserializerType.createSerializer(executionConfig); 166 | } 167 | 168 | if (valueDeserializer == null) { 169 | valueDeserializer = valueDeserializerType.createSerializer(executionConfig); 170 | } 171 | 172 | LOGGER.info("Initialized KeyedStateReader: keyDeserializer: {} valueDeserializer: {}", keyDeserializer, 173 | valueDeserializer); 174 | } 175 | 176 | /** 177 | * Create a reader for reading the state key-value pairs for the given value 178 | * state name. The provided type info will be used to deserialize the state 179 | * (allowing possible optimizations) 180 | */ 181 | public static KeyedStateReader> forValueStateKVPairs(String stateName, 182 | TypeInformation outKeyType, 183 | TypeInformation outValueType) { 184 | return new ValueStateKVReader<>(stateName, outKeyType, outValueType); 185 | } 186 | 187 | /** 188 | * Create a reader for reading the state key-value pairs for the given value 189 | * state name. The provided type info will be used to deserialize the state 190 | * (allowing possible optimizations) 191 | */ 192 | public static KeyedStateReader> forValueStateKVPairs(String stateName, 193 | TypeHint> tupleTypeHint) { 194 | TupleTypeInfo> tupleType = (TupleTypeInfo>) tupleTypeHint.getTypeInfo(); 195 | return forValueStateKVPairs(stateName, tupleType.getTypeAt(0), tupleType.getTypeAt(1)); 196 | } 197 | 198 | /** 199 | * Create a reader for reading the state values for the given value state 200 | * name. The provided type info will be used to deserialize the state 201 | * (allowing possible optimizations) 202 | */ 203 | public static KeyedStateReader forValueStateValues(String stateName, 204 | TypeInformation outValueType) { 205 | return new ValueStateValueReader<>(stateName, outValueType); 206 | } 207 | 208 | /** 209 | * Create a reader for reading the state values for the given map state 210 | * name. The provided type info will be used to deserialize the state 211 | * (allowing possible optimizations) 212 | */ 213 | public static KeyedStateReader forMapStateValues(String stateName, 214 | TypeInformation outValueType) { 215 | return forMapStateValues(stateName, outValueType, false); 216 | } 217 | 218 | /** 219 | * Create a reader for reading the state values for the given map state 220 | * name. The provided type info will be used to deserialize the state 221 | * (allowing possible optimizations) 222 | */ 223 | public static KeyedStateReader forMapStateValues(String stateName, 224 | TypeInformation outValueType, boolean ttlState) { 225 | return new MapStateValueReader(stateName, outValueType); 226 | } 227 | 228 | /** 229 | * Create a reader for reading the state values for the given list state. 230 | * The provided type info will be used to deserialize the state (allowing 231 | * possible optimizations) 232 | */ 233 | public static KeyedStateReader> forListStateValues(String stateName, 234 | TypeInformation outKeyType, TypeInformation outValueType) { 235 | return new ListStateFlattenReader<>(stateName, outKeyType, outValueType); 236 | } 237 | 238 | /** 239 | * Create a reader for reading the state values for the given list state. 240 | * The provided type info will be used to deserialize the state (allowing 241 | * possible optimizations) 242 | */ 243 | public static KeyedStateReader>> forListStates(String stateName, 244 | TypeInformation outKeyType, TypeInformation outValueType) { 245 | return new ListStateListReader<>(stateName, outKeyType, outValueType); 246 | } 247 | 248 | /** 249 | * Create a reader for reading the state key-mapkey-value triplets for the 250 | * given map state name. The provided type info will be used to deserialize 251 | * the state (allowing possible optimizations) 252 | */ 253 | public static KeyedStateReader> forMapStateEntries(String stateName, 254 | TypeInformation outKeyType, TypeInformation outMapKeyType, TypeInformation outValueType) { 255 | return new MapStateKKVReader<>(stateName, outKeyType, outMapKeyType, outValueType); 256 | } 257 | 258 | /** 259 | * Create a reader for state values of a window operator. The provided type info will be used to deserialize the 260 | * state (allowing possible optimizations) 261 | */ 262 | public static KeyedStateReader forWindowStateValues(TypeInformation outValueType) { 263 | return new ValueStateValueReader(WINDOW_STATE_NAME, outValueType); 264 | } 265 | 266 | /** 267 | * Create a reader for state key-value pairs of a window operator. The provided type info will be used to 268 | * deserialize the state (allowing possible optimizations) 269 | */ 270 | public static KeyedStateReader> forWindowStateKVPairs( 271 | TypeInformation outKeyType, TypeInformation outValueType) { 272 | return new ValueStateKVReader(WINDOW_STATE_NAME, outKeyType, outValueType); 273 | } 274 | 275 | public String getStateName() { 276 | return stateName; 277 | } 278 | 279 | protected void skipTimestampIfTtlEnabled(DataInputViewStreamWrapper iw) throws IOException { 280 | if (ttlState) { 281 | iw.skipBytesToRead(Long.BYTES); 282 | } 283 | } 284 | } 285 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/reader/ListStateFlattenReader.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.java.tuple.Tuple2; 22 | import org.apache.flink.api.java.typeutils.TupleTypeInfo; 23 | import org.apache.flink.core.memory.ByteArrayInputStreamWithPos; 24 | import org.apache.flink.core.memory.DataInputViewStreamWrapper; 25 | import org.apache.flink.util.Collector; 26 | 27 | import com.king.bravo.types.KeyedStateRow; 28 | import com.king.bravo.utils.RocksDBUtils; 29 | 30 | public class ListStateFlattenReader extends AbstractListStateReader> { 31 | 32 | private static final long serialVersionUID = 1L; 33 | 34 | @SuppressWarnings({ "rawtypes", "unchecked" }) 35 | public ListStateFlattenReader(String stateName, TypeInformation outKeyType, TypeInformation outValueType) { 36 | super(stateName, outKeyType, outValueType, new TupleTypeInfo(Tuple2.class, outKeyType, outValueType)); 37 | } 38 | 39 | @Override 40 | public void flatMap(KeyedStateRow row, Collector> out) throws Exception { 41 | if (!stateName.equals(row.getStateName())) { 42 | return; 43 | } 44 | 45 | byte[] keyAndNamespaceBytes = row.getKeyAndNamespaceBytes(); 46 | byte[] valueBytes = row.getValueBytes(); 47 | 48 | K key; 49 | try (ByteArrayInputStreamWithPos keyIs = new ByteArrayInputStreamWithPos(keyAndNamespaceBytes)) { 50 | DataInputViewStreamWrapper iw = new DataInputViewStreamWrapper(keyIs); 51 | iw.skipBytesToRead(keygroupPrefixBytes); 52 | key = RocksDBUtils.readKey(keyDeserializer, keyIs, iw, false); 53 | } 54 | 55 | try (ByteArrayInputStreamWithPos valueIs = new ByteArrayInputStreamWithPos(valueBytes)) { 56 | DataInputViewStreamWrapper iw = new DataInputViewStreamWrapper(valueIs); 57 | V next = RocksDBUtils.deserializeNextElement(iw, valueDeserializer, ttlState); 58 | while (next != null) { 59 | out.collect(Tuple2.of(key, next)); 60 | next = RocksDBUtils.deserializeNextElement(iw, valueDeserializer, ttlState); 61 | } 62 | } 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/reader/ListStateListReader.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 java.util.List; 21 | 22 | import org.apache.flink.api.common.typeinfo.TypeInformation; 23 | import org.apache.flink.api.java.tuple.Tuple2; 24 | import org.apache.flink.api.java.typeutils.ListTypeInfo; 25 | import org.apache.flink.api.java.typeutils.TupleTypeInfo; 26 | import org.apache.flink.core.memory.ByteArrayInputStreamWithPos; 27 | import org.apache.flink.core.memory.DataInputViewStreamWrapper; 28 | import org.apache.flink.util.Collector; 29 | 30 | import com.king.bravo.types.KeyedStateRow; 31 | import com.king.bravo.utils.RocksDBUtils; 32 | 33 | public class ListStateListReader extends AbstractListStateReader>> { 34 | 35 | private static final long serialVersionUID = 1L; 36 | 37 | @SuppressWarnings({ "rawtypes", "unchecked" }) 38 | public ListStateListReader(String stateName, TypeInformation outKeyType, TypeInformation outValueType) { 39 | super(stateName, outKeyType, outValueType, 40 | new TupleTypeInfo(Tuple2.class, outKeyType, new ListTypeInfo<>(outValueType))); 41 | } 42 | 43 | @Override 44 | public void flatMap(KeyedStateRow row, Collector>> out) throws Exception { 45 | if (!stateName.equals(row.getStateName())) { 46 | return; 47 | } 48 | 49 | byte[] keyAndNamespaceBytes = row.getKeyAndNamespaceBytes(); 50 | byte[] valueBytes = row.getValueBytes(); 51 | 52 | K key; 53 | try (ByteArrayInputStreamWithPos keyIs = new ByteArrayInputStreamWithPos(keyAndNamespaceBytes)) { 54 | DataInputViewStreamWrapper iw = new DataInputViewStreamWrapper(keyIs); 55 | iw.skipBytesToRead(keygroupPrefixBytes); 56 | key = RocksDBUtils.readKey(keyDeserializer, keyIs, iw, false); 57 | } 58 | 59 | out.collect(Tuple2.of(key, RocksDBUtils.deserializeList(valueBytes, valueDeserializer, ttlState))); 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/reader/MapStateKKVReader.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.java.tuple.Tuple3; 22 | import org.apache.flink.api.java.typeutils.TupleTypeInfo; 23 | import org.apache.flink.core.memory.ByteArrayInputStreamWithPos; 24 | import org.apache.flink.core.memory.DataInputViewStreamWrapper; 25 | import org.apache.flink.runtime.state.VoidNamespaceSerializer; 26 | import org.apache.flink.util.Collector; 27 | 28 | import com.king.bravo.types.KeyedStateRow; 29 | import com.king.bravo.utils.RocksDBUtils; 30 | 31 | public class MapStateKKVReader extends AbstractMapStateReader> { 32 | 33 | private static final long serialVersionUID = 1L; 34 | private TypeInformation outMapKeyType; 35 | 36 | @SuppressWarnings({ "rawtypes", "unchecked" }) 37 | public MapStateKKVReader(String stateName, TypeInformation outKeyType, TypeInformation mapKeytype, 38 | TypeInformation outValueType) { 39 | super(stateName, outKeyType, outValueType, 40 | new TupleTypeInfo(Tuple3.class, outKeyType, mapKeytype, outValueType)); 41 | outMapKeyType = mapKeytype; 42 | } 43 | 44 | @Override 45 | public KeyedStateReader> withOutputTypesForDeserialization() { 46 | mapKeytype = outMapKeyType; 47 | return super.withOutputTypesForDeserialization(); 48 | } 49 | 50 | @SuppressWarnings("unchecked") 51 | @Override 52 | public void flatMap(KeyedStateRow row, Collector> out) throws Exception { 53 | if (!stateName.equals(row.getStateName())) { 54 | return; 55 | } 56 | 57 | byte[] keyAndNamespaceBytes = row.getKeyAndNamespaceBytes(); 58 | byte[] valueBytes = row.getValueBytes(); 59 | 60 | K key; 61 | MK mapKey; 62 | try (ByteArrayInputStreamWithPos keyIs = new ByteArrayInputStreamWithPos(keyAndNamespaceBytes)) { 63 | DataInputViewStreamWrapper iw = new DataInputViewStreamWrapper(keyIs); 64 | iw.skipBytesToRead(keygroupPrefixBytes); 65 | key = RocksDBUtils.readKey(keyDeserializer, keyIs, iw, false); 66 | VoidNamespaceSerializer.INSTANCE.deserialize(iw); 67 | mapKey = (MK) mapKeySerializer.deserialize(iw); 68 | } 69 | 70 | V value = null; 71 | try (ByteArrayInputStreamWithPos valIs = new ByteArrayInputStreamWithPos(valueBytes)) { 72 | DataInputViewStreamWrapper iw = new DataInputViewStreamWrapper(valIs); 73 | if (!iw.readBoolean()) { 74 | skipTimestampIfTtlEnabled(iw); 75 | value = valueDeserializer.deserialize(iw); 76 | } 77 | } 78 | if (value == null) { 79 | throw new RuntimeException("MapStates with null values are not supported at the moment."); 80 | } 81 | out.collect(Tuple3.of(key, mapKey, value)); 82 | } 83 | 84 | } 85 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/reader/MapStateValueReader.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.core.memory.ByteArrayInputStreamWithPos; 22 | import org.apache.flink.core.memory.DataInputViewStreamWrapper; 23 | import org.apache.flink.util.Collector; 24 | 25 | import com.king.bravo.types.KeyedStateRow; 26 | 27 | public class MapStateValueReader extends AbstractMapStateReader { 28 | 29 | private static final long serialVersionUID = 1L; 30 | 31 | public MapStateValueReader(String stateName, TypeInformation outValueType) { 32 | super(stateName, null, outValueType, outValueType); 33 | } 34 | 35 | @Override 36 | public void flatMap(KeyedStateRow row, Collector out) throws Exception { 37 | if (!stateName.equals(row.getStateName())) { 38 | return; 39 | } 40 | 41 | byte[] valueBytes = row.getValueBytes(); 42 | 43 | V value = null; 44 | try (ByteArrayInputStreamWithPos valIs = new ByteArrayInputStreamWithPos(valueBytes)) { 45 | DataInputViewStreamWrapper iw = new DataInputViewStreamWrapper(valIs); 46 | if (!iw.readBoolean()) { 47 | skipTimestampIfTtlEnabled(iw); 48 | value = valueDeserializer.deserialize(iw); 49 | } 50 | } 51 | if (value == null) { 52 | throw new RuntimeException("MapStates with null values are not supported at the moment."); 53 | } else { 54 | out.collect(value); 55 | } 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/reader/OperatorStateReader.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 java.io.Serializable; 21 | import java.util.ArrayList; 22 | import java.util.Collection; 23 | import java.util.HashSet; 24 | import java.util.List; 25 | import java.util.Map; 26 | import java.util.Set; 27 | 28 | import org.apache.flink.api.common.ExecutionConfig; 29 | import org.apache.flink.api.common.functions.FilterFunction; 30 | import org.apache.flink.api.common.state.ListState; 31 | import org.apache.flink.api.common.typeutils.TypeSerializer; 32 | import org.apache.flink.api.java.DataSet; 33 | import org.apache.flink.api.java.ExecutionEnvironment; 34 | import org.apache.flink.api.java.tuple.Tuple1; 35 | import org.apache.flink.api.java.typeutils.runtime.TupleSerializerBase; 36 | import org.apache.flink.runtime.checkpoint.OperatorState; 37 | import org.apache.flink.runtime.checkpoint.StateObjectCollection; 38 | import org.apache.flink.runtime.checkpoint.savepoint.Savepoint; 39 | import org.apache.flink.runtime.state.DefaultOperatorStateBackend; 40 | import org.apache.flink.runtime.state.KeyedBackendSerializationProxy; 41 | import org.apache.flink.runtime.state.OperatorStateBackend; 42 | import org.apache.flink.runtime.state.OperatorStateHandle; 43 | import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.collect.Maps; 44 | 45 | import com.king.bravo.reader.inputformat.RocksDBKeyedStateInputFormat; 46 | import com.king.bravo.types.KeyedStateRow; 47 | import com.king.bravo.utils.StateMetadataUtils; 48 | import com.king.bravo.writer.OperatorStateWriter; 49 | 50 | /** 51 | * Utility for reading the states stored in a Flink operator into DataSets. 52 | * There are methods for reading both keyed and non-keyed states. 53 | * 54 | */ 55 | public class OperatorStateReader { 56 | 57 | private final OperatorState opState; 58 | private final FilterFunction keyedStateFilter; 59 | private final ExecutionEnvironment env; 60 | 61 | private final HashSet readStates = new HashSet<>(); 62 | 63 | private DataSet allKeyedStateRows; 64 | 65 | private OperatorStateReader(ExecutionEnvironment env, OperatorState opState, 66 | FilterFunction keyedStateFilter) { 67 | this.env = env; 68 | this.opState = opState; 69 | this.keyedStateFilter = keyedStateFilter; 70 | } 71 | 72 | public OperatorStateReader(ExecutionEnvironment env, Savepoint sp, String uid) { 73 | this(env, sp, uid, s -> true); 74 | } 75 | 76 | public OperatorStateReader(ExecutionEnvironment env, Savepoint sp, String uid, Collection stateNames) { 77 | this(env, sp, uid, new FilterFunction() { 78 | private static final long serialVersionUID = 1L; 79 | HashSet filtered = new HashSet<>(stateNames); 80 | 81 | @Override 82 | public boolean filter(String s) throws Exception { 83 | return filtered.contains(s); 84 | } 85 | }); 86 | } 87 | 88 | public OperatorStateReader(ExecutionEnvironment env, Savepoint sp, String uid, 89 | FilterFunction keyedStateFilter) { 90 | this(env, StateMetadataUtils.getOperatorState(sp, uid), keyedStateFilter); 91 | } 92 | 93 | /** 94 | * Read keyed states using the provided reader for further processing 95 | * 96 | * @return The DataSet containing the deseralized state elements 97 | */ 98 | public DataSet readKeyedStates(KeyedStateReader reader) throws Exception { 99 | readKeyedStates(); 100 | KeyedBackendSerializationProxy proxy = StateMetadataUtils.getKeyedBackendSerializationProxy(opState) 101 | .orElseThrow(() -> new IllegalStateException("Cannot read state of a stateless operator.")); 102 | reader.configure(opState.getMaxParallelism(), getKeySerializer(proxy), 103 | StateMetadataUtils.getSerializer(proxy, reader.getStateName()) 104 | .orElseThrow(() -> new IllegalArgumentException("Cannot find state " + reader.getStateName()))); 105 | DataSet parsedState = allKeyedStateRows.flatMap(reader); 106 | readStates.add(reader.getStateName()); 107 | return parsedState; 108 | } 109 | 110 | private TypeSerializer getKeySerializer(KeyedBackendSerializationProxy proxy) { 111 | TypeSerializer keySerializer = proxy.getKeySerializerConfigSnapshot().restoreSerializer(); 112 | if (keySerializer instanceof TupleSerializerBase) { 113 | TupleSerializerBase ts = (TupleSerializerBase) keySerializer; 114 | if (ts.getTupleClass().equals(Tuple1.class)) { 115 | return ts.getFieldSerializers()[0]; 116 | } 117 | } 118 | return keySerializer; 119 | } 120 | 121 | /** 122 | * @return DataSet containing all keyed states of the operator in a 123 | * serialized form 124 | */ 125 | public DataSet getAllKeyedStateRows() { 126 | readKeyedStates(); 127 | return allKeyedStateRows; 128 | } 129 | 130 | /** 131 | * Return all the keyed state rows that were not accessed using a reader. 132 | * This is a convenience method so we can union the untouched part of the 133 | * state with the changed parts before writing them back using the 134 | * {@link OperatorStateWriter}. 135 | */ 136 | public DataSet getAllUnreadKeyedStateRows() { 137 | readKeyedStates(); 138 | HashSet parsed = new HashSet<>(readStates); 139 | return allKeyedStateRows.filter(row -> !parsed.contains(row.f0)); 140 | } 141 | 142 | /** 143 | * Get the serialized keyed state rows for the specified state names 144 | * 145 | * @param stateNames 146 | * @return The dataset of the rows 147 | */ 148 | public DataSet getKeyedStateRows(Set stateNames) { 149 | readKeyedStates(); 150 | HashSet filtered = new HashSet<>(stateNames); 151 | return allKeyedStateRows.filter(row -> filtered.contains(row.f0)); 152 | } 153 | 154 | private void readKeyedStates() { 155 | if (allKeyedStateRows == null) { 156 | allKeyedStateRows = env.createInput(new RocksDBKeyedStateInputFormat(opState, keyedStateFilter)); 157 | } 158 | } 159 | 160 | /** 161 | * Restores the OperatorStateBackend corresponding to the given subtask. The 162 | * backend is completely restored in-memory. 163 | */ 164 | public OperatorStateBackend createOperatorStateBackendFromSnapshot(int subtask) throws Exception { 165 | return restoreOperatorStateBackend(opState.getState(subtask).getManagedOperatorState()); 166 | } 167 | 168 | /** 169 | * Read the serializableListState stored in the checkpoint for the given 170 | * operator subtask 171 | */ 172 | public List getSerializableListState(int subtask) throws Exception { 173 | OperatorStateBackend backend = createOperatorStateBackendFromSnapshot(subtask); 174 | @SuppressWarnings("deprecation") 175 | ListState listState = backend 176 | .getSerializableListState(DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME); 177 | 178 | List list = new ArrayList<>(); 179 | 180 | for (Serializable serializable : listState.get()) { 181 | list.add(serializable); 182 | } 183 | 184 | return list; 185 | } 186 | 187 | /** 188 | * Restores the OperatorStateBackends corresponding to the different 189 | * subtasks. The backends are completely restored in-memory. 190 | */ 191 | public Map createOperatorStateBackendsFromSnapshot() throws Exception { 192 | return Maps.transformValues(opState.getSubtaskStates(), 193 | sst -> { 194 | try { 195 | return restoreOperatorStateBackend(sst.getManagedOperatorState()); 196 | } catch (Exception e) { 197 | throw new RuntimeException(e); 198 | } 199 | }); 200 | } 201 | 202 | public static OperatorStateBackend restoreOperatorStateBackend( 203 | StateObjectCollection managedOpState) 204 | throws Exception { 205 | 206 | DefaultOperatorStateBackend stateBackend = new DefaultOperatorStateBackend( 207 | OperatorStateReader.class.getClassLoader(), new ExecutionConfig(), false); 208 | 209 | stateBackend.restore(managedOpState); 210 | return stateBackend; 211 | } 212 | } 213 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/reader/ValueStateKVReader.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.java.tuple.Tuple2; 22 | import org.apache.flink.api.java.typeutils.TupleTypeInfo; 23 | import org.apache.flink.core.memory.ByteArrayInputStreamWithPos; 24 | import org.apache.flink.core.memory.DataInputViewStreamWrapper; 25 | import org.apache.flink.util.Collector; 26 | 27 | import com.king.bravo.types.KeyedStateRow; 28 | import com.king.bravo.utils.RocksDBUtils; 29 | 30 | public class ValueStateKVReader extends KeyedStateReader> { 31 | 32 | private static final long serialVersionUID = 1L; 33 | 34 | @SuppressWarnings({ "rawtypes", "unchecked" }) 35 | public ValueStateKVReader(String stateName, TypeInformation outKeyType, TypeInformation outValueType) { 36 | super(stateName, outKeyType, outValueType, new TupleTypeInfo(Tuple2.class, outKeyType, outValueType)); 37 | } 38 | 39 | @Override 40 | public void flatMap(KeyedStateRow row, Collector> out) throws Exception { 41 | if (!stateName.equals(row.getStateName())) { 42 | return; 43 | } 44 | 45 | byte[] keyAndNamespaceBytes = row.getKeyAndNamespaceBytes(); 46 | byte[] valueBytes = row.getValueBytes(); 47 | 48 | K key; 49 | try (ByteArrayInputStreamWithPos keyIs = new ByteArrayInputStreamWithPos(keyAndNamespaceBytes)) { 50 | DataInputViewStreamWrapper iw = new DataInputViewStreamWrapper(keyIs); 51 | iw.skipBytesToRead(keygroupPrefixBytes); 52 | key = RocksDBUtils.readKey(keyDeserializer, keyIs, iw, false); 53 | } 54 | 55 | V value = null; 56 | try (ByteArrayInputStreamWithPos valIs = new ByteArrayInputStreamWithPos(valueBytes)) { 57 | DataInputViewStreamWrapper iw = new DataInputViewStreamWrapper(valIs); 58 | skipTimestampIfTtlEnabled(iw); 59 | value = valueDeserializer.deserialize(iw); 60 | } 61 | out.collect(Tuple2.of(key, value)); 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/reader/ValueStateValueReader.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.core.memory.ByteArrayInputStreamWithPos; 22 | import org.apache.flink.core.memory.DataInputViewStreamWrapper; 23 | import org.apache.flink.util.Collector; 24 | 25 | import com.king.bravo.types.KeyedStateRow; 26 | 27 | public class ValueStateValueReader extends KeyedStateReader { 28 | 29 | private static final long serialVersionUID = 1L; 30 | 31 | public ValueStateValueReader(String stateName, TypeInformation outValueType) { 32 | super(stateName, null, outValueType, outValueType); 33 | } 34 | 35 | @Override 36 | public void flatMap(KeyedStateRow row, Collector out) throws Exception { 37 | if (!stateName.equals(row.getStateName())) { 38 | return; 39 | } 40 | 41 | byte[] valueBytes = row.getValueBytes(); 42 | 43 | V value = null; 44 | try (ByteArrayInputStreamWithPos valIs = new ByteArrayInputStreamWithPos(valueBytes)) { 45 | DataInputViewStreamWrapper iw = new DataInputViewStreamWrapper(valIs); 46 | skipTimestampIfTtlEnabled(iw); 47 | value = valueDeserializer.deserialize(iw); 48 | } 49 | if (value == null) { 50 | throw new RuntimeException("MapStates with null values are not supported at the moment."); 51 | } else { 52 | out.collect(value); 53 | } 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/reader/inputformat/KeyedStateInputSplit.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.inputformat; 19 | 20 | import org.apache.flink.core.io.InputSplit; 21 | import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; 22 | 23 | @SuppressWarnings("serial") 24 | public class KeyedStateInputSplit implements InputSplit { 25 | 26 | private final int splitNumber; 27 | private final OperatorSubtaskState subtaskState; 28 | 29 | public KeyedStateInputSplit(int splitNumber, OperatorSubtaskState subtaskState) { 30 | this.splitNumber = splitNumber; 31 | this.subtaskState = subtaskState; 32 | } 33 | 34 | @Override 35 | public int getSplitNumber() { 36 | return splitNumber; 37 | } 38 | 39 | public OperatorSubtaskState getOperatorSubtaskState() { 40 | return subtaskState; 41 | } 42 | 43 | @Override 44 | public String toString() { 45 | return new StringBuilder("OperatorStateInputSplit [") 46 | .append("splitNumber=").append(splitNumber) 47 | .append(", subtaskState=").append(subtaskState) 48 | .append("]") 49 | .toString(); 50 | } 51 | 52 | } 53 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/reader/inputformat/RocksDBCheckpointIterator.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.inputformat; 19 | 20 | import java.io.Closeable; 21 | import java.io.File; 22 | import java.io.IOException; 23 | import java.util.ArrayList; 24 | import java.util.HashMap; 25 | import java.util.Iterator; 26 | import java.util.LinkedList; 27 | import java.util.List; 28 | import java.util.Map; 29 | import java.util.Map.Entry; 30 | 31 | import org.apache.flink.api.common.functions.FilterFunction; 32 | import org.apache.flink.configuration.ConfigConstants; 33 | import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend; 34 | import org.apache.flink.contrib.streaming.state.RocksIteratorWrapper; 35 | import org.apache.flink.core.fs.CloseableRegistry; 36 | import org.apache.flink.core.fs.FSDataInputStream; 37 | import org.apache.flink.core.fs.FSDataOutputStream; 38 | import org.apache.flink.core.fs.FileSystem; 39 | import org.apache.flink.core.fs.Path; 40 | import org.apache.flink.runtime.state.IncrementalKeyedStateHandle; 41 | import org.apache.flink.runtime.state.StateHandleID; 42 | import org.apache.flink.runtime.state.StreamStateHandle; 43 | import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot; 44 | import org.apache.flink.util.FileUtils; 45 | import org.apache.flink.util.IOUtils; 46 | import org.apache.flink.util.Preconditions; 47 | import org.rocksdb.ColumnFamilyDescriptor; 48 | import org.rocksdb.ColumnFamilyHandle; 49 | import org.rocksdb.ColumnFamilyOptions; 50 | import org.rocksdb.DBOptions; 51 | import org.rocksdb.RocksDB; 52 | import org.rocksdb.RocksDBException; 53 | 54 | import com.king.bravo.types.KeyedStateRow; 55 | import com.king.bravo.utils.StateMetadataUtils; 56 | 57 | public class RocksDBCheckpointIterator implements Iterator, Closeable, Iterable { 58 | 59 | private final DBOptions dbOptions = new DBOptions(); 60 | private final ColumnFamilyOptions colOptions = new ColumnFamilyOptions() 61 | .setMergeOperatorName(RocksDBKeyedStateBackend.MERGE_OPERATOR_NAME); 62 | 63 | private final RocksDB db; 64 | private LinkedList> iteratorQueue; 65 | 66 | private String currentName; 67 | private RocksIteratorWrapper currentIterator; 68 | 69 | private ArrayList stateColumnFamilyHandles; 70 | private CloseableRegistry cancelStreamRegistry; 71 | private String localPath; 72 | 73 | public RocksDBCheckpointIterator(IncrementalKeyedStateHandle handle, FilterFunction stateFilter, 74 | String localPath) { 75 | this.localPath = localPath; 76 | this.cancelStreamRegistry = new CloseableRegistry(); 77 | List stateMetaInfoSnapshots = StateMetadataUtils 78 | .getKeyedBackendSerializationProxy(handle.getMetaStateHandle()).getStateMetaInfoSnapshots(); 79 | 80 | stateColumnFamilyHandles = new ArrayList<>(stateMetaInfoSnapshots.size() + 1); 81 | List stateColumnFamilyDescriptors = createAndRegisterColumnFamilyDescriptors( 82 | stateMetaInfoSnapshots); 83 | try { 84 | transferAllStateDataToDirectory(handle, new Path(localPath)); 85 | this.db = openDB(localPath, stateColumnFamilyDescriptors, stateColumnFamilyHandles); 86 | createColumnIterators(stateFilter, stateMetaInfoSnapshots); 87 | } catch (Exception e) { 88 | throw new IllegalStateException(e); 89 | } 90 | } 91 | 92 | private void transferAllStateDataToDirectory( 93 | IncrementalKeyedStateHandle restoreStateHandle, 94 | Path dest) throws IOException { 95 | 96 | final Map sstFiles = restoreStateHandle.getSharedState(); 97 | final Map miscFiles = restoreStateHandle.getPrivateState(); 98 | 99 | transferAllDataFromStateHandles(sstFiles, dest); 100 | transferAllDataFromStateHandles(miscFiles, dest); 101 | } 102 | 103 | private void transferAllDataFromStateHandles( 104 | Map stateHandleMap, 105 | Path restoreInstancePath) throws IOException { 106 | 107 | for (Map.Entry entry : stateHandleMap.entrySet()) { 108 | StateHandleID stateHandleID = entry.getKey(); 109 | StreamStateHandle remoteFileHandle = entry.getValue(); 110 | copyStateDataHandleData(new Path(restoreInstancePath, stateHandleID.toString()), remoteFileHandle); 111 | } 112 | } 113 | 114 | private void copyStateDataHandleData( 115 | Path restoreFilePath, 116 | StreamStateHandle remoteFileHandle) throws IOException { 117 | 118 | FileSystem restoreFileSystem = restoreFilePath.getFileSystem(); 119 | 120 | FSDataInputStream inputStream = null; 121 | FSDataOutputStream outputStream = null; 122 | 123 | try { 124 | inputStream = remoteFileHandle.openInputStream(); 125 | cancelStreamRegistry.registerCloseable(inputStream); 126 | 127 | outputStream = restoreFileSystem.create(restoreFilePath, FileSystem.WriteMode.OVERWRITE); 128 | cancelStreamRegistry.registerCloseable(outputStream); 129 | 130 | byte[] buffer = new byte[8 * 1024]; 131 | while (true) { 132 | int numBytes = inputStream.read(buffer); 133 | if (numBytes == -1) { 134 | break; 135 | } 136 | 137 | outputStream.write(buffer, 0, numBytes); 138 | } 139 | } finally { 140 | if (cancelStreamRegistry.unregisterCloseable(inputStream)) { 141 | inputStream.close(); 142 | } 143 | 144 | if (cancelStreamRegistry.unregisterCloseable(outputStream)) { 145 | outputStream.close(); 146 | } 147 | } 148 | } 149 | 150 | private void createColumnIterators(FilterFunction stateFilter, 151 | List stateMetaInfoSnapshots) 152 | throws Exception { 153 | Map iterators = new HashMap<>(); 154 | for (int i = 0; i < stateMetaInfoSnapshots.size(); i++) { 155 | String name = stateMetaInfoSnapshots.get(i).getName(); 156 | if (stateFilter.filter(name)) { 157 | RocksIteratorWrapper iterator = new RocksIteratorWrapper( 158 | this.db.newIterator(stateColumnFamilyHandles.get(i + 1))); 159 | iterators.put(name, iterator); 160 | iterator.seekToFirst(); 161 | } 162 | } 163 | 164 | iteratorQueue = new LinkedList<>(iterators.entrySet()); 165 | updateCurrentIterator(); 166 | } 167 | 168 | private void updateCurrentIterator() { 169 | IOUtils.closeQuietly(currentIterator); 170 | if (iteratorQueue.isEmpty()) { 171 | currentIterator = null; 172 | currentName = null; 173 | return; 174 | } else { 175 | Entry e = iteratorQueue.pop(); 176 | currentName = e.getKey(); 177 | currentIterator = e.getValue(); 178 | } 179 | } 180 | 181 | private List createAndRegisterColumnFamilyDescriptors( 182 | List stateMetaInfoSnapshots) { 183 | 184 | List columnFamilyDescriptors = new ArrayList<>(stateMetaInfoSnapshots.size()); 185 | 186 | for (StateMetaInfoSnapshot stateMetaInfoSnapshot : stateMetaInfoSnapshots) { 187 | ColumnFamilyDescriptor columnFamilyDescriptor = new ColumnFamilyDescriptor( 188 | stateMetaInfoSnapshot.getName().getBytes(ConfigConstants.DEFAULT_CHARSET), 189 | colOptions); 190 | 191 | columnFamilyDescriptors.add(columnFamilyDescriptor); 192 | } 193 | return columnFamilyDescriptors; 194 | } 195 | 196 | private RocksDB openDB(String path, List stateColumnFamilyDescriptors, 197 | List stateColumnFamilyHandles) throws IOException { 198 | 199 | List columnFamilyDescriptors = new ArrayList<>(1 + stateColumnFamilyDescriptors.size()); 200 | 201 | // we add the required descriptor for the default CF in FIRST position, 202 | // see 203 | // https://github.com/facebook/rocksdb/wiki/RocksJava-Basics#opening-a-database-with-column-families 204 | columnFamilyDescriptors.add(new ColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY, colOptions)); 205 | columnFamilyDescriptors.addAll(stateColumnFamilyDescriptors); 206 | 207 | try { 208 | return RocksDB.open( 209 | dbOptions, 210 | Preconditions.checkNotNull(path), 211 | columnFamilyDescriptors, 212 | stateColumnFamilyHandles); 213 | } catch (RocksDBException e) { 214 | throw new IOException("Error while opening RocksDB instance.", e); 215 | } 216 | } 217 | 218 | @Override 219 | public final boolean hasNext() { 220 | if (currentIterator == null) { 221 | return false; 222 | } 223 | 224 | if (currentIterator.isValid()) { 225 | return true; 226 | } else { 227 | updateCurrentIterator(); 228 | return hasNext(); 229 | } 230 | } 231 | 232 | @Override 233 | public final KeyedStateRow next() { 234 | byte[] key = currentIterator.key(); 235 | byte[] value = currentIterator.value(); 236 | currentIterator.next(); 237 | return new KeyedStateRow(currentName, key, value); 238 | } 239 | 240 | @Override 241 | public void close() throws IOException { 242 | IOUtils.closeQuietly(cancelStreamRegistry); 243 | IOUtils.closeAllQuietly(stateColumnFamilyHandles); 244 | IOUtils.closeQuietly(db); 245 | IOUtils.closeQuietly(dbOptions); 246 | IOUtils.closeQuietly(colOptions); 247 | FileUtils.deleteDirectoryQuietly(new File(localPath)); 248 | } 249 | 250 | @Override 251 | public Iterator iterator() { 252 | return this; 253 | } 254 | } 255 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/reader/inputformat/RocksDBKeyedStateInputFormat.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.inputformat; 19 | 20 | import java.io.File; 21 | import java.io.IOException; 22 | import java.util.ArrayList; 23 | import java.util.Iterator; 24 | import java.util.List; 25 | import java.util.UUID; 26 | import java.util.stream.StreamSupport; 27 | 28 | import org.apache.flink.api.common.functions.FilterFunction; 29 | import org.apache.flink.api.common.io.DefaultInputSplitAssigner; 30 | import org.apache.flink.api.common.io.RichInputFormat; 31 | import org.apache.flink.api.common.io.statistics.BaseStatistics; 32 | import org.apache.flink.configuration.Configuration; 33 | import org.apache.flink.core.io.InputSplitAssigner; 34 | import org.apache.flink.runtime.checkpoint.OperatorState; 35 | import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; 36 | import org.apache.flink.runtime.state.IncrementalKeyedStateHandle; 37 | import org.apache.flink.runtime.state.KeyGroupsStateHandle; 38 | import org.apache.flink.util.IOUtils; 39 | 40 | import com.king.bravo.types.KeyedStateRow; 41 | 42 | /** 43 | * InputFormat for reading all {@link KeyedStateRow} for the specified state ids 44 | * (or all if empty) from an {@link OperatorState}. 45 | * 46 | * Right now the input splits are created by subtaskstate, this could be 47 | * improved to be split by keygroup in the future 48 | */ 49 | public class RocksDBKeyedStateInputFormat extends RichInputFormat { 50 | 51 | private static final long serialVersionUID = 1L; 52 | private final OperatorState operatorState; 53 | 54 | private transient Iterator mergeIterator; 55 | 56 | private FilterFunction stateFilter; 57 | private List iterators = new ArrayList<>(); 58 | 59 | public RocksDBKeyedStateInputFormat(OperatorState operatorState) { 60 | this(operatorState, i -> true); 61 | } 62 | 63 | public RocksDBKeyedStateInputFormat(OperatorState operatorState, FilterFunction stateFilter) { 64 | this.operatorState = operatorState; 65 | this.stateFilter = stateFilter; 66 | } 67 | 68 | @Override 69 | public void open(KeyedStateInputSplit split) throws IOException { 70 | IOManagerAsync iomanager = new IOManagerAsync(); 71 | String[] spillingDirectoriesPaths = iomanager.getSpillingDirectoriesPaths(); 72 | 73 | mergeIterator = split.getOperatorSubtaskState() 74 | .getManagedKeyedState() 75 | .stream() 76 | .map(keyedStateHandle -> { 77 | if (keyedStateHandle instanceof IncrementalKeyedStateHandle) { 78 | File localDir = new File(spillingDirectoriesPaths[0], "rocksdb_" + UUID.randomUUID()); 79 | if (!localDir.mkdirs()) { 80 | throw new RuntimeException("Could not create " + localDir); 81 | } 82 | RocksDBCheckpointIterator iterator = new RocksDBCheckpointIterator( 83 | (IncrementalKeyedStateHandle) keyedStateHandle, 84 | stateFilter, localDir.getAbsolutePath()); 85 | iterators.add(iterator); 86 | return iterator; 87 | } 88 | 89 | if (!(keyedStateHandle instanceof KeyGroupsStateHandle)) { 90 | throw new IllegalStateException("Unexpected state handle type, " + 91 | "expected: " + KeyGroupsStateHandle.class + 92 | ", but found: " + keyedStateHandle.getClass()); 93 | } else { 94 | RocksDBSavepointIterator iterator = new RocksDBSavepointIterator( 95 | (KeyGroupsStateHandle) keyedStateHandle, stateFilter); 96 | iterators.add(iterator); 97 | return iterator; 98 | } 99 | }).flatMap(it -> StreamSupport.stream(it.spliterator(), false)).iterator(); 100 | } 101 | 102 | @Override 103 | public boolean reachedEnd() throws IOException { 104 | return !mergeIterator.hasNext(); 105 | } 106 | 107 | @Override 108 | public KeyedStateRow nextRecord(KeyedStateRow reuse) throws IOException { 109 | return mergeIterator.next(); 110 | } 111 | 112 | @Override 113 | public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException { 114 | return cachedStatistics; 115 | } 116 | 117 | @Override 118 | public KeyedStateInputSplit[] createInputSplits(int minNumSplits) throws IOException { 119 | return operatorState.getSubtaskStates().entrySet().stream() 120 | .map(entry -> new KeyedStateInputSplit(entry.getKey(), entry.getValue())) 121 | .toArray(KeyedStateInputSplit[]::new); 122 | } 123 | 124 | @Override 125 | public InputSplitAssigner getInputSplitAssigner(KeyedStateInputSplit[] inputSplits) { 126 | return new DefaultInputSplitAssigner(inputSplits); 127 | } 128 | 129 | @Override 130 | public void close() throws IOException { 131 | IOUtils.closeAllQuietly(iterators); 132 | iterators.clear(); 133 | } 134 | 135 | @Override 136 | public void configure(Configuration parameters) {} 137 | } 138 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/reader/inputformat/RocksDBSavepointIterator.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.inputformat; 19 | 20 | import static com.king.bravo.utils.KeyGroupFlags.END_OF_KEY_GROUP_MARK; 21 | import static com.king.bravo.utils.KeyGroupFlags.clearMetaDataFollowsFlag; 22 | import static com.king.bravo.utils.KeyGroupFlags.hasMetaDataFollowsFlag; 23 | 24 | import java.io.Closeable; 25 | import java.io.IOException; 26 | import java.io.InputStream; 27 | import java.util.Iterator; 28 | import java.util.Map; 29 | 30 | import org.apache.flink.api.common.functions.FilterFunction; 31 | import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer; 32 | import org.apache.flink.core.fs.FSDataInputStream; 33 | import org.apache.flink.core.memory.DataInputViewStreamWrapper; 34 | import org.apache.flink.runtime.state.KeyGroupRangeOffsets; 35 | import org.apache.flink.runtime.state.KeyGroupsStateHandle; 36 | import org.apache.flink.runtime.state.KeyedBackendSerializationProxy; 37 | import org.apache.flink.runtime.state.KeyedStateHandle; 38 | import org.apache.flink.runtime.state.StreamCompressionDecorator; 39 | import org.slf4j.Logger; 40 | import org.slf4j.LoggerFactory; 41 | 42 | import com.king.bravo.types.KeyedStateRow; 43 | import com.king.bravo.utils.StateMetadataUtils; 44 | 45 | /** 46 | * Iterator over the raw keyed state stored in a RocksDB full snapshot 47 | * {@link KeyedStateHandle}. The logic is more or less extracted/replicated from 48 | * the RocksDBFullRestoreOperation inner class. 49 | * 50 | * Optionally supports specifying a set of state ids to read, in this case 51 | * others are ignored. 52 | * 53 | */ 54 | public class RocksDBSavepointIterator implements Iterator, Closeable, Iterable { 55 | 56 | private static final Logger LOGGER = LoggerFactory.getLogger(RocksDBSavepointIterator.class); 57 | 58 | private final KeyGroupsStateHandle keyGroupsStateHandle; 59 | 60 | private FSDataInputStream stateHandleInStream; 61 | private Iterator offsetsIt; 62 | private DataInputViewStreamWrapper compressedInputView; 63 | private Map stateIdMapping; 64 | 65 | private boolean hasNext = true; 66 | private int stateId; 67 | 68 | private String stateName; 69 | 70 | private final FilterFunction stateFilter; 71 | 72 | public RocksDBSavepointIterator(KeyGroupsStateHandle keyedStateHandle, FilterFunction stateFilter) { 73 | this.stateFilter = stateFilter; 74 | this.keyGroupsStateHandle = keyedStateHandle; 75 | } 76 | 77 | @Override 78 | public final boolean hasNext() { 79 | return hasNext; 80 | } 81 | 82 | @Override 83 | public final KeyedStateRow next() { 84 | try { 85 | // TODO reduce GC pressure 86 | // return nextRecord(reuse); 87 | return nextRecord(new KeyedStateRow()); 88 | } catch (Exception e) { 89 | throw new RuntimeException(e); 90 | } 91 | } 92 | 93 | @Override 94 | public void close() throws IOException { 95 | if (stateHandleInStream != null) { 96 | LOGGER.debug("Closing {}", this.keyGroupsStateHandle.getDelegateStateHandle()); 97 | stateHandleInStream.close(); 98 | } 99 | } 100 | 101 | private final KeyedStateRow nextRecord(KeyedStateRow reuse) throws Exception { 102 | if (!openIfNeeded()) { 103 | return null; 104 | } 105 | 106 | byte[] key = BytePrimitiveArraySerializer.INSTANCE.deserialize(compressedInputView); 107 | byte[] value = BytePrimitiveArraySerializer.INSTANCE.deserialize(compressedInputView); 108 | 109 | reuse.f0 = stateName; 110 | reuse.f1 = key; 111 | reuse.f2 = value; 112 | 113 | if (hasMetaDataFollowsFlag(reuse.f1)) { 114 | clearMetaDataFollowsFlag(reuse.f1); 115 | 116 | seekNextStateId(true); 117 | while (stateId == END_OF_KEY_GROUP_MARK && hasNext) { 118 | hasNext = seekNextOffset(); 119 | if (hasNext) { 120 | seekNextStateId(false); 121 | } 122 | } 123 | } 124 | 125 | LOGGER.trace("{}", reuse); 126 | 127 | return reuse; 128 | } 129 | 130 | private boolean openIfNeeded() throws Exception { 131 | if (stateHandleInStream == null) { 132 | LOGGER.debug("Opening {}", keyGroupsStateHandle.getDelegateStateHandle()); 133 | stateHandleInStream = keyGroupsStateHandle.openInputStream(); 134 | 135 | final KeyedBackendSerializationProxy serializationProxy = StateMetadataUtils 136 | .getKeyedBackendSerializationProxy(keyGroupsStateHandle); 137 | 138 | this.stateIdMapping = StateMetadataUtils.getStateIdMapping(serializationProxy); 139 | final StreamCompressionDecorator streamCompressionDecorator = StateMetadataUtils 140 | .getCompressionDecorator(serializationProxy); 141 | 142 | final KeyGroupRangeOffsets rangeOffsets = keyGroupsStateHandle.getGroupRangeOffsets(); 143 | LOGGER.debug("{}", rangeOffsets); 144 | 145 | offsetsIt = new ValidOffsetsIterator(rangeOffsets); 146 | 147 | hasNext = seekNextOffset(); 148 | 149 | if (hasNext) { 150 | final InputStream compressedInputStream = streamCompressionDecorator 151 | .decorateWithCompression(stateHandleInStream); 152 | compressedInputView = new DataInputViewStreamWrapper(compressedInputStream); 153 | seekNextStateId(false); 154 | } 155 | } 156 | 157 | return hasNext; 158 | } 159 | 160 | private boolean seekNextOffset() throws IOException { 161 | final boolean hasNext = offsetsIt.hasNext(); 162 | if (hasNext) { 163 | final long offset = offsetsIt.next(); 164 | LOGGER.debug("Seeking offset {}", offset); 165 | stateHandleInStream.seek(offset); 166 | } 167 | return hasNext; 168 | } 169 | 170 | private void seekNextStateId(boolean metaFollows) throws Exception { 171 | 172 | int stateId = compressedInputView.readShort(); 173 | String stateName = stateIdMapping.get(stateId); 174 | if (metaFollows) { 175 | stateId = END_OF_KEY_GROUP_MARK & stateId; 176 | } 177 | 178 | while (!stateFilter.filter(stateName) && stateId != END_OF_KEY_GROUP_MARK) { 179 | 180 | final int keySize = compressedInputView.readInt(); 181 | final byte keyByte0 = compressedInputView.readByte(); 182 | compressedInputView.skip(keySize - 1); 183 | 184 | final int valueSize = compressedInputView.readInt(); 185 | compressedInputView.skip(valueSize); 186 | 187 | if (hasMetaDataFollowsFlag(keyByte0)) { 188 | stateId = END_OF_KEY_GROUP_MARK & compressedInputView.readShort(); 189 | stateName = stateIdMapping.get(stateId); 190 | } 191 | } 192 | 193 | this.stateId = stateId; 194 | this.stateName = stateName; 195 | } 196 | 197 | @Override 198 | public Iterator iterator() { 199 | return this; 200 | } 201 | } 202 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/reader/inputformat/ValidOffsetsIterator.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.inputformat; 19 | 20 | import java.util.Iterator; 21 | import java.util.stream.StreamSupport; 22 | 23 | import org.apache.flink.runtime.state.KeyGroupRangeOffsets; 24 | 25 | public class ValidOffsetsIterator implements Iterator { 26 | 27 | private final Iterator delegate; 28 | 29 | public ValidOffsetsIterator(KeyGroupRangeOffsets keyGroupRangeOffsets) { 30 | delegate = StreamSupport.stream(keyGroupRangeOffsets.spliterator(), false) 31 | .map(tuple -> tuple.f1) 32 | .filter(offset -> offset > 0) 33 | .iterator(); 34 | } 35 | 36 | @Override 37 | public boolean hasNext() { 38 | return delegate.hasNext(); 39 | } 40 | 41 | @Override 42 | public Long next() { 43 | return delegate.next(); 44 | } 45 | 46 | } 47 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/types/KeyedStateRow.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.types; 19 | 20 | import java.io.ByteArrayInputStream; 21 | import java.io.IOException; 22 | 23 | import org.apache.flink.api.java.tuple.Tuple2; 24 | import org.apache.flink.api.java.tuple.Tuple3; 25 | import org.apache.flink.core.memory.DataInputViewStreamWrapper; 26 | import org.apache.flink.runtime.state.KeyGroupRangeAssignment; 27 | 28 | import com.king.bravo.utils.RocksDBUtils; 29 | 30 | /** 31 | * Raw state row containing minimal data necessary for the RocksDB state backend 32 | * to write it back. 33 | * 34 | */ 35 | public class KeyedStateRow extends Tuple3 { 36 | 37 | private static final long serialVersionUID = 1L; 38 | 39 | public KeyedStateRow() { 40 | super(); 41 | } 42 | 43 | public KeyedStateRow(String stateName, byte[] keyAndNamespaceBytes, byte[] valueBytes) { 44 | super(stateName, keyAndNamespaceBytes, valueBytes); 45 | } 46 | 47 | public String getStateName() { 48 | return f0; 49 | } 50 | 51 | public byte[] getKeyAndNamespaceBytes() { 52 | return f1; 53 | } 54 | 55 | public byte[] getValueBytes() { 56 | return f2; 57 | } 58 | 59 | public int getKeyGroup(int maxParallelism) throws IOException { 60 | try (ByteArrayInputStream is = new ByteArrayInputStream(getKeyAndNamespaceBytes())) { 61 | return RocksDBUtils.readKeyGroup(1, new DataInputViewStreamWrapper(is)); 62 | } 63 | } 64 | 65 | public Integer getOperatorIndex(int maxParallelism, int parallelism) throws Exception { 66 | return KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup(maxParallelism, parallelism, 67 | getKeyGroup(maxParallelism)); 68 | } 69 | 70 | public Tuple2 getKeyGroupAndStateName(int maxParallelism) throws IOException { 71 | return Tuple2.of(getKeyGroup(maxParallelism), getStateName()); 72 | } 73 | } 74 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/utils/Identifiers.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.utils; 19 | 20 | import static java.nio.charset.StandardCharsets.UTF_8; 21 | 22 | import org.apache.flink.runtime.jobgraph.OperatorID; 23 | 24 | import com.google.common.hash.HashFunction; 25 | import com.google.common.hash.Hashing; 26 | 27 | public final class Identifiers { 28 | 29 | private static final HashFunction HASH_FN = Hashing.murmur3_128(0); 30 | 31 | public static OperatorID operatorId(String id) { 32 | return new OperatorID(hash(id)); 33 | } 34 | 35 | private static final byte[] hash(String id) { 36 | return HASH_FN.newHasher().putString(id, UTF_8).hash().asBytes(); 37 | } 38 | 39 | } 40 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/utils/KeyGroupFlags.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.utils; 19 | 20 | /** 21 | * Utility class that should go away once ported to the refactored rocksdb keyed 22 | * backend in Flink 1.7 23 | */ 24 | public final class KeyGroupFlags { 25 | 26 | public static final int FIRST_BIT_IN_BYTE_MASK = 0x80; 27 | public static final int END_OF_KEY_GROUP_MARK = 0xFFFF; 28 | 29 | public static void clearMetaDataFollowsFlag(byte[] key) { 30 | key[0] &= (~FIRST_BIT_IN_BYTE_MASK); 31 | } 32 | 33 | public static boolean hasMetaDataFollowsFlag(byte[] key) { 34 | return hasMetaDataFollowsFlag(key[0]); 35 | } 36 | 37 | public static boolean hasMetaDataFollowsFlag(final byte keyByte0) { 38 | return 0 != (keyByte0 & FIRST_BIT_IN_BYTE_MASK); 39 | } 40 | 41 | public static void setMetaDataFollowsFlagInKey(byte[] key) { 42 | key[0] |= FIRST_BIT_IN_BYTE_MASK; 43 | } 44 | 45 | } 46 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/utils/RocksDBUtils.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 | package com.king.bravo.utils; 18 | 19 | import java.io.IOException; 20 | import java.util.ArrayList; 21 | import java.util.List; 22 | 23 | import org.apache.flink.api.common.typeutils.TypeSerializer; 24 | import org.apache.flink.core.memory.ByteArrayInputStreamWithPos; 25 | import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos; 26 | import org.apache.flink.core.memory.DataInputView; 27 | import org.apache.flink.core.memory.DataInputViewStreamWrapper; 28 | import org.apache.flink.core.memory.DataOutputView; 29 | 30 | /** 31 | * Utils for RocksDB state serialization and deserialization. 32 | */ 33 | public class RocksDBUtils { 34 | 35 | public static int readKeyGroup(int keyGroupPrefixBytes, DataInputView inputView) throws IOException { 36 | int keyGroup = 0; 37 | for (int i = 0; i < keyGroupPrefixBytes; ++i) { 38 | keyGroup <<= 8; 39 | keyGroup |= (inputView.readByte() & 0xFF); 40 | } 41 | return keyGroup; 42 | } 43 | 44 | public static K readKey( 45 | TypeSerializer keySerializer, 46 | ByteArrayInputStreamWithPos inputStream, 47 | DataInputView inputView, 48 | boolean ambiguousKeyPossible) throws IOException { 49 | int beforeRead = inputStream.getPosition(); 50 | K key = keySerializer.deserialize(inputView); 51 | if (ambiguousKeyPossible) { 52 | int length = inputStream.getPosition() - beforeRead; 53 | readVariableIntBytes(inputView, length); 54 | } 55 | return key; 56 | } 57 | 58 | public static N readNamespace( 59 | TypeSerializer namespaceSerializer, 60 | ByteArrayInputStreamWithPos inputStream, 61 | DataInputView inputView, 62 | boolean ambiguousKeyPossible) throws IOException { 63 | int beforeRead = inputStream.getPosition(); 64 | N namespace = namespaceSerializer.deserialize(inputView); 65 | if (ambiguousKeyPossible) { 66 | int length = inputStream.getPosition() - beforeRead; 67 | readVariableIntBytes(inputView, length); 68 | } 69 | return namespace; 70 | } 71 | 72 | public static void writeNameSpace( 73 | N namespace, 74 | TypeSerializer namespaceSerializer, 75 | ByteArrayOutputStreamWithPos keySerializationStream, 76 | DataOutputView keySerializationDataOutputView, 77 | boolean ambiguousKeyPossible) throws IOException { 78 | 79 | int beforeWrite = keySerializationStream.getPosition(); 80 | namespaceSerializer.serialize(namespace, keySerializationDataOutputView); 81 | 82 | if (ambiguousKeyPossible) { 83 | // write length of namespace 84 | writeLengthFrom(beforeWrite, keySerializationStream, 85 | keySerializationDataOutputView); 86 | } 87 | } 88 | 89 | public static boolean isAmbiguousKeyPossible(TypeSerializer keySerializer, 90 | TypeSerializer namespaceSerializer) { 91 | return (keySerializer.getLength() < 0) && (namespaceSerializer.getLength() < 0); 92 | } 93 | 94 | public static void writeKeyGroup( 95 | int keyGroup, 96 | int keyGroupPrefixBytes, 97 | DataOutputView keySerializationDateDataOutputView) throws IOException { 98 | for (int i = keyGroupPrefixBytes; --i >= 0;) { 99 | keySerializationDateDataOutputView.writeByte(extractByteAtPosition(keyGroup, i)); 100 | } 101 | } 102 | 103 | public static void writeKey( 104 | K key, 105 | TypeSerializer keySerializer, 106 | ByteArrayOutputStreamWithPos keySerializationStream, 107 | DataOutputView keySerializationDataOutputView, 108 | boolean ambiguousKeyPossible) throws IOException { 109 | // write key 110 | int beforeWrite = keySerializationStream.getPosition(); 111 | keySerializer.serialize(key, keySerializationDataOutputView); 112 | 113 | if (ambiguousKeyPossible) { 114 | // write size of key 115 | writeLengthFrom(beforeWrite, keySerializationStream, 116 | keySerializationDataOutputView); 117 | } 118 | } 119 | 120 | private static void readVariableIntBytes(DataInputView inputView, int value) throws IOException { 121 | do { 122 | inputView.readByte(); 123 | value >>>= 8; 124 | } while (value != 0); 125 | } 126 | 127 | private static void writeLengthFrom( 128 | int fromPosition, 129 | ByteArrayOutputStreamWithPos keySerializationStream, 130 | DataOutputView keySerializationDateDataOutputView) throws IOException { 131 | int length = keySerializationStream.getPosition() - fromPosition; 132 | writeVariableIntBytes(length, keySerializationDateDataOutputView); 133 | } 134 | 135 | private static void writeVariableIntBytes( 136 | int value, 137 | DataOutputView keySerializationDateDataOutputView) 138 | throws IOException { 139 | do { 140 | keySerializationDateDataOutputView.writeByte(value); 141 | value >>>= 8; 142 | } while (value != 0); 143 | } 144 | 145 | public static void serializeKeyGroup(int keyGroup, byte[] startKeyGroupPrefixBytes) { 146 | final int keyGroupPrefixBytes = startKeyGroupPrefixBytes.length; 147 | for (int j = 0; j < keyGroupPrefixBytes; ++j) { 148 | startKeyGroupPrefixBytes[j] = extractByteAtPosition(keyGroup, keyGroupPrefixBytes - j - 1); 149 | } 150 | } 151 | 152 | private static byte extractByteAtPosition(int value, int byteIdx) { 153 | return (byte) ((value >>> (byteIdx << 3))); 154 | } 155 | 156 | public static int computeRequiredBytesInKeyGroupPrefix(int totalKeyGroupsInJob) { 157 | return totalKeyGroupsInJob > (Byte.MAX_VALUE + 1) ? 2 : 1; 158 | } 159 | 160 | public static List deserializeList( 161 | byte[] valueBytes, TypeSerializer elementSerializer, boolean ttl) throws IOException { 162 | if (valueBytes == null) { 163 | return null; 164 | } 165 | 166 | 167 | ByteArrayInputStreamWithPos is = new ByteArrayInputStreamWithPos(valueBytes); 168 | DataInputViewStreamWrapper in = new DataInputViewStreamWrapper(is); 169 | 170 | List result = new ArrayList<>(); 171 | V next; 172 | while ((next = deserializeNextElement(in, elementSerializer, ttl)) != null) { 173 | result.add(next); 174 | } 175 | return result; 176 | } 177 | 178 | public static V deserializeNextElement( 179 | DataInputViewStreamWrapper in, TypeSerializer elementSerializer, boolean ttl) throws IOException { 180 | if (in.available() > 0) { 181 | if (ttl) { 182 | in.skipBytesToRead(Long.BYTES); 183 | } 184 | V element = elementSerializer.deserialize(in); 185 | if (in.available() > 0) { 186 | in.readByte(); 187 | } 188 | System.err.println(element); 189 | return element; 190 | } 191 | return null; 192 | } 193 | } 194 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/utils/StateMetadataUtils.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.utils; 19 | 20 | import org.apache.flink.api.common.typeutils.CompositeSerializer; 21 | import org.apache.flink.api.common.typeutils.TypeSerializer; 22 | import org.apache.flink.core.fs.FSDataInputStream; 23 | import org.apache.flink.core.fs.FileSystem.WriteMode; 24 | import org.apache.flink.core.fs.Path; 25 | import org.apache.flink.core.memory.DataInputViewStreamWrapper; 26 | import org.apache.flink.runtime.checkpoint.Checkpoints; 27 | import org.apache.flink.runtime.checkpoint.OperatorState; 28 | import org.apache.flink.runtime.checkpoint.savepoint.Savepoint; 29 | import org.apache.flink.runtime.checkpoint.savepoint.SavepointV2; 30 | import org.apache.flink.runtime.jobgraph.OperatorID; 31 | import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; 32 | import org.apache.flink.runtime.state.IncrementalKeyedStateHandle; 33 | import org.apache.flink.runtime.state.KeyedBackendSerializationProxy; 34 | import org.apache.flink.runtime.state.KeyedStateHandle; 35 | import org.apache.flink.runtime.state.SnappyStreamCompressionDecorator; 36 | import org.apache.flink.runtime.state.StreamCompressionDecorator; 37 | import org.apache.flink.runtime.state.StreamStateHandle; 38 | import org.apache.flink.runtime.state.UncompressedStreamCompressionDecorator; 39 | import org.apache.flink.runtime.state.filesystem.AbstractFsCheckpointStorage; 40 | import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot; 41 | import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot.CommonSerializerKeys; 42 | 43 | import java.io.DataInputStream; 44 | import java.io.IOException; 45 | import java.lang.reflect.Field; 46 | import java.lang.reflect.Method; 47 | import java.util.Arrays; 48 | import java.util.Collection; 49 | import java.util.HashMap; 50 | import java.util.Map; 51 | import java.util.Optional; 52 | import java.util.stream.Collectors; 53 | 54 | public class StateMetadataUtils { 55 | 56 | /** 57 | * Load the Savepoint metadata object from the given path 58 | */ 59 | public static Savepoint loadSavepoint(String checkpointPointer) throws IOException { 60 | try { 61 | Method resolveCheckpointPointer = AbstractFsCheckpointStorage.class.getDeclaredMethod( 62 | "resolveCheckpointPointer", 63 | String.class); 64 | resolveCheckpointPointer.setAccessible(true); 65 | CompletedCheckpointStorageLocation loc = (CompletedCheckpointStorageLocation) resolveCheckpointPointer 66 | .invoke(null, checkpointPointer); 67 | 68 | return Checkpoints.loadCheckpointMetadata(new DataInputStream(loc.getMetadataHandle().openInputStream()), 69 | StateMetadataUtils.class.getClassLoader()); 70 | } catch (Exception e) { 71 | throw new RuntimeException(e); 72 | } 73 | 74 | } 75 | 76 | public static OperatorState getOperatorState(Savepoint savepoint, String uid) { 77 | return getOperatorState(savepoint, Identifiers.operatorId(uid)); 78 | } 79 | 80 | public static OperatorState getOperatorState(Savepoint savepoint, OperatorID opId) { 81 | return savepoint 82 | .getOperatorStates() 83 | .stream() 84 | .filter(os -> os.getOperatorID().equals(opId)) 85 | .findAny() 86 | .orElseThrow(() -> new RuntimeException("No operator state with id " + opId.toString())); 87 | } 88 | 89 | public static int getKeyGroupPrefixBytes(int maxParallelism) { 90 | return maxParallelism > (Byte.MAX_VALUE + 1) ? 2 : 1; 91 | } 92 | 93 | /** 94 | * Create a new {@link Savepoint} by replacing certain 95 | * {@link OperatorState}s of an old {@link Savepoint} 96 | * 97 | * @param oldSavepoint 98 | * {@link Savepoint} to base the new state on 99 | * @param statesToReplace 100 | * States that will be replaced, all else will be kept 101 | * @return A new valid {@link Savepoint} metadata object. 102 | */ 103 | public static Savepoint createNewSavepoint(Savepoint oldSavepoint, OperatorState... statesToReplace) { 104 | return createNewSavepoint(oldSavepoint, Arrays.asList(statesToReplace)); 105 | } 106 | 107 | /** 108 | * Create a new {@link Savepoint} by replacing certain 109 | * {@link OperatorState}s of an old {@link Savepoint} 110 | * 111 | * @param oldSavepoint 112 | * {@link Savepoint} to base the new state on 113 | * @param statesToReplace 114 | * States that will be replaced, all else will be kept 115 | * @return A new valid {@link Savepoint} metadata object. 116 | */ 117 | public static Savepoint createNewSavepoint(Savepoint oldSavepoint, Collection statesToReplace) { 118 | 119 | Map newStates = oldSavepoint.getOperatorStates().stream() 120 | .collect(Collectors.toMap(OperatorState::getOperatorID, o -> o)); 121 | 122 | statesToReplace.forEach(os -> newStates.put(os.getOperatorID(), os)); 123 | 124 | return new SavepointV2(oldSavepoint.getCheckpointId(), newStates.values(), oldSavepoint.getMasterStates()); 125 | } 126 | 127 | public static Optional> getKeyedBackendSerializationProxy(OperatorState opState) { 128 | try { 129 | KeyedStateHandle firstHandle = opState.getStates().iterator().next().getManagedKeyedState().iterator() 130 | .next(); 131 | if (firstHandle instanceof IncrementalKeyedStateHandle) { 132 | return Optional.of(getKeyedBackendSerializationProxy( 133 | ((IncrementalKeyedStateHandle) firstHandle).getMetaStateHandle())); 134 | } else { 135 | return Optional.of(getKeyedBackendSerializationProxy((StreamStateHandle) firstHandle)); 136 | } 137 | } catch (Exception e) { 138 | return Optional.empty(); 139 | } 140 | } 141 | 142 | public static StreamCompressionDecorator getCompressionDecorator(KeyedBackendSerializationProxy proxy) { 143 | return proxy.isUsingKeyGroupCompression() 144 | ? SnappyStreamCompressionDecorator.INSTANCE 145 | : UncompressedStreamCompressionDecorator.INSTANCE; 146 | } 147 | 148 | @SuppressWarnings("unchecked") 149 | public static Optional> getSerializer(KeyedBackendSerializationProxy proxy, 150 | String stateName) { 151 | 152 | for (StateMetaInfoSnapshot snapshot : proxy.getStateMetaInfoSnapshots()) { 153 | if (snapshot.getName().equals(stateName)) { 154 | return Optional 155 | .of((TypeSerializer) snapshot 156 | .getTypeSerializerSnapshot(CommonSerializerKeys.VALUE_SERIALIZER) 157 | .restoreSerializer()); 158 | } 159 | } 160 | 161 | return Optional.empty(); 162 | } 163 | 164 | public static Map getStateIdMapping(KeyedBackendSerializationProxy proxy) { 165 | Map stateIdMapping = new HashMap<>(); 166 | 167 | int stateId = 0; 168 | for (StateMetaInfoSnapshot snapshot : proxy.getStateMetaInfoSnapshots()) { 169 | stateIdMapping.put(stateId, snapshot.getName()); 170 | stateId++; 171 | } 172 | 173 | return stateIdMapping; 174 | } 175 | 176 | public static KeyedBackendSerializationProxy getKeyedBackendSerializationProxy( 177 | StreamStateHandle streamStateHandle) { 178 | KeyedBackendSerializationProxy serializationProxy = new KeyedBackendSerializationProxy<>( 179 | StateMetadataUtils.class.getClassLoader()); 180 | try (FSDataInputStream is = streamStateHandle.openInputStream()) { 181 | DataInputViewStreamWrapper iw = new DataInputViewStreamWrapper(is); 182 | serializationProxy.read(iw); 183 | return serializationProxy; 184 | } catch (IOException e) { 185 | throw new RuntimeException(e); 186 | } 187 | } 188 | 189 | public static Path writeSavepointMetadata(Path newCheckpointBasePath, Savepoint savepoint) throws IOException { 190 | Path p = new Path(newCheckpointBasePath, AbstractFsCheckpointStorage.METADATA_FILE_NAME); 191 | Checkpoints.storeCheckpointMetadata(savepoint, 192 | newCheckpointBasePath.getFileSystem().create(p, WriteMode.NO_OVERWRITE)); 193 | return p; 194 | } 195 | 196 | public static boolean isTtlState(TypeSerializer valueSerializer) { 197 | boolean ttlSerializer = valueSerializer.getClass().getName() 198 | .startsWith("org.apache.flink.runtime.state.ttl.TtlStateFactory$TtlSerializer"); 199 | return ttlSerializer; 200 | } 201 | 202 | public static TypeSerializer unwrapTtlSerializer(TypeSerializer valueSerializer) throws Exception { 203 | Field f = CompositeSerializer.class.getDeclaredField("fieldSerializers"); 204 | f.setAccessible(true); 205 | return (TypeSerializer) ((TypeSerializer[]) f.get(valueSerializer))[1]; 206 | } 207 | 208 | } 209 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/writer/functions/KeyGroupAndStateNameKey.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.writer.functions; 19 | 20 | import org.apache.flink.api.java.functions.KeySelector; 21 | import org.apache.flink.api.java.tuple.Tuple2; 22 | 23 | import com.king.bravo.types.KeyedStateRow; 24 | 25 | public class KeyGroupAndStateNameKey 26 | implements KeySelector> { 27 | 28 | private static final long serialVersionUID = 1L; 29 | private int maxParallelism; 30 | 31 | public KeyGroupAndStateNameKey(int maxParallelism) { 32 | this.maxParallelism = maxParallelism; 33 | } 34 | 35 | @Override 36 | public Tuple2 getKey(KeyedStateRow row) throws Exception { 37 | return row.getKeyGroupAndStateName(maxParallelism); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/writer/functions/OperatorIndexForKeyGroupKey.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.writer.functions; 19 | 20 | import org.apache.flink.api.java.functions.KeySelector; 21 | 22 | import com.king.bravo.types.KeyedStateRow; 23 | 24 | public class OperatorIndexForKeyGroupKey implements KeySelector { 25 | 26 | private static final long serialVersionUID = 1L; 27 | 28 | private final int maxParallelism; 29 | private final int parallelism; 30 | 31 | public OperatorIndexForKeyGroupKey(int maxParallelism, int parallelism) { 32 | this.maxParallelism = maxParallelism; 33 | this.parallelism = parallelism; 34 | } 35 | 36 | @Override 37 | public Integer getKey(KeyedStateRow state) throws Exception { 38 | return state.getOperatorIndex(maxParallelism, parallelism); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/writer/functions/RocksDBSavepointWriter.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.writer.functions; 19 | 20 | import static com.king.bravo.utils.KeyGroupFlags.END_OF_KEY_GROUP_MARK; 21 | import static com.king.bravo.utils.KeyGroupFlags.setMetaDataFollowsFlagInKey; 22 | 23 | import java.io.OutputStream; 24 | import java.util.Map; 25 | import java.util.UUID; 26 | 27 | import org.apache.flink.api.common.functions.RichGroupReduceFunction; 28 | import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer; 29 | import org.apache.flink.api.java.tuple.Tuple2; 30 | import org.apache.flink.core.fs.FSDataOutputStream; 31 | import org.apache.flink.core.fs.FileSystem.WriteMode; 32 | import org.apache.flink.core.fs.Path; 33 | import org.apache.flink.core.memory.DataOutputView; 34 | import org.apache.flink.core.memory.DataOutputViewStreamWrapper; 35 | import org.apache.flink.runtime.state.KeyGroupRangeAssignment; 36 | import org.apache.flink.runtime.state.KeyGroupRangeOffsets; 37 | import org.apache.flink.runtime.state.KeyGroupsStateHandle; 38 | import org.apache.flink.runtime.state.KeyedStateHandle; 39 | import org.apache.flink.runtime.state.SnappyStreamCompressionDecorator; 40 | import org.apache.flink.runtime.state.StreamCompressionDecorator; 41 | import org.apache.flink.runtime.state.UncompressedStreamCompressionDecorator; 42 | import org.apache.flink.runtime.state.filesystem.FileStateHandle; 43 | import org.apache.flink.util.Collector; 44 | import org.slf4j.Logger; 45 | import org.slf4j.LoggerFactory; 46 | 47 | import com.king.bravo.types.KeyedStateRow; 48 | 49 | public class RocksDBSavepointWriter extends RichGroupReduceFunction> { 50 | 51 | private static final long serialVersionUID = 1L; 52 | 53 | private static final Logger LOGGER = LoggerFactory.getLogger(RocksDBSavepointWriter.class); 54 | 55 | private StreamCompressionDecorator keyGroupCompressionDecorator; 56 | private Map stateIdMapping; 57 | 58 | private final int maxParallelism; 59 | private final int parallelism; 60 | private final Path opStateDir; 61 | private byte[] metaBytes; 62 | 63 | public RocksDBSavepointWriter(int maxParallelism, int parallelism, Map stateIdMapping, 64 | boolean compression, Path opStateDir, byte[] metaBytes) { 65 | 66 | this.maxParallelism = maxParallelism; 67 | this.parallelism = parallelism; 68 | this.stateIdMapping = stateIdMapping; 69 | this.opStateDir = opStateDir; 70 | this.metaBytes = metaBytes; 71 | this.keyGroupCompressionDecorator = compression ? SnappyStreamCompressionDecorator.INSTANCE 72 | : UncompressedStreamCompressionDecorator.INSTANCE; 73 | } 74 | 75 | @Override 76 | public void reduce(Iterable values, 77 | Collector> out) 78 | throws Exception { 79 | 80 | Path checkpointFilePath = new Path(opStateDir, String.valueOf(UUID.randomUUID())); 81 | FSDataOutputStream checkpointFileStream = null; 82 | 83 | KeyGroupRangeOffsets keyGroupRangeOffsets = null; 84 | int opIndex = -1; 85 | 86 | DataOutputView kgOutView = null; 87 | OutputStream kgOutStream = null; 88 | 89 | try { 90 | 91 | byte[] previousKey = null; 92 | byte[] previousValue = null; 93 | int previousKeyGroup = -1; 94 | int previousStateId = -1; 95 | 96 | java.util.Iterator iterator = values.iterator(); 97 | 98 | KeyedStateRow nextRow; 99 | if (iterator.hasNext()) { 100 | nextRow = iterator.next(); 101 | previousKeyGroup = nextRow.getKeyGroup(maxParallelism); 102 | opIndex = nextRow.getOperatorIndex(maxParallelism, parallelism); 103 | previousStateId = stateIdMapping.get(nextRow.getStateName()); 104 | 105 | LOGGER.info("Writing to {}", checkpointFilePath); 106 | 107 | checkpointFileStream = checkpointFilePath.getFileSystem().create(checkpointFilePath, 108 | WriteMode.NO_OVERWRITE); 109 | 110 | // Write rocks keyed state metadata 111 | checkpointFileStream.write(metaBytes); 112 | 113 | keyGroupRangeOffsets = new KeyGroupRangeOffsets( 114 | KeyGroupRangeAssignment.computeKeyGroupRangeForOperatorIndex(maxParallelism, 115 | parallelism, opIndex)); 116 | 117 | // begin first key-group by recording the offset 118 | keyGroupRangeOffsets.setKeyGroupOffset( 119 | previousKeyGroup, 120 | checkpointFileStream.getPos()); 121 | 122 | kgOutStream = keyGroupCompressionDecorator.decorateWithCompression(checkpointFileStream); 123 | kgOutView = new DataOutputViewStreamWrapper(kgOutStream); 124 | kgOutView.writeShort(previousStateId); 125 | 126 | previousKey = nextRow.getKeyAndNamespaceBytes(); 127 | previousValue = nextRow.getValueBytes(); 128 | } 129 | 130 | // main loop: write k/v pairs ordered by (key-group, kv-state), thereby tracking 131 | // key-group offsets. 132 | while (iterator.hasNext()) { 133 | nextRow = iterator.next(); 134 | 135 | // set signal in first key byte that meta data will follow in the stream after 136 | // this k/v pair 137 | int nextKeygroup = nextRow.getKeyGroup(maxParallelism); 138 | if (nextKeygroup != previousKeyGroup || stateIdMapping.get(nextRow.getStateName()) != previousStateId) { 139 | setMetaDataFollowsFlagInKey(previousKey); 140 | } 141 | 142 | BytePrimitiveArraySerializer.INSTANCE.serialize(previousKey, kgOutView); 143 | BytePrimitiveArraySerializer.INSTANCE.serialize(previousValue, kgOutView); 144 | 145 | // write meta data if we have to 146 | if (nextKeygroup != previousKeyGroup) { 147 | kgOutView.writeShort(END_OF_KEY_GROUP_MARK); 148 | // this will just close the outer stream 149 | kgOutStream.close(); 150 | // begin new key-group 151 | keyGroupRangeOffsets.setKeyGroupOffset( 152 | nextRow.getKeyGroup(maxParallelism), 153 | checkpointFileStream.getPos()); 154 | 155 | kgOutStream = keyGroupCompressionDecorator.decorateWithCompression(checkpointFileStream); 156 | kgOutView = new DataOutputViewStreamWrapper(kgOutStream); 157 | kgOutView.writeShort(stateIdMapping.get(nextRow.getStateName())); 158 | } else if (stateIdMapping.get(nextRow.getStateName()) != previousStateId) { 159 | kgOutView.writeShort(stateIdMapping.get(nextRow.getStateName())); 160 | } 161 | 162 | previousKeyGroup = nextKeygroup; 163 | previousStateId = stateIdMapping.get(nextRow.getStateName()); 164 | previousKey = nextRow.getKeyAndNamespaceBytes(); 165 | previousValue = nextRow.getValueBytes(); 166 | } 167 | 168 | if (previousKey != null) { 169 | setMetaDataFollowsFlagInKey(previousKey); 170 | BytePrimitiveArraySerializer.INSTANCE.serialize(previousKey, kgOutView); 171 | BytePrimitiveArraySerializer.INSTANCE.serialize(previousValue, kgOutView); 172 | kgOutView.writeShort(END_OF_KEY_GROUP_MARK); 173 | kgOutStream.close(); 174 | kgOutStream = null; 175 | } 176 | } finally { 177 | if (checkpointFileStream != null) { 178 | checkpointFileStream.close(); 179 | } 180 | } 181 | out.collect(Tuple2.of(opIndex, 182 | new KeyGroupsStateHandle(keyGroupRangeOffsets, new FileStateHandle(checkpointFilePath, 0)))); 183 | } 184 | } 185 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/writer/functions/RowFilter.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.writer.functions; 19 | 20 | import java.util.HashSet; 21 | import java.util.Set; 22 | 23 | import org.apache.flink.api.common.functions.FilterFunction; 24 | 25 | import com.king.bravo.types.KeyedStateRow; 26 | 27 | public final class RowFilter implements FilterFunction { 28 | private static final long serialVersionUID = 1L; 29 | private final Set states; 30 | 31 | public RowFilter(Set states) { 32 | this.states = new HashSet<>(states); 33 | } 34 | 35 | @Override 36 | public boolean filter(KeyedStateRow row) throws Exception { 37 | return states.contains(row.getStateName()); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /bravo/src/main/java/com/king/bravo/writer/functions/ValueStateToKeyedStateRow.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.writer.functions; 19 | 20 | import org.apache.flink.api.common.functions.MapFunction; 21 | import org.apache.flink.api.common.typeutils.TypeSerializer; 22 | import org.apache.flink.api.java.tuple.Tuple2; 23 | import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos; 24 | import org.apache.flink.core.memory.DataOutputViewStreamWrapper; 25 | import org.apache.flink.runtime.state.KeyGroupRangeAssignment; 26 | import org.apache.flink.runtime.state.VoidNamespace; 27 | import org.apache.flink.runtime.state.VoidNamespaceSerializer; 28 | import org.apache.flink.util.InstantiationUtil; 29 | 30 | import com.king.bravo.types.KeyedStateRow; 31 | import com.king.bravo.utils.RocksDBUtils; 32 | import com.king.bravo.utils.StateMetadataUtils; 33 | 34 | public class ValueStateToKeyedStateRow implements MapFunction, KeyedStateRow> { 35 | 36 | private static final long serialVersionUID = 1L; 37 | private final int maxParallelism; 38 | private final TypeSerializer keySerializer; 39 | private final TypeSerializer valueSerializer; 40 | private final int keygroupPrefixBytes; 41 | private final String stateName; 42 | 43 | public ValueStateToKeyedStateRow(String stateName, TypeSerializer keySerializer, 44 | TypeSerializer valueSerializer, 45 | int maxParallelism) { 46 | this.stateName = stateName; 47 | this.maxParallelism = maxParallelism; 48 | keygroupPrefixBytes = StateMetadataUtils.getKeyGroupPrefixBytes(maxParallelism); 49 | this.keySerializer = keySerializer; 50 | this.valueSerializer = valueSerializer; 51 | } 52 | 53 | @Override 54 | public KeyedStateRow map(Tuple2 t) throws Exception { 55 | int keyGroup = KeyGroupRangeAssignment.assignToKeyGroup(t.f0, maxParallelism); 56 | ByteArrayOutputStreamWithPos os = new ByteArrayOutputStreamWithPos(); 57 | DataOutputViewStreamWrapper ov = new DataOutputViewStreamWrapper(os); 58 | 59 | RocksDBUtils.writeKeyGroup(keyGroup, keygroupPrefixBytes, ov); 60 | RocksDBUtils.writeKey(t.f0, keySerializer, os, ov, false); 61 | RocksDBUtils.writeNameSpace(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, os, 62 | ov, false); 63 | 64 | os.close(); 65 | return new KeyedStateRow(stateName, os.toByteArray(), 66 | InstantiationUtil.serializeToByteArray(valueSerializer, t.f1)); 67 | } 68 | } 69 | -------------------------------------------------------------------------------- /bravo/src/test/java/com/king/bravo/BroadcastStateTransformationTest.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; 19 | 20 | import static org.junit.Assert.assertEquals; 21 | 22 | import java.io.IOException; 23 | import java.util.ArrayList; 24 | import java.util.List; 25 | import java.util.Optional; 26 | 27 | import org.apache.flink.api.common.state.BroadcastState; 28 | import org.apache.flink.api.common.state.MapStateDescriptor; 29 | import org.apache.flink.api.common.typeinfo.BasicTypeInfo; 30 | import org.apache.flink.api.java.ExecutionEnvironment; 31 | import org.apache.flink.api.java.typeutils.ListTypeInfo; 32 | import org.apache.flink.core.fs.Path; 33 | import org.apache.flink.runtime.checkpoint.savepoint.Savepoint; 34 | import org.apache.flink.runtime.state.OperatorStateBackend; 35 | import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; 36 | import org.apache.flink.streaming.api.datastream.BroadcastStream; 37 | import org.apache.flink.streaming.api.datastream.DataStream; 38 | import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; 39 | import org.apache.flink.streaming.api.functions.ProcessFunction; 40 | import org.apache.flink.streaming.api.functions.co.KeyedBroadcastProcessFunction; 41 | import org.apache.flink.util.Collector; 42 | import org.apache.flink.util.OutputTag; 43 | import org.junit.Test; 44 | 45 | import com.king.bravo.reader.OperatorStateReader; 46 | import com.king.bravo.testing.BravoTestPipeline; 47 | import com.king.bravo.utils.StateMetadataUtils; 48 | import com.king.bravo.writer.OperatorStateWriter; 49 | 50 | public class BroadcastStateTransformationTest extends BravoTestPipeline { 51 | 52 | private static final long serialVersionUID = 1L; 53 | 54 | static MapStateDescriptor> bcstate = new MapStateDescriptor<>("filteredKeys", 55 | BasicTypeInfo.BOOLEAN_TYPE_INFO, new ListTypeInfo<>(Integer.class)); 56 | 57 | @Test 58 | public void test() throws Exception { 59 | process("process 1"); 60 | sleep(500); 61 | process("filter 1"); 62 | sleep(1000); 63 | process("process 1"); 64 | process("process 2"); 65 | process("process 3"); 66 | sleep(1000); 67 | cancelJob(); 68 | List output = runTestPipeline(this::constructTestPipeline); 69 | assertEquals(3, output.size()); 70 | 71 | process("process 1"); 72 | process("process 2"); 73 | process("process 3"); 74 | triggerSavepoint(); 75 | 76 | System.err.println("Round two"); 77 | List output2 = restoreTestPipelineFromSnapshot( 78 | validateStateAndTransform(getLastCheckpoint()).getPath(), 79 | this::constructTestPipeline); 80 | 81 | Savepoint savepoint = getLastSavepoint(); 82 | 83 | validateSecondSavepoint(savepoint); 84 | assertEquals(4, output2.size()); 85 | } 86 | 87 | private Path validateStateAndTransform(Savepoint savepoint) throws IOException, Exception { 88 | ExecutionEnvironment environment = ExecutionEnvironment.createLocalEnvironment(); 89 | 90 | // Validate the contents of the broadcast state 91 | OperatorStateReader reader = new OperatorStateReader(environment, savepoint, "stateful"); 92 | OperatorStateBackend backend = reader.createOperatorStateBackendFromSnapshot(1); 93 | BroadcastState> state = backend.getBroadcastState(bcstate); 94 | assertEquals(Lists.newArrayList(1), state.get(true)); 95 | 96 | Path newCheckpointBasePath = new Path(getCheckpointDir(), "new"); 97 | OperatorStateWriter writer = new OperatorStateWriter(savepoint, "stateful", newCheckpointBasePath); 98 | writer.transformNonKeyedState((i, b) -> { 99 | try { 100 | b.getBroadcastState(bcstate).put(true, Lists.newArrayList(2, 3)); 101 | } catch (Exception e) { 102 | throw new RuntimeException(e); 103 | } 104 | }); 105 | 106 | StateMetadataUtils.writeSavepointMetadata(newCheckpointBasePath, 107 | StateMetadataUtils.createNewSavepoint(savepoint, writer.writeAll())); 108 | return newCheckpointBasePath; 109 | } 110 | 111 | private void validateSecondSavepoint(Savepoint savepoint) throws Exception { 112 | // Validate the contents of the broadcast state 113 | OperatorStateReader reader = new OperatorStateReader(ExecutionEnvironment.createLocalEnvironment(), savepoint, 114 | "stateful"); 115 | OperatorStateBackend backend = reader.createOperatorStateBackendFromSnapshot(1); 116 | BroadcastState> state = backend.getBroadcastState(bcstate); 117 | assertEquals(Lists.newArrayList(2, 3), state.get(true)); 118 | } 119 | 120 | public DataStream constructTestPipeline(DataStream source) { 121 | 122 | OutputTag filtered = new OutputTag<>("filter", BasicTypeInfo.INT_TYPE_INFO); 123 | OutputTag process = new OutputTag<>("process", BasicTypeInfo.INT_TYPE_INFO); 124 | 125 | SingleOutputStreamOperator input = source.process(new ProcessFunction() { 126 | private static final long serialVersionUID = 1L; 127 | 128 | @Override 129 | public void processElement(String s, Context ctx, 130 | Collector out) throws Exception { 131 | 132 | if (s.startsWith("filter ")) { 133 | ctx.output(filtered, Integer.parseInt(s.substring(7))); 134 | } else if (s.startsWith("process ")) { 135 | ctx.output(process, Integer.parseInt(s.substring(8))); 136 | } else { 137 | throw new RuntimeException("oOoO"); 138 | } 139 | 140 | } 141 | }); 142 | 143 | BroadcastStream broadcast = input.getSideOutput(filtered).broadcast(bcstate); 144 | 145 | return input.getSideOutput(process).keyBy(i -> i).connect(broadcast).process(new BroadcastProcessor(bcstate)) 146 | .uid("stateful"); 147 | } 148 | 149 | public static class BroadcastProcessor extends KeyedBroadcastProcessFunction { 150 | 151 | private static final long serialVersionUID = 7317800376639115920L; 152 | private MapStateDescriptor> desc; 153 | 154 | public BroadcastProcessor(MapStateDescriptor> desc) { 155 | this.desc = desc; 156 | } 157 | 158 | @Override 159 | public void processElement(Integer i, ReadOnlyContext ctx, Collector out) throws Exception { 160 | List filtered = Optional.ofNullable(ctx.getBroadcastState(desc).get(true)) 161 | .orElse(new ArrayList<>()); 162 | if (!filtered.contains(i)) { 163 | out.collect(i.toString()); 164 | } 165 | } 166 | 167 | @Override 168 | public void processBroadcastElement(Integer i, Context ctx, Collector out) throws Exception { 169 | List keys = ctx.getBroadcastState(desc).get(true); 170 | if (keys == null) { 171 | keys = new ArrayList<>(); 172 | } 173 | if (!keys.contains(i)) { 174 | keys.add(i); 175 | ctx.getBroadcastState(desc).put(true, keys); 176 | } 177 | } 178 | } 179 | 180 | } 181 | -------------------------------------------------------------------------------- /bravo/src/test/java/com/king/bravo/KeyedStateAddRemoveTest.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; 19 | 20 | import static org.junit.Assert.assertEquals; 21 | 22 | import java.io.IOException; 23 | import java.util.HashSet; 24 | import java.util.List; 25 | import java.util.Optional; 26 | 27 | import org.apache.flink.api.common.functions.RichFilterFunction; 28 | import org.apache.flink.api.common.functions.RichMapFunction; 29 | import org.apache.flink.api.common.state.ValueState; 30 | import org.apache.flink.api.common.state.ValueStateDescriptor; 31 | import org.apache.flink.api.common.typeutils.base.IntSerializer; 32 | import org.apache.flink.api.java.DataSet; 33 | import org.apache.flink.api.java.ExecutionEnvironment; 34 | import org.apache.flink.api.java.tuple.Tuple2; 35 | import org.apache.flink.configuration.Configuration; 36 | import org.apache.flink.core.fs.Path; 37 | import org.apache.flink.runtime.checkpoint.savepoint.Savepoint; 38 | import org.apache.flink.streaming.api.datastream.DataStream; 39 | import org.junit.Test; 40 | 41 | import com.google.common.collect.Sets; 42 | import com.king.bravo.testing.BravoTestPipeline; 43 | import com.king.bravo.utils.StateMetadataUtils; 44 | import com.king.bravo.writer.OperatorStateWriter; 45 | 46 | public class KeyedStateAddRemoveTest extends BravoTestPipeline { 47 | 48 | private static final long serialVersionUID = 1L; 49 | 50 | @Test 51 | public void test() throws Exception { 52 | process("1"); 53 | process("2"); 54 | // Ignored by the statful filter... 55 | process("2"); 56 | process("1"); 57 | process("2"); 58 | sleep(2000); 59 | triggerSavepoint(); 60 | List output = runTestPipeline(this::constructTestPipeline); 61 | assertEquals(Sets.newHashSet("(1,0)", "(2,0)"), new HashSet<>(output)); 62 | Path newSavepointPath = transformLastSavepoint(); 63 | 64 | // Filter state is dropped, should process this now 65 | process("1"); 66 | process("2"); 67 | List restoredOutput = restoreTestPipelineFromSnapshot(newSavepointPath.getPath(), 68 | this::restoreTestPipeline); 69 | assertEquals(Sets.newHashSet("(1,0)", "(2,0)", "(1,101)", "(2,101)"), 70 | new HashSet<>(restoredOutput)); 71 | } 72 | 73 | private Path transformLastSavepoint() throws IOException, Exception { 74 | ExecutionEnvironment environment = ExecutionEnvironment.createLocalEnvironment(); 75 | Savepoint savepoint = getLastSavepoint(); 76 | 77 | DataSet> bootstrapState = environment.fromElements(Tuple2.of(1, 100), 78 | Tuple2.of(2, 100)); 79 | 80 | Path newCheckpointBasePath = new Path(getCheckpointDir(), "new"); 81 | 82 | OperatorStateWriter counterStateWriter = new OperatorStateWriter(savepoint, "counter", newCheckpointBasePath); 83 | 84 | counterStateWriter.setKeySerializer(IntSerializer.INSTANCE); 85 | counterStateWriter.createNewValueState("count", bootstrapState, IntSerializer.INSTANCE); 86 | 87 | OperatorStateWriter filterStateWriter = new OperatorStateWriter(savepoint, "filter", newCheckpointBasePath); 88 | filterStateWriter.deleteKeyedState("seen"); 89 | 90 | Savepoint newSavepoint = StateMetadataUtils.createNewSavepoint(savepoint, 91 | filterStateWriter.writeAll(), 92 | counterStateWriter.writeAll()); 93 | StateMetadataUtils.writeSavepointMetadata(newCheckpointBasePath, newSavepoint); 94 | return newCheckpointBasePath; 95 | } 96 | 97 | public DataStream constructTestPipeline(DataStream source) { 98 | return source 99 | .map(Integer::parseInt) 100 | .returns(Integer.class) 101 | .keyBy(i -> i) 102 | .filter(new StatefulFilter()) 103 | .uid("filter") 104 | .keyBy(i -> i) 105 | .map(new StatelessMap()) 106 | .uid("counter") 107 | .map(Tuple2::toString) 108 | .returns(String.class); 109 | } 110 | 111 | public DataStream restoreTestPipeline(DataStream source) { 112 | return source 113 | .map(Integer::parseInt) 114 | .returns(Integer.class) 115 | .keyBy(i -> i) 116 | .filter(new StatefulFilter()) 117 | .uid("filter") 118 | .keyBy(i -> i) 119 | .map(new StatefulCounter()) 120 | .uid("counter") 121 | .map(Tuple2::toString) 122 | .returns(String.class); 123 | } 124 | 125 | public static class StatefulFilter extends RichFilterFunction { 126 | private static final long serialVersionUID = 1L; 127 | private ValueState seen; 128 | 129 | @Override 130 | public boolean filter(Integer value) throws Exception { 131 | if (seen.value() == null) { 132 | seen.update(true); 133 | return true; 134 | } 135 | return false; 136 | } 137 | 138 | @Override 139 | public void open(Configuration parameters) throws Exception { 140 | seen = getRuntimeContext().getState(new ValueStateDescriptor<>("seen", Boolean.class)); 141 | } 142 | } 143 | 144 | public static class StatefulCounter extends RichMapFunction> { 145 | 146 | private static final long serialVersionUID = 7317800376639115920L; 147 | private ValueState count; 148 | 149 | @Override 150 | public void open(Configuration parameters) throws Exception { 151 | count = getRuntimeContext().getState(new ValueStateDescriptor<>("count", Integer.class)); 152 | } 153 | 154 | @Override 155 | public Tuple2 map(Integer value) throws Exception { 156 | count.update(Optional.ofNullable(count.value()).orElse(0) + 1); 157 | return Tuple2.of(value, count.value()); 158 | } 159 | } 160 | 161 | public static class StatelessMap extends RichMapFunction> { 162 | 163 | private static final long serialVersionUID = 7317800376639115920L; 164 | 165 | @Override 166 | public Tuple2 map(Integer value) throws Exception { 167 | return Tuple2.of(value, 0); 168 | } 169 | } 170 | } 171 | -------------------------------------------------------------------------------- /bravo/src/test/java/com/king/bravo/MapStateReadingTest.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; 19 | 20 | import static org.junit.Assert.assertEquals; 21 | 22 | import java.io.IOException; 23 | import java.util.Collections; 24 | import java.util.HashSet; 25 | import java.util.List; 26 | import java.util.Optional; 27 | 28 | import org.apache.flink.api.common.functions.RichMapFunction; 29 | import org.apache.flink.api.common.state.MapState; 30 | import org.apache.flink.api.common.state.MapStateDescriptor; 31 | import org.apache.flink.api.common.typeinfo.BasicTypeInfo; 32 | import org.apache.flink.api.common.typeinfo.TypeHint; 33 | import org.apache.flink.api.java.ExecutionEnvironment; 34 | import org.apache.flink.api.java.tuple.Tuple2; 35 | import org.apache.flink.api.java.tuple.Tuple3; 36 | import org.apache.flink.configuration.Configuration; 37 | import org.apache.flink.runtime.checkpoint.savepoint.Savepoint; 38 | import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.collect.Lists; 39 | import org.apache.flink.streaming.api.datastream.DataStream; 40 | import org.junit.Test; 41 | 42 | import com.google.common.collect.Sets; 43 | import com.king.bravo.reader.KeyedStateReader; 44 | import com.king.bravo.reader.OperatorStateReader; 45 | import com.king.bravo.testing.BravoTestPipeline; 46 | 47 | public class MapStateReadingTest extends BravoTestPipeline { 48 | 49 | private static final long serialVersionUID = 1L; 50 | 51 | @Test 52 | public void test() throws Exception { 53 | process("1,1"); 54 | process("2,3"); 55 | process("1,2"); 56 | process("1,1"); 57 | sleep(1000); 58 | cancelJob(); 59 | runTestPipeline(this::constructTestPipeline); 60 | validateCheckpointedStateReading(); 61 | } 62 | 63 | @SuppressWarnings("unchecked") 64 | private void validateCheckpointedStateReading() throws IOException, Exception { 65 | ExecutionEnvironment environment = ExecutionEnvironment.createLocalEnvironment(); 66 | Savepoint savepoint = getLastCheckpoint(); 67 | OperatorStateReader reader = new OperatorStateReader(environment, savepoint, "hello"); 68 | 69 | List> countState = reader 70 | .readKeyedStates(KeyedStateReader.forMapStateEntries("Count", BasicTypeInfo.INT_TYPE_INFO, 71 | BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO)) 72 | .collect(); 73 | 74 | List mapValues = reader 75 | .readKeyedStates(KeyedStateReader.forMapStateValues("Count", BasicTypeInfo.INT_TYPE_INFO)) 76 | .collect(); 77 | 78 | assertEquals(Sets.newHashSet(Tuple3.of(1, "1", 2), Tuple3.of(1, "2", 1), Tuple3.of(2, "3", 1)), 79 | new HashSet<>(countState)); 80 | 81 | Collections.sort(mapValues); 82 | assertEquals(Lists.newArrayList(1, 1, 2), mapValues); 83 | } 84 | 85 | public DataStream constructTestPipeline(DataStream source) { 86 | return source 87 | .map(s -> { 88 | String[] split = s.split(","); 89 | return Tuple2.of(Integer.parseInt(split[0]), Integer.parseInt(split[1])); 90 | }) 91 | .returns(new TypeHint>() {}) 92 | .keyBy(0) 93 | .map(new MapCounter()) 94 | .uid("hello"); 95 | } 96 | 97 | public static class MapCounter extends RichMapFunction, String> { 98 | 99 | private static final long serialVersionUID = 7317800376639115920L; 100 | private MapState count; 101 | 102 | @Override 103 | public void open(Configuration parameters) throws Exception { 104 | MapStateDescriptor descriptor = new MapStateDescriptor<>("Count", String.class, 105 | Integer.class); 106 | count = getRuntimeContext().getMapState(descriptor); 107 | } 108 | 109 | @Override 110 | public String map(Tuple2 value) throws Exception { 111 | count.put(value.f1.toString(), Optional.ofNullable(count.get(value.f1.toString())).orElse(0) + 1); 112 | return ""; 113 | } 114 | } 115 | 116 | } 117 | -------------------------------------------------------------------------------- /bravo/src/test/java/com/king/bravo/RocksDBCheckpointReadingTest.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; 19 | 20 | import static org.junit.Assert.assertEquals; 21 | 22 | import java.io.IOException; 23 | import java.util.HashSet; 24 | import java.util.Optional; 25 | 26 | import org.apache.flink.api.common.functions.RichMapFunction; 27 | import org.apache.flink.api.common.state.ValueState; 28 | import org.apache.flink.api.common.state.ValueStateDescriptor; 29 | import org.apache.flink.api.common.typeinfo.TypeHint; 30 | import org.apache.flink.api.java.DataSet; 31 | import org.apache.flink.api.java.ExecutionEnvironment; 32 | import org.apache.flink.api.java.tuple.Tuple2; 33 | import org.apache.flink.configuration.Configuration; 34 | import org.apache.flink.runtime.checkpoint.savepoint.Savepoint; 35 | import org.apache.flink.streaming.api.datastream.DataStream; 36 | import org.junit.Test; 37 | 38 | import com.google.common.collect.Sets; 39 | import com.king.bravo.reader.KeyedStateReader; 40 | import com.king.bravo.reader.OperatorStateReader; 41 | import com.king.bravo.testing.BravoTestPipeline; 42 | 43 | public class RocksDBCheckpointReadingTest extends BravoTestPipeline { 44 | 45 | private static final long serialVersionUID = 1L; 46 | 47 | @Test 48 | public void test() throws Exception { 49 | process("1"); 50 | process("2"); 51 | process("1"); 52 | sleep(1000); 53 | cancelJob(); 54 | runTestPipeline(this::constructTestPipeline); 55 | validateCheckpointedStateReading(); 56 | } 57 | 58 | @SuppressWarnings("unchecked") 59 | private void validateCheckpointedStateReading() throws IOException, Exception { 60 | ExecutionEnvironment environment = ExecutionEnvironment.createLocalEnvironment(); 61 | Savepoint savepoint = getLastCheckpoint(); 62 | OperatorStateReader reader = new OperatorStateReader(environment, savepoint, "hello"); 63 | 64 | DataSet> countState = reader.readKeyedStates( 65 | KeyedStateReader.forValueStateKVPairs("Count", new TypeHint>() {})); 66 | 67 | assertEquals(Sets.newHashSet(Tuple2.of(1, 2), Tuple2.of(2, 1)), new HashSet<>(countState.collect())); 68 | } 69 | 70 | public DataStream constructTestPipeline(DataStream source) { 71 | return source 72 | .map(Integer::parseInt) 73 | .returns(Integer.class) 74 | .keyBy(i -> i) 75 | .map(new StatefulCounter()) 76 | .uid("hello") 77 | .map(Tuple2::toString) 78 | .returns(String.class); 79 | } 80 | 81 | public static class StatefulCounter extends RichMapFunction> { 82 | 83 | private static final long serialVersionUID = 7317800376639115920L; 84 | private ValueState count; 85 | 86 | @Override 87 | public void open(Configuration parameters) throws Exception { 88 | count = getRuntimeContext().getState(new ValueStateDescriptor<>("Count", Integer.class)); 89 | } 90 | 91 | @Override 92 | public Tuple2 map(Integer value) throws Exception { 93 | count.update(Optional.ofNullable(count.value()).orElse(0) + 1); 94 | return Tuple2.of(value, count.value()); 95 | } 96 | } 97 | 98 | } 99 | -------------------------------------------------------------------------------- /bravo/src/test/java/com/king/bravo/TtlStateTest.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; 19 | 20 | import static org.junit.Assert.assertEquals; 21 | 22 | import java.io.IOException; 23 | import java.util.Collections; 24 | import java.util.HashSet; 25 | import java.util.List; 26 | import java.util.Optional; 27 | 28 | import org.apache.flink.api.common.functions.RichMapFunction; 29 | import org.apache.flink.api.common.state.ListState; 30 | import org.apache.flink.api.common.state.ListStateDescriptor; 31 | import org.apache.flink.api.common.state.MapState; 32 | import org.apache.flink.api.common.state.MapStateDescriptor; 33 | import org.apache.flink.api.common.state.StateTtlConfig; 34 | import org.apache.flink.api.common.state.ValueState; 35 | import org.apache.flink.api.common.state.ValueStateDescriptor; 36 | import org.apache.flink.api.common.time.Time; 37 | import org.apache.flink.api.common.typeinfo.BasicTypeInfo; 38 | import org.apache.flink.api.common.typeinfo.TypeHint; 39 | import org.apache.flink.api.java.ExecutionEnvironment; 40 | import org.apache.flink.api.java.tuple.Tuple2; 41 | import org.apache.flink.api.java.tuple.Tuple3; 42 | import org.apache.flink.configuration.Configuration; 43 | import org.apache.flink.runtime.checkpoint.savepoint.Savepoint; 44 | import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.collect.Lists; 45 | import org.apache.flink.streaming.api.datastream.DataStream; 46 | import org.junit.Test; 47 | 48 | import com.google.common.collect.Sets; 49 | import com.king.bravo.reader.KeyedStateReader; 50 | import com.king.bravo.reader.OperatorStateReader; 51 | import com.king.bravo.testing.BravoTestPipeline; 52 | 53 | public class TtlStateTest extends BravoTestPipeline { 54 | 55 | private static final long serialVersionUID = 1L; 56 | 57 | @Test 58 | public void test() throws Exception { 59 | process("1,1"); 60 | process("2,3"); 61 | process("1,2"); 62 | process("1,1"); 63 | sleep(1000); 64 | cancelJob(); 65 | runTestPipeline(this::constructTestPipeline); 66 | validateCheckpointedStateReading(); 67 | } 68 | 69 | @SuppressWarnings("unchecked") 70 | private void validateCheckpointedStateReading() throws IOException, Exception { 71 | ExecutionEnvironment environment = ExecutionEnvironment.createLocalEnvironment(); 72 | Savepoint savepoint = getLastCheckpoint(); 73 | OperatorStateReader reader = new OperatorStateReader(environment, savepoint, "hello"); 74 | 75 | List> countState = reader 76 | .readKeyedStates(KeyedStateReader.forMapStateEntries("Map", BasicTypeInfo.INT_TYPE_INFO, 77 | BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO)) 78 | .collect(); 79 | 80 | List mapValues = reader 81 | .readKeyedStates(KeyedStateReader.forMapStateValues("Map", BasicTypeInfo.INT_TYPE_INFO, true)) 82 | .collect(); 83 | 84 | Collections.sort(mapValues); 85 | assertEquals(Lists.newArrayList(1, 1, 2), mapValues); 86 | 87 | assertEquals(Sets.newHashSet(Tuple3.of(1, "1", 2), Tuple3.of(1, "2", 1), Tuple3.of(2, "3", 1)), 88 | new HashSet<>(countState)); 89 | 90 | List>> listState = reader.readKeyedStates( 91 | KeyedStateReader.forListStates("List", BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO)) 92 | .collect(); 93 | 94 | assertEquals(Sets.newHashSet(Tuple2.of(1, Lists.newArrayList(1, 2, 1)), Tuple2.of(2, Lists.newArrayList(3))), 95 | new HashSet<>(listState)); 96 | 97 | List> listStateValues = reader.readKeyedStates( 98 | KeyedStateReader.forListStateValues("List", BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO)) 99 | .collect(); 100 | 101 | assertEquals(Sets.newHashSet(Tuple2.of(1, 1), Tuple2.of(1, 2), Tuple2.of(2, 3)), 102 | new HashSet<>(listStateValues)); 103 | 104 | List> valuePairs = reader.readKeyedStates( 105 | KeyedStateReader.forValueStateKVPairs("Val", BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO)) 106 | .collect(); 107 | 108 | assertEquals(Sets.newHashSet(Tuple2.of(1, 1), Tuple2.of(2, 3)), new HashSet<>(valuePairs)); 109 | 110 | List values = reader.readKeyedStates( 111 | KeyedStateReader.forValueStateValues("Val", BasicTypeInfo.INT_TYPE_INFO)) 112 | .collect(); 113 | 114 | assertEquals(Sets.newHashSet(1, 3), new HashSet<>(values)); 115 | 116 | } 117 | 118 | public DataStream constructTestPipeline(DataStream source) { 119 | return source 120 | .map(s -> { 121 | String[] split = s.split(","); 122 | return Tuple2.of(Integer.parseInt(split[0]), Integer.parseInt(split[1])); 123 | }) 124 | .returns(new TypeHint>() {}) 125 | .keyBy(0) 126 | .map(new MapCounter()) 127 | .uid("hello"); 128 | } 129 | 130 | public static class MapCounter extends RichMapFunction, String> { 131 | 132 | private static final long serialVersionUID = 7317800376639115920L; 133 | private MapState mapState; 134 | private ListState listState; 135 | private ValueState valueState; 136 | 137 | @Override 138 | public void open(Configuration parameters) throws Exception { 139 | MapStateDescriptor mapDesc = new MapStateDescriptor<>("Map", String.class, 140 | Integer.class); 141 | mapDesc.enableTimeToLive(StateTtlConfig.newBuilder(Time.minutes(100)).build()); 142 | mapState = getRuntimeContext().getMapState(mapDesc); 143 | 144 | ListStateDescriptor listDesc = new ListStateDescriptor<>("List", Integer.class); 145 | listDesc.enableTimeToLive(StateTtlConfig.newBuilder(Time.minutes(100)).build()); 146 | listState = getRuntimeContext().getListState(listDesc); 147 | 148 | ValueStateDescriptor valueStateDesc = new ValueStateDescriptor("Val", Integer.class); 149 | valueStateDesc.enableTimeToLive(StateTtlConfig.newBuilder(Time.minutes(100)).build()); 150 | valueState = getRuntimeContext().getState(valueStateDesc); 151 | } 152 | 153 | @Override 154 | public String map(Tuple2 value) throws Exception { 155 | mapState.put(value.f1.toString(), Optional.ofNullable(mapState.get(value.f1.toString())).orElse(0) + 1); 156 | listState.add(value.f1); 157 | valueState.update(value.f1); 158 | return ""; 159 | } 160 | } 161 | 162 | } 163 | -------------------------------------------------------------------------------- /bravo/src/test/java/com/king/bravo/ValueStateTransformationTest.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; 19 | 20 | import static org.junit.Assert.assertEquals; 21 | 22 | import java.io.IOException; 23 | import java.util.HashSet; 24 | import java.util.List; 25 | import java.util.Optional; 26 | 27 | import org.apache.flink.api.common.functions.MapFunction; 28 | import org.apache.flink.api.common.functions.RichMapFunction; 29 | import org.apache.flink.api.common.state.ValueState; 30 | import org.apache.flink.api.common.state.ValueStateDescriptor; 31 | import org.apache.flink.api.common.typeinfo.TypeHint; 32 | import org.apache.flink.api.common.typeutils.base.IntSerializer; 33 | import org.apache.flink.api.java.DataSet; 34 | import org.apache.flink.api.java.ExecutionEnvironment; 35 | import org.apache.flink.api.java.tuple.Tuple2; 36 | import org.apache.flink.api.java.tuple.Tuple3; 37 | import org.apache.flink.configuration.Configuration; 38 | import org.apache.flink.core.fs.Path; 39 | import org.apache.flink.runtime.checkpoint.OperatorState; 40 | import org.apache.flink.runtime.checkpoint.savepoint.Savepoint; 41 | import org.apache.flink.streaming.api.datastream.DataStream; 42 | import org.junit.Test; 43 | 44 | import com.google.common.collect.Sets; 45 | import com.king.bravo.reader.KeyedStateReader; 46 | import com.king.bravo.reader.OperatorStateReader; 47 | import com.king.bravo.testing.BravoTestPipeline; 48 | import com.king.bravo.utils.StateMetadataUtils; 49 | import com.king.bravo.writer.OperatorStateWriter; 50 | 51 | public class ValueStateTransformationTest extends BravoTestPipeline { 52 | 53 | private static final long serialVersionUID = 1L; 54 | 55 | @Test 56 | public void test() throws Exception { 57 | process("1"); 58 | process("2"); 59 | process("1"); 60 | triggerSavepoint(); 61 | List output = runTestPipeline(this::constructTestPipeline); 62 | assertEquals(Sets.newHashSet("(1,1)", "(2,1)", "(1,2)"), new HashSet<>(output)); 63 | Path newSavepointPath = transformLastSavepoint(); 64 | 65 | process("1"); 66 | process("2"); 67 | List restoredOutput = restoreTestPipelineFromSnapshot(newSavepointPath.getPath(), 68 | this::restoreTestPipeline); 69 | assertEquals(Sets.newHashSet("(1,1)", "(2,1)", "(1,2)", "(1,5,103)", "(2,3,1002)"), 70 | new HashSet<>(restoredOutput)); 71 | } 72 | 73 | private Path transformLastSavepoint() throws IOException, Exception { 74 | ExecutionEnvironment environment = ExecutionEnvironment.createLocalEnvironment(); 75 | Savepoint savepoint = getLastSavepoint(); 76 | OperatorStateReader reader = new OperatorStateReader(environment, savepoint, "hello"); 77 | 78 | DataSet> countState = reader.readKeyedStates( 79 | KeyedStateReader.forValueStateKVPairs("Count", new TypeHint>() {})); 80 | 81 | DataSet> newCountsToAdd = environment 82 | .fromElements(Tuple2.of(0, 100), Tuple2.of(3, 1000), Tuple2.of(1, 100), Tuple2.of(2, 1000)); 83 | 84 | DataSet> newStates = countState.join(newCountsToAdd).where(0).equalTo(0) 85 | .map(new SumValues()); 86 | 87 | Path newCheckpointBasePath = new Path(getCheckpointDir(), "new"); 88 | OperatorStateWriter operatorStateWriter = new OperatorStateWriter(savepoint, "hello", newCheckpointBasePath); 89 | 90 | operatorStateWriter.addValueState("Count", 91 | countState.map(t -> Tuple2.of(t.f0, t.f1 * 2)).returns(new TypeHint>() {})); 92 | 93 | operatorStateWriter.createNewValueState("Count2", newStates, IntSerializer.INSTANCE); 94 | operatorStateWriter.addKeyedStateRows(reader.getAllUnreadKeyedStateRows()); 95 | 96 | OperatorState newOpState = operatorStateWriter.writeAll(); 97 | Savepoint newSavepoint = StateMetadataUtils.createNewSavepoint(savepoint, newOpState); 98 | StateMetadataUtils.writeSavepointMetadata(newCheckpointBasePath, newSavepoint); 99 | return newCheckpointBasePath; 100 | } 101 | 102 | public DataStream constructTestPipeline(DataStream source) { 103 | return source 104 | .map(Integer::parseInt) 105 | .returns(Integer.class) 106 | .keyBy(i -> i) 107 | .map(new StatefulCounter()) 108 | .uid("hello") 109 | .map(Tuple2::toString) 110 | .returns(String.class); 111 | } 112 | 113 | public DataStream restoreTestPipeline(DataStream source) { 114 | return source 115 | .map(Integer::parseInt) 116 | .returns(Integer.class) 117 | .keyBy(i -> i) 118 | .map(new StatefulCounter2()) 119 | .uid("hello").map(Tuple3::toString) 120 | .returns(String.class); 121 | } 122 | 123 | public static class StatefulCounter extends RichMapFunction> { 124 | 125 | private static final long serialVersionUID = 7317800376639115920L; 126 | private ValueState count; 127 | 128 | @Override 129 | public void open(Configuration parameters) throws Exception { 130 | ValueStateDescriptor descriptor = new ValueStateDescriptor<>("Count", Integer.class); 131 | count = getRuntimeContext().getState(descriptor); 132 | } 133 | 134 | @Override 135 | public Tuple2 map(Integer value) throws Exception { 136 | count.update(Optional.ofNullable(count.value()).orElse(0) + 1); 137 | return Tuple2.of(value, count.value()); 138 | } 139 | } 140 | 141 | public static class StatefulCounter2 extends RichMapFunction> { 142 | 143 | private static final long serialVersionUID = 7317800376639115920L; 144 | private ValueState count; 145 | private ValueState count2; 146 | 147 | @Override 148 | public void open(Configuration parameters) throws Exception { 149 | count = getRuntimeContext().getState(new ValueStateDescriptor<>("Count", Integer.class)); 150 | count2 = getRuntimeContext().getState(new ValueStateDescriptor<>("Count2", Integer.class)); 151 | } 152 | 153 | @Override 154 | public Tuple3 map(Integer value) throws Exception { 155 | count.update(Optional.ofNullable(count.value()).orElse(0) + 1); 156 | count2.update(Optional.ofNullable(count2.value()).orElse(0) + 1); 157 | return Tuple3.of(value, count.value(), count2.value()); 158 | } 159 | } 160 | 161 | public static class SumValues implements 162 | MapFunction, Tuple2>, Tuple2> { 163 | private static final long serialVersionUID = 1L; 164 | 165 | @Override 166 | public Tuple2 map(Tuple2, Tuple2> t) 167 | throws Exception { 168 | return Tuple2.of(t.f0.f0, t.f0.f1 + t.f1.f1); 169 | } 170 | } 171 | } 172 | -------------------------------------------------------------------------------- /bravo/src/test/java/com/king/bravo/ValueStateTypeChangeTest.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; 19 | 20 | import static org.junit.Assert.assertEquals; 21 | 22 | import java.io.IOException; 23 | import java.util.HashSet; 24 | import java.util.List; 25 | import java.util.Optional; 26 | 27 | import org.apache.flink.api.common.functions.RichMapFunction; 28 | import org.apache.flink.api.common.state.ValueState; 29 | import org.apache.flink.api.common.state.ValueStateDescriptor; 30 | import org.apache.flink.api.common.typeinfo.TypeHint; 31 | import org.apache.flink.api.common.typeutils.base.IntSerializer; 32 | import org.apache.flink.api.java.DataSet; 33 | import org.apache.flink.api.java.ExecutionEnvironment; 34 | import org.apache.flink.api.java.tuple.Tuple2; 35 | import org.apache.flink.configuration.Configuration; 36 | import org.apache.flink.core.fs.Path; 37 | import org.apache.flink.runtime.checkpoint.savepoint.Savepoint; 38 | import org.apache.flink.streaming.api.datastream.DataStream; 39 | import org.junit.Test; 40 | 41 | import com.google.common.collect.Sets; 42 | import com.king.bravo.reader.KeyedStateReader; 43 | import com.king.bravo.reader.OperatorStateReader; 44 | import com.king.bravo.testing.BravoTestPipeline; 45 | import com.king.bravo.utils.StateMetadataUtils; 46 | import com.king.bravo.writer.OperatorStateWriter; 47 | 48 | public class ValueStateTypeChangeTest extends BravoTestPipeline { 49 | 50 | private static final long serialVersionUID = 1L; 51 | 52 | @Test 53 | public void test() throws Exception { 54 | process("1"); 55 | process("1"); 56 | process("3"); 57 | process("1"); 58 | process("3"); 59 | triggerSavepoint(); 60 | List output = runTestPipeline(this::pipelineWithStringState); 61 | assertEquals( 62 | Sets.newHashSet("(1,1)", "(1,2)", "(1,3)", "(3,3)", "(3,6)"), new HashSet<>(output)); 63 | Path newSavepointPath = transformLastSavepoint(); 64 | 65 | process("1"); 66 | process("2"); 67 | process("3"); 68 | 69 | List restoredOutput = restoreTestPipelineFromSnapshot(newSavepointPath.getPath(), 70 | this::pipelineWithIntState); 71 | assertEquals(Sets.newHashSet("(1,1)", "(1,2)", "(1,3)", "(3,3)", "(3,6)", "(1,4)", "(3,9)", "(2,2)"), 72 | new HashSet<>(restoredOutput)); 73 | } 74 | 75 | private Path transformLastSavepoint() throws IOException, Exception { 76 | ExecutionEnvironment environment = ExecutionEnvironment.createLocalEnvironment(); 77 | Savepoint savepoint = getLastSavepoint(); 78 | OperatorStateReader reader = new OperatorStateReader(environment, savepoint, "hello"); 79 | 80 | DataSet> sumStringState = reader.readKeyedStates( 81 | KeyedStateReader.forValueStateKVPairs("sum", new TypeHint>() {})); 82 | 83 | DataSet> sumIntState = sumStringState.map(t -> Tuple2.of(t.f0, Integer.parseInt(t.f1))) 84 | .returns(new TypeHint>() {}); 85 | 86 | Path newCheckpointBasePath = new Path(getCheckpointDir(), "new"); 87 | 88 | OperatorStateWriter sumStateWriter = new OperatorStateWriter(savepoint, "hello", newCheckpointBasePath); 89 | sumStateWriter.createNewValueState("sum", sumIntState, IntSerializer.INSTANCE); 90 | 91 | Savepoint newSavepoint = StateMetadataUtils.createNewSavepoint(savepoint, sumStateWriter.writeAll()); 92 | StateMetadataUtils.writeSavepointMetadata(newCheckpointBasePath, newSavepoint); 93 | return newCheckpointBasePath; 94 | } 95 | 96 | public DataStream pipelineWithStringState(DataStream source) { 97 | return source 98 | .map(Integer::parseInt) 99 | .returns(Integer.class) 100 | .keyBy(i -> i) 101 | .map(new MapWithStringState()) 102 | .uid("hello") 103 | .map(Tuple2::toString); 104 | } 105 | 106 | public DataStream pipelineWithIntState(DataStream source) { 107 | return source 108 | .map(Integer::parseInt) 109 | .returns(Integer.class) 110 | .keyBy(i -> i) 111 | .map(new MapWithIntState()) 112 | .uid("hello") 113 | .map(Tuple2::toString); 114 | } 115 | 116 | public static class MapWithStringState extends RichMapFunction> { 117 | 118 | private static final long serialVersionUID = 7317800376639115920L; 119 | private ValueState sum; 120 | 121 | @Override 122 | public void open(Configuration parameters) throws Exception { 123 | sum = getRuntimeContext().getState(new ValueStateDescriptor<>("sum", String.class)); 124 | } 125 | 126 | @Override 127 | public Tuple2 map(Integer value) throws Exception { 128 | int current = Integer.parseInt(Optional.ofNullable(sum.value()).orElse("0")); 129 | int newVal = current + value; 130 | sum.update(Integer.toString(newVal)); 131 | return Tuple2.of(value, newVal); 132 | } 133 | } 134 | 135 | public static class MapWithIntState extends RichMapFunction> { 136 | 137 | private static final long serialVersionUID = 7317800376639115920L; 138 | private ValueState sum; 139 | 140 | @Override 141 | public void open(Configuration parameters) throws Exception { 142 | sum = getRuntimeContext().getState(new ValueStateDescriptor<>("sum", Integer.class)); 143 | } 144 | 145 | @Override 146 | public Tuple2 map(Integer value) throws Exception { 147 | int current = Optional.ofNullable(sum.value()).orElse(0); 148 | int newVal = current + value; 149 | sum.update(newVal); 150 | return Tuple2.of(value, newVal); 151 | } 152 | } 153 | 154 | } 155 | -------------------------------------------------------------------------------- /bravo/src/test/java/com/king/bravo/WindowStateReadingTest.java: -------------------------------------------------------------------------------- 1 | package com.king.bravo; 2 | 3 | import com.google.common.collect.ImmutableMap; 4 | import com.google.common.collect.ImmutableSet; 5 | import com.king.bravo.reader.KeyedStateReader; 6 | import com.king.bravo.reader.OperatorStateReader; 7 | import com.king.bravo.testing.BravoTestPipeline; 8 | import org.apache.flink.api.common.typeinfo.BasicTypeInfo; 9 | import org.apache.flink.api.common.typeinfo.TypeHint; 10 | import org.apache.flink.api.java.tuple.Tuple2; 11 | import org.apache.flink.api.java.typeutils.MapTypeInfo; 12 | import org.apache.flink.streaming.api.datastream.DataStream; 13 | import org.apache.flink.streaming.api.windowing.time.Time; 14 | import org.junit.Test; 15 | 16 | import java.util.Arrays; 17 | import java.util.HashMap; 18 | import java.util.List; 19 | import java.util.Map; 20 | import java.util.stream.Collectors; 21 | 22 | import static org.apache.flink.api.java.ExecutionEnvironment.createLocalEnvironment; 23 | import static org.junit.Assert.assertEquals; 24 | 25 | public class WindowStateReadingTest extends BravoTestPipeline { 26 | 27 | private static final long serialVersionUID = 1L; 28 | private static final String REDUCER_UID = "test-reducer"; 29 | private static final MapTypeInfo MAP_TYPE_INFO = new MapTypeInfo<>(String.class, String.class); 30 | 31 | @Test 32 | public void readWindowState() throws Exception { 33 | Arrays.asList("1,1", "2,3", "1,2", "1,1").stream().forEach(this::process); 34 | sleep(1000); 35 | cancelJob(); 36 | runTestPipeline(this::constructTestPipeline); 37 | OperatorStateReader reader = new OperatorStateReader(createLocalEnvironment(), getLastCheckpoint(), 38 | REDUCER_UID); 39 | assertReadWindowStateValues(reader); 40 | assertReadWindowStateKVPairs(reader); 41 | } 42 | 43 | private void assertReadWindowStateValues(OperatorStateReader reader) throws Exception { 44 | List> mapValues = reader.readKeyedStates(KeyedStateReader 45 | .forWindowStateValues(MAP_TYPE_INFO)).collect(); 46 | 47 | assertEquals( 48 | ImmutableSet.of( 49 | ImmutableMap.of("2", "3"), 50 | ImmutableMap.of("1", "1") 51 | ), ImmutableSet.copyOf(mapValues)); 52 | } 53 | 54 | private void assertReadWindowStateKVPairs(OperatorStateReader reader) throws Exception { 55 | List>> mapKeysAndValues = reader.readKeyedStates(KeyedStateReader 56 | .forWindowStateKVPairs(BasicTypeInfo.STRING_TYPE_INFO, MAP_TYPE_INFO)).collect(); 57 | 58 | assertEquals( 59 | ImmutableMap.of( 60 | "2", ImmutableMap.of("2", "3"), 61 | "1", ImmutableMap.of("1", "1") 62 | ), mapKeysAndValues.stream().collect(Collectors.toMap(t -> t.f0, t -> t.f1))); 63 | } 64 | 65 | public DataStream constructTestPipeline(DataStream source) { 66 | return source 67 | .map(s -> new HashMap<>(ImmutableMap.of(s.split(",")[0], s.split(",")[1]))) 68 | .returns(new TypeHint>() {}) 69 | .keyBy(map -> map.keySet().iterator().next()) 70 | .timeWindow(Time.milliseconds(1)) 71 | .reduce((l, r) -> l.values().iterator().next().compareTo(r.values().iterator().next()) > 0 ? r : l) 72 | .uid(REDUCER_UID) 73 | // convert output type to be compatible with BravoTestPipeline#runTestPipeline 74 | .map(Map::toString); 75 | } 76 | 77 | } 78 | -------------------------------------------------------------------------------- /bravo/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 | -------------------------------------------------------------------------------- /build.gradle: -------------------------------------------------------------------------------- 1 | buildscript { 2 | repositories { 3 | maven { url "https://plugins.gradle.org/m2/" } 4 | } 5 | } 6 | 7 | ext.versions = [ 8 | flink : '1.7.1', 9 | slf4j : '1.7.25', 10 | logback : '1.2.3', 11 | junit : '4.12', 12 | ] 13 | 14 | subprojects { 15 | apply plugin: 'java' 16 | apply plugin: 'maven' 17 | apply plugin: 'signing' 18 | 19 | group = 'com.king.bravo' 20 | version = '0.6' 21 | 22 | repositories { 23 | mavenCentral() 24 | } 25 | 26 | task javadocJar(type: Jar) { 27 | classifier = 'javadoc' 28 | from javadoc 29 | } 30 | 31 | task sourcesJar(type: Jar) { 32 | classifier = 'sources' 33 | from sourceSets.main.allSource 34 | } 35 | 36 | configurations.all { 37 | exclude module: 'commons-logging' 38 | exclude module: 'log4j' 39 | exclude module: 'slf4j-log4j12' 40 | } 41 | 42 | dependencies { 43 | 44 | testCompile group: 'junit', name: 'junit', version: versions.junit 45 | testRuntime group: 'ch.qos.logback', name: 'logback-classic', version: versions.logback 46 | testRuntime group: 'org.slf4j', name: 'log4j-over-slf4j', version: versions.slf4j 47 | testRuntime group: 'org.slf4j', name: 'jcl-over-slf4j', version: versions.slf4j 48 | } 49 | 50 | artifacts { 51 | archives javadocJar, sourcesJar 52 | } 53 | 54 | signing { 55 | sign configurations.archives 56 | } 57 | 58 | tasks.withType(Sign) { 59 | onlyIf {project.hasProperty('release') && release} 60 | } 61 | 62 | uploadArchives { 63 | repositories { 64 | mavenDeployer { 65 | 66 | pom.project { 67 | name 'bravo' 68 | packaging 'jar' 69 | description 'Utilities for processing Flink checkpoints/savepoints' 70 | url 'https://github.com/king/bravo' 71 | 72 | scm { 73 | connection 'https://github.com/king/bravo.git' 74 | developerConnection 'https://github.com/king/bravo.git' 75 | url 'https://github.com/king/bravo' 76 | } 77 | 78 | licenses { 79 | license { 80 | name 'The Apache License, Version 2.0' 81 | url 'http://www.apache.org/licenses/LICENSE-2.0.txt' 82 | } 83 | } 84 | 85 | developers { 86 | developer { 87 | id 'gyula' 88 | name 'Gyula Fora' 89 | email 'gyula.fora@king.com' 90 | } 91 | } 92 | } 93 | 94 | beforeDeployment { MavenDeployment deployment -> signing.signPom(deployment) } 95 | 96 | repository(url: "https://oss.sonatype.org/service/local/staging/deploy/maven2/") { 97 | authentication(userName: project.findProperty('nexusUsername') ?: '', password: project.findProperty('nexusPassword') ?: '') 98 | } 99 | 100 | snapshotRepository(url: "https://oss.sonatype.org/content/repositories/snapshots/") { 101 | authentication(userName: project.findProperty('nexusUsername') ?: '', password: project.findProperty('nexusPassword') ?: '') 102 | } 103 | } 104 | } 105 | } 106 | 107 | } 108 | 109 | wrapper { 110 | gradleVersion = 4.7 111 | } 112 | -------------------------------------------------------------------------------- /gradle/wrapper/gradle-wrapper.jar: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/king/bravo/f461107f8d7352421761c267d64ad9c2abf6caa2/gradle/wrapper/gradle-wrapper.jar -------------------------------------------------------------------------------- /gradle/wrapper/gradle-wrapper.properties: -------------------------------------------------------------------------------- 1 | distributionBase=GRADLE_USER_HOME 2 | distributionPath=wrapper/dists 3 | distributionUrl=https\://services.gradle.org/distributions/gradle-4.7-bin.zip 4 | zipStoreBase=GRADLE_USER_HOME 5 | zipStorePath=wrapper/dists 6 | -------------------------------------------------------------------------------- /gradlew: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env sh 2 | 3 | ############################################################################## 4 | ## 5 | ## Gradle start up script for UN*X 6 | ## 7 | ############################################################################## 8 | 9 | # Attempt to set APP_HOME 10 | # Resolve links: $0 may be a link 11 | PRG="$0" 12 | # Need this for relative symlinks. 13 | while [ -h "$PRG" ] ; do 14 | ls=`ls -ld "$PRG"` 15 | link=`expr "$ls" : '.*-> \(.*\)$'` 16 | if expr "$link" : '/.*' > /dev/null; then 17 | PRG="$link" 18 | else 19 | PRG=`dirname "$PRG"`"/$link" 20 | fi 21 | done 22 | SAVED="`pwd`" 23 | cd "`dirname \"$PRG\"`/" >/dev/null 24 | APP_HOME="`pwd -P`" 25 | cd "$SAVED" >/dev/null 26 | 27 | APP_NAME="Gradle" 28 | APP_BASE_NAME=`basename "$0"` 29 | 30 | # Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. 31 | DEFAULT_JVM_OPTS="" 32 | 33 | # Use the maximum available, or set MAX_FD != -1 to use that value. 34 | MAX_FD="maximum" 35 | 36 | warn () { 37 | echo "$*" 38 | } 39 | 40 | die () { 41 | echo 42 | echo "$*" 43 | echo 44 | exit 1 45 | } 46 | 47 | # OS specific support (must be 'true' or 'false'). 48 | cygwin=false 49 | msys=false 50 | darwin=false 51 | nonstop=false 52 | case "`uname`" in 53 | CYGWIN* ) 54 | cygwin=true 55 | ;; 56 | Darwin* ) 57 | darwin=true 58 | ;; 59 | MINGW* ) 60 | msys=true 61 | ;; 62 | NONSTOP* ) 63 | nonstop=true 64 | ;; 65 | esac 66 | 67 | CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar 68 | 69 | # Determine the Java command to use to start the JVM. 70 | if [ -n "$JAVA_HOME" ] ; then 71 | if [ -x "$JAVA_HOME/jre/sh/java" ] ; then 72 | # IBM's JDK on AIX uses strange locations for the executables 73 | JAVACMD="$JAVA_HOME/jre/sh/java" 74 | else 75 | JAVACMD="$JAVA_HOME/bin/java" 76 | fi 77 | if [ ! -x "$JAVACMD" ] ; then 78 | die "ERROR: JAVA_HOME is set to an invalid directory: $JAVA_HOME 79 | 80 | Please set the JAVA_HOME variable in your environment to match the 81 | location of your Java installation." 82 | fi 83 | else 84 | JAVACMD="java" 85 | which java >/dev/null 2>&1 || die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. 86 | 87 | Please set the JAVA_HOME variable in your environment to match the 88 | location of your Java installation." 89 | fi 90 | 91 | # Increase the maximum file descriptors if we can. 92 | if [ "$cygwin" = "false" -a "$darwin" = "false" -a "$nonstop" = "false" ] ; then 93 | MAX_FD_LIMIT=`ulimit -H -n` 94 | if [ $? -eq 0 ] ; then 95 | if [ "$MAX_FD" = "maximum" -o "$MAX_FD" = "max" ] ; then 96 | MAX_FD="$MAX_FD_LIMIT" 97 | fi 98 | ulimit -n $MAX_FD 99 | if [ $? -ne 0 ] ; then 100 | warn "Could not set maximum file descriptor limit: $MAX_FD" 101 | fi 102 | else 103 | warn "Could not query maximum file descriptor limit: $MAX_FD_LIMIT" 104 | fi 105 | fi 106 | 107 | # For Darwin, add options to specify how the application appears in the dock 108 | if $darwin; then 109 | GRADLE_OPTS="$GRADLE_OPTS \"-Xdock:name=$APP_NAME\" \"-Xdock:icon=$APP_HOME/media/gradle.icns\"" 110 | fi 111 | 112 | # For Cygwin, switch paths to Windows format before running java 113 | if $cygwin ; then 114 | APP_HOME=`cygpath --path --mixed "$APP_HOME"` 115 | CLASSPATH=`cygpath --path --mixed "$CLASSPATH"` 116 | JAVACMD=`cygpath --unix "$JAVACMD"` 117 | 118 | # We build the pattern for arguments to be converted via cygpath 119 | ROOTDIRSRAW=`find -L / -maxdepth 1 -mindepth 1 -type d 2>/dev/null` 120 | SEP="" 121 | for dir in $ROOTDIRSRAW ; do 122 | ROOTDIRS="$ROOTDIRS$SEP$dir" 123 | SEP="|" 124 | done 125 | OURCYGPATTERN="(^($ROOTDIRS))" 126 | # Add a user-defined pattern to the cygpath arguments 127 | if [ "$GRADLE_CYGPATTERN" != "" ] ; then 128 | OURCYGPATTERN="$OURCYGPATTERN|($GRADLE_CYGPATTERN)" 129 | fi 130 | # Now convert the arguments - kludge to limit ourselves to /bin/sh 131 | i=0 132 | for arg in "$@" ; do 133 | CHECK=`echo "$arg"|egrep -c "$OURCYGPATTERN" -` 134 | CHECK2=`echo "$arg"|egrep -c "^-"` ### Determine if an option 135 | 136 | if [ $CHECK -ne 0 ] && [ $CHECK2 -eq 0 ] ; then ### Added a condition 137 | eval `echo args$i`=`cygpath --path --ignore --mixed "$arg"` 138 | else 139 | eval `echo args$i`="\"$arg\"" 140 | fi 141 | i=$((i+1)) 142 | done 143 | case $i in 144 | (0) set -- ;; 145 | (1) set -- "$args0" ;; 146 | (2) set -- "$args0" "$args1" ;; 147 | (3) set -- "$args0" "$args1" "$args2" ;; 148 | (4) set -- "$args0" "$args1" "$args2" "$args3" ;; 149 | (5) set -- "$args0" "$args1" "$args2" "$args3" "$args4" ;; 150 | (6) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" ;; 151 | (7) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" ;; 152 | (8) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" ;; 153 | (9) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" "$args8" ;; 154 | esac 155 | fi 156 | 157 | # Escape application args 158 | save () { 159 | for i do printf %s\\n "$i" | sed "s/'/'\\\\''/g;1s/^/'/;\$s/\$/' \\\\/" ; done 160 | echo " " 161 | } 162 | APP_ARGS=$(save "$@") 163 | 164 | # Collect all arguments for the java command, following the shell quoting and substitution rules 165 | eval set -- $DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS "\"-Dorg.gradle.appname=$APP_BASE_NAME\"" -classpath "\"$CLASSPATH\"" org.gradle.wrapper.GradleWrapperMain "$APP_ARGS" 166 | 167 | # by default we should be in the correct project dir, but when run from Finder on Mac, the cwd is wrong 168 | if [ "$(uname)" = "Darwin" ] && [ "$HOME" = "$PWD" ]; then 169 | cd "$(dirname "$0")" 170 | fi 171 | 172 | exec "$JAVACMD" "$@" 173 | -------------------------------------------------------------------------------- /gradlew.bat: -------------------------------------------------------------------------------- 1 | @if "%DEBUG%" == "" @echo off 2 | @rem ########################################################################## 3 | @rem 4 | @rem Gradle startup script for Windows 5 | @rem 6 | @rem ########################################################################## 7 | 8 | @rem Set local scope for the variables with windows NT shell 9 | if "%OS%"=="Windows_NT" setlocal 10 | 11 | set DIRNAME=%~dp0 12 | if "%DIRNAME%" == "" set DIRNAME=. 13 | set APP_BASE_NAME=%~n0 14 | set APP_HOME=%DIRNAME% 15 | 16 | @rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. 17 | set DEFAULT_JVM_OPTS= 18 | 19 | @rem Find java.exe 20 | if defined JAVA_HOME goto findJavaFromJavaHome 21 | 22 | set JAVA_EXE=java.exe 23 | %JAVA_EXE% -version >NUL 2>&1 24 | if "%ERRORLEVEL%" == "0" goto init 25 | 26 | echo. 27 | echo ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. 28 | echo. 29 | echo Please set the JAVA_HOME variable in your environment to match the 30 | echo location of your Java installation. 31 | 32 | goto fail 33 | 34 | :findJavaFromJavaHome 35 | set JAVA_HOME=%JAVA_HOME:"=% 36 | set JAVA_EXE=%JAVA_HOME%/bin/java.exe 37 | 38 | if exist "%JAVA_EXE%" goto init 39 | 40 | echo. 41 | echo ERROR: JAVA_HOME is set to an invalid directory: %JAVA_HOME% 42 | echo. 43 | echo Please set the JAVA_HOME variable in your environment to match the 44 | echo location of your Java installation. 45 | 46 | goto fail 47 | 48 | :init 49 | @rem Get command-line arguments, handling Windows variants 50 | 51 | if not "%OS%" == "Windows_NT" goto win9xME_args 52 | 53 | :win9xME_args 54 | @rem Slurp the command line arguments. 55 | set CMD_LINE_ARGS= 56 | set _SKIP=2 57 | 58 | :win9xME_args_slurp 59 | if "x%~1" == "x" goto execute 60 | 61 | set CMD_LINE_ARGS=%* 62 | 63 | :execute 64 | @rem Setup the command line 65 | 66 | set CLASSPATH=%APP_HOME%\gradle\wrapper\gradle-wrapper.jar 67 | 68 | @rem Execute Gradle 69 | "%JAVA_EXE%" %DEFAULT_JVM_OPTS% %JAVA_OPTS% %GRADLE_OPTS% "-Dorg.gradle.appname=%APP_BASE_NAME%" -classpath "%CLASSPATH%" org.gradle.wrapper.GradleWrapperMain %CMD_LINE_ARGS% 70 | 71 | :end 72 | @rem End local scope for the variables with windows NT shell 73 | if "%ERRORLEVEL%"=="0" goto mainEnd 74 | 75 | :fail 76 | rem Set variable GRADLE_EXIT_CONSOLE if you need the _script_ return code instead of 77 | rem the _cmd.exe /c_ return code! 78 | if not "" == "%GRADLE_EXIT_CONSOLE%" exit 1 79 | exit /b 1 80 | 81 | :mainEnd 82 | if "%OS%"=="Windows_NT" endlocal 83 | 84 | :omega 85 | -------------------------------------------------------------------------------- /settings.gradle: -------------------------------------------------------------------------------- 1 | rootProject.name = 'bravo' 2 | 3 | include ':bravo-test-utils' 4 | include ':bravo' 5 | --------------------------------------------------------------------------------