├── bravo
├── .gitignore
├── build.gradle
└── src
│ ├── test
│ ├── resources
│ │ └── logback-test.xml
│ └── java
│ │ └── com
│ │ └── king
│ │ └── bravo
│ │ ├── WindowStateReadingTest.java
│ │ ├── RocksDBCheckpointReadingTest.java
│ │ ├── MapStateReadingTest.java
│ │ ├── ValueStateTypeChangeTest.java
│ │ ├── KeyedStateAddRemoveTest.java
│ │ ├── TtlStateTest.java
│ │ ├── ValueStateTransformationTest.java
│ │ └── BroadcastStateTransformationTest.java
│ └── main
│ └── java
│ └── com
│ └── king
│ └── bravo
│ ├── utils
│ ├── Identifiers.java
│ ├── KeyGroupFlags.java
│ ├── RocksDBUtils.java
│ └── StateMetadataUtils.java
│ ├── writer
│ └── functions
│ │ ├── RowFilter.java
│ │ ├── KeyGroupAndStateNameKey.java
│ │ ├── OperatorIndexForKeyGroupKey.java
│ │ ├── ValueStateToKeyedStateRow.java
│ │ └── RocksDBSavepointWriter.java
│ ├── reader
│ ├── inputformat
│ │ ├── ValidOffsetsIterator.java
│ │ ├── KeyedStateInputSplit.java
│ │ ├── RocksDBKeyedStateInputFormat.java
│ │ ├── RocksDBSavepointIterator.java
│ │ └── RocksDBCheckpointIterator.java
│ ├── AbstractListStateReader.java
│ ├── ValueStateValueReader.java
│ ├── MapStateValueReader.java
│ ├── ListStateListReader.java
│ ├── ValueStateKVReader.java
│ ├── AbstractMapStateReader.java
│ ├── ListStateFlattenReader.java
│ ├── MapStateKKVReader.java
│ ├── OperatorStateReader.java
│ └── KeyedStateReader.java
│ └── types
│ └── KeyedStateRow.java
├── bravo-test-utils
├── .gitignore
├── build.gradle
└── src
│ ├── test
│ └── resources
│ │ └── logback-test.xml
│ └── main
│ └── java
│ └── com
│ └── king
│ └── bravo
│ └── testing
│ ├── actions
│ ├── CancelJob.java
│ ├── TriggerFailure.java
│ ├── TriggerSavepoint.java
│ ├── Sleep.java
│ ├── Process.java
│ ├── NextWatermark.java
│ ├── OneTimePipelineAction.java
│ └── TestPipelineSource.java
│ ├── PipelineAction.java
│ ├── MiniClusterResourceFactory.java
│ ├── CollectingSink.java
│ └── BravoTestPipeline.java
├── settings.gradle
├── .gitignore
├── gradle
└── wrapper
│ ├── gradle-wrapper.jar
│ └── gradle-wrapper.properties
├── gradlew.bat
├── gradlew
├── README.md
└── LICENSE
/bravo/.gitignore:
--------------------------------------------------------------------------------
1 | /build/
2 |
--------------------------------------------------------------------------------
/bravo-test-utils/.gitignore:
--------------------------------------------------------------------------------
1 | /build/
2 |
--------------------------------------------------------------------------------
/settings.gradle:
--------------------------------------------------------------------------------
1 | rootProject.name = 'bravo'
2 |
3 | include ':bravo-test-utils'
4 | include ':bravo'
5 |
--------------------------------------------------------------------------------
/.gitignore:
--------------------------------------------------------------------------------
1 | /.gradle/
2 | .classpath
3 | .project
4 | .settings/
5 | bin/
6 | build/
7 | gradle.properties
8 |
--------------------------------------------------------------------------------
/gradle/wrapper/gradle-wrapper.jar:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/king/bravo/HEAD/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 |
--------------------------------------------------------------------------------
/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-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/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-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-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/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/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/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/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/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/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-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/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/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/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/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-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/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/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-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/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-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/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 |
--------------------------------------------------------------------------------
/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 |
--------------------------------------------------------------------------------
/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/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-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/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/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