├── logo.png ├── .gitignore ├── da-streamingledger-runtime-serial ├── src │ └── main │ │ ├── resources │ │ └── META-INF │ │ │ └── services │ │ │ └── com.dataartisans.streamingledger.sdk.spi.StreamingLedgerRuntimeProvider │ │ └── java │ │ └── com │ │ └── dataartisans │ │ └── streamingledger │ │ └── runtime │ │ └── serial │ │ ├── SideOutputContext.java │ │ ├── SerialStateAccess.java │ │ ├── SerialStreamingLedgerRuntimeProvider.java │ │ ├── SerialTransactor.java │ │ └── SingleStreamSerialTransactor.java └── pom.xml ├── da-streamingledger-examples ├── src │ └── main │ │ ├── resources │ │ └── log4j.properties │ │ └── java │ │ └── com │ │ └── dataartisans │ │ └── streamingledger │ │ └── examples │ │ └── simpletrade │ │ ├── generator │ │ ├── Throttler.java │ │ ├── SyntheticSources.java │ │ └── DepositsThenTransactionsSource.java │ │ ├── DepositEvent.java │ │ ├── TransactionResult.java │ │ ├── TransactionEvent.java │ │ └── SimpleTradeExample.java └── pom.xml ├── da-streamingledger-sdk ├── src │ ├── main │ │ └── java │ │ │ └── com │ │ │ └── dataartisans │ │ │ └── streamingledger │ │ │ └── sdk │ │ │ ├── api │ │ │ ├── package-info.java │ │ │ ├── StateNotReadableException.java │ │ │ ├── StateNotWritableException.java │ │ │ ├── StateAccessException.java │ │ │ ├── AccessType.java │ │ │ ├── TransactionProcessFunction.java │ │ │ └── StateAccess.java │ │ │ ├── common │ │ │ ├── reflection │ │ │ │ ├── ProcessFunctionInvoker.java │ │ │ │ ├── Methods.java │ │ │ │ └── ByteBuddyProcessFunctionInvoker.java │ │ │ └── union │ │ │ │ ├── UnionSerializerConfigSnapshot.java │ │ │ │ ├── TaggedElement.java │ │ │ │ ├── UnionTypeInfo.java │ │ │ │ ├── Union.java │ │ │ │ └── UnionSerializer.java │ │ │ └── spi │ │ │ ├── StreamingLedgerRuntimeProvider.java │ │ │ ├── InputAndSpec.java │ │ │ ├── StreamingLedgerSpec.java │ │ │ ├── StreamingLedgerRuntimeLoader.java │ │ │ └── StreamingLedgerSpecFactory.java │ └── test │ │ └── java │ │ └── com │ │ └── dataartisans │ │ └── streamingledger │ │ └── sdk │ │ ├── common │ │ ├── union │ │ │ └── UnionSerializerTest.java │ │ └── reflection │ │ │ └── ByteBuddyProcessFunctionInvokerTest.java │ │ └── api │ │ └── StreamingLedgerSpecTest.java └── pom.xml ├── tools ├── maven │ ├── checkstyle-suppressions.xml │ ├── spotbugs-exclude.xml │ └── checkstyle.xml └── intellij │ ├── codestyle-java.xml │ └── inspections.xml ├── README.md ├── LICENSE └── pom.xml /logo.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dataArtisans/da-streamingledger/HEAD/logo.png -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | # Compiled class file 2 | *.class 3 | 4 | # Log file 5 | *.log 6 | 7 | # BlueJ files 8 | *.ctxt 9 | 10 | # Mobile Tools for Java (J2ME) 11 | .mtj.tmp/ 12 | 13 | # Package Files # 14 | *.jar 15 | *.war 16 | *.nar 17 | *.ear 18 | *.zip 19 | *.tar.gz 20 | *.rar 21 | 22 | # virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml 23 | hs_err_pid* 24 | 25 | .DS_Store 26 | *.iml 27 | .idea 28 | 29 | # maven shade plugin 30 | dependency-reduced-pom.xml 31 | target 32 | -------------------------------------------------------------------------------- /da-streamingledger-runtime-serial/src/main/resources/META-INF/services/com.dataartisans.streamingledger.sdk.spi.StreamingLedgerRuntimeProvider: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2018 Data Artisans GmbH 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); 5 | # you may not use this file except in compliance with the License. 6 | # You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | # 16 | 17 | com.dataartisans.streamingledger.runtime.serial.SerialStreamingLedgerRuntimeProvider 18 | -------------------------------------------------------------------------------- /da-streamingledger-examples/src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2018 Data Artisans GmbH 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); 5 | # you may not use this file except in compliance with the License. 6 | # You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | # 16 | log4j.rootLogger=INFO, console 17 | log4j.appender.console=org.apache.log4j.ConsoleAppender 18 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 19 | log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n 20 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/api/package-info.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | /** 18 | * The API package for the dA-StreamingLedger support 19 | * for Apache Flink. 20 | */ 21 | 22 | package com.dataartisans.streamingledger.sdk.api; 23 | -------------------------------------------------------------------------------- /tools/maven/checkstyle-suppressions.xml: -------------------------------------------------------------------------------- 1 | 2 | 20 | 21 | 24 | 25 | 26 | 27 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/api/StateNotReadableException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.api; 18 | 19 | /** 20 | * An exception that indicates that state was accessed for read, without being 21 | * declared as readable. 22 | */ 23 | public class StateNotReadableException extends StateAccessException { 24 | 25 | private static final long serialVersionUID = 1L; 26 | 27 | public StateNotReadableException(StateAccess state) { 28 | super(String.format("State access %s for state %s has not declared read access.", 29 | state.getStateAccessName(), state.getStateName())); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/api/StateNotWritableException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.api; 18 | 19 | /** 20 | * An exception that indicates that state was accessed for write, without being 21 | * declared as writable. 22 | */ 23 | public class StateNotWritableException extends StateAccessException { 24 | 25 | private static final long serialVersionUID = 1L; 26 | 27 | public StateNotWritableException(StateAccess state) { 28 | super(String.format("State access %s for state %s has not declared write access.", 29 | state.getStateAccessName(), state.getStateName())); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/common/reflection/ProcessFunctionInvoker.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.common.reflection; 18 | 19 | import com.dataartisans.streamingledger.sdk.api.StateAccess; 20 | import com.dataartisans.streamingledger.sdk.api.TransactionProcessFunction; 21 | import com.dataartisans.streamingledger.sdk.api.TransactionProcessFunction.Context; 22 | 23 | /** 24 | * An auto-generated class that invokes a user provided {@link TransactionProcessFunction}. 25 | * 26 | * @param transaction event type. 27 | * @param transaction result type. 28 | */ 29 | public abstract class ProcessFunctionInvoker { 30 | 31 | public abstract void invoke(InT input, Context context, StateAccess[] arguments); 32 | } 33 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/spi/StreamingLedgerRuntimeProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.spi; 18 | 19 | import com.dataartisans.streamingledger.sdk.api.StreamingLedger.ResultStreams; 20 | 21 | import java.util.List; 22 | 23 | /** 24 | * The stream ledger runtime provider is responsible for taking a stream ledger 25 | * program and creating the Flink streaming graph that should execute it. 26 | * 27 | *

The exact way of executing the distributed transactions is implementation specific. 28 | * Some runtime providers instantiate a serial (trivially correct) execution model, some 29 | * a sophisticated parallel data flow. 30 | */ 31 | public interface StreamingLedgerRuntimeProvider { 32 | 33 | ResultStreams translate(String name, List> streamLedgerSpecs); 34 | } 35 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/api/StateAccessException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.api; 18 | 19 | /** 20 | * Base exception for failures during a state access. 21 | * 22 | *

This is an unchecked exception, because instances of this exception are raised as a result 23 | * of an incorrect program, for example when read and write accesses are incorrectly declared. 24 | * Instances of this exception are hence rarely expected to be handled explicitly within an user's 25 | * implementation of an application, but rather expected to "bubble up" and report a proper 26 | * application failure. 27 | */ 28 | public class StateAccessException extends RuntimeException { 29 | 30 | private static final long serialVersionUID = 1L; 31 | 32 | public StateAccessException(String message) { 33 | super(message); 34 | } 35 | 36 | public StateAccessException(String message, Throwable cause) { 37 | super(message, cause); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/spi/InputAndSpec.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.spi; 18 | 19 | import org.apache.flink.streaming.api.datastream.DataStream; 20 | 21 | import com.dataartisans.streamingledger.sdk.api.StreamingLedger; 22 | 23 | import java.util.List; 24 | 25 | import static java.util.Objects.requireNonNull; 26 | 27 | /** 28 | * An input to {@link StreamingLedgerRuntimeProvider#translate(String, List)}. 29 | * This would be created by {@link StreamingLedger#resultStreams()}. 30 | */ 31 | public final class InputAndSpec { 32 | 33 | public final DataStream inputStream; 34 | 35 | public final String streamName; 36 | 37 | public final StreamingLedgerSpec streamSpec; 38 | 39 | public InputAndSpec(DataStream inputStream, String streamName, StreamingLedgerSpec spec) { 40 | this.inputStream = requireNonNull(inputStream); 41 | this.streamName = requireNonNull(streamName); 42 | this.streamSpec = requireNonNull(spec); 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /da-streamingledger-runtime-serial/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 17 | 19 | 20 | 4.0.0 21 | 22 | 23 | com.data-artisans.streamingledger 24 | da-streamingledger 25 | 1.1-SNAPSHOT 26 | .. 27 | 28 | 29 | da-streamingledger-runtime-serial 30 | da-streamingledger :: da-streamingledger-runtime-serial 31 | 32 | jar 33 | 34 | 35 | 36 | com.data-artisans.streamingledger 37 | da-streamingledger-sdk 38 | ${project.version} 39 | 40 | 41 | org.apache.flink 42 | flink-streaming-java_2.11 43 | ${flink.version} 44 | provided 45 | 46 | 47 | 48 | 49 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/common/reflection/Methods.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.common.reflection; 18 | 19 | import java.lang.annotation.Annotation; 20 | import java.lang.reflect.Method; 21 | import java.util.Arrays; 22 | import java.util.Iterator; 23 | import java.util.stream.Stream; 24 | 25 | /** 26 | * Reflection utility functions. 27 | */ 28 | public final class Methods { 29 | private Methods() { 30 | } 31 | 32 | /** 33 | * Finds all the {@link Method}s that are annotated with the supplied annotation. 34 | * 35 | *

This method will traverse up the superclass hierarchy looking for methods annotated with the supplied 36 | * annotation. 37 | * 38 | * @return an iterator of {@link Method}s found. 39 | */ 40 | public static Iterator findAnnotatedMethods(Class javaClass, Class annotation) { 41 | return definedMethods(javaClass) 42 | .filter(method -> method.getAnnotation(annotation) != null) 43 | .iterator(); 44 | } 45 | 46 | private static Stream definedMethods(Class javaClass) { 47 | if (javaClass == null || javaClass == Object.class) { 48 | return Stream.empty(); 49 | } 50 | Stream selfMethods = Arrays.stream(javaClass.getDeclaredMethods()); 51 | Stream superMethods = definedMethods(javaClass.getSuperclass()); 52 | 53 | return Stream.concat(selfMethods, superMethods); 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/common/union/UnionSerializerConfigSnapshot.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.common.union; 18 | 19 | import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; 20 | import org.apache.flink.api.common.typeutils.TypeSerializer; 21 | 22 | import java.util.List; 23 | 24 | /** 25 | * UnionSerializerConfigSnapshot - A serializer configuration snapshot for the {@link UnionSerializer}. 26 | */ 27 | public final class UnionSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { 28 | 29 | private static final int VERSION = 1; 30 | 31 | /** 32 | * A default constructor is required by Flink's serialization framework. 33 | */ 34 | @SuppressWarnings("unused") 35 | public UnionSerializerConfigSnapshot() { 36 | } 37 | 38 | /** 39 | * Creates a new instance of UnionSerializerConfigSnapshot. 40 | */ 41 | @SuppressWarnings("WeakerAccess") 42 | public UnionSerializerConfigSnapshot(List> underlyingSerializers) { 43 | super(toArray(underlyingSerializers)); 44 | } 45 | 46 | private static TypeSerializer[] toArray(List> underlyingSerializers) { 47 | final int n = underlyingSerializers.size(); 48 | TypeSerializer[] rawSerializers = new TypeSerializer[n]; 49 | return underlyingSerializers.toArray(rawSerializers); 50 | } 51 | 52 | @Override 53 | public int getVersion() { 54 | return VERSION; 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /tools/maven/spotbugs-exclude.xml: -------------------------------------------------------------------------------- 1 | 2 | 20 | 21 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | 35 | 36 | 37 | 38 | 39 | 40 | 41 | 42 | 43 | 44 | 45 | 46 | 47 | 48 | 49 | 50 | -------------------------------------------------------------------------------- /tools/intellij/codestyle-java.xml: -------------------------------------------------------------------------------- 1 | 16 | 17 | 23 | 38 | 39 | 49 | -------------------------------------------------------------------------------- /da-streamingledger-runtime-serial/src/main/java/com/dataartisans/streamingledger/runtime/serial/SideOutputContext.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.runtime.serial; 18 | 19 | import org.apache.flink.streaming.api.functions.ProcessFunction; 20 | import org.apache.flink.streaming.api.functions.ProcessFunction.Context; 21 | import org.apache.flink.util.OutputTag; 22 | 23 | import com.dataartisans.streamingledger.sdk.api.TransactionProcessFunction; 24 | 25 | import java.io.Serializable; 26 | import java.util.ArrayList; 27 | 28 | final class SideOutputContext implements TransactionProcessFunction.Context, Serializable { 29 | 30 | private static final long serialVersionUID = 1; 31 | private final ArrayList records = new ArrayList<>(); 32 | private transient OutputTag outputTag; 33 | private transient ProcessFunction.Context context; 34 | private boolean aborted; 35 | 36 | void setContext(Context context) { 37 | this.context = context; 38 | } 39 | 40 | void setOutputTag(OutputTag outputTag) { 41 | this.outputTag = outputTag; 42 | } 43 | 44 | void prepare() { 45 | aborted = false; 46 | records.clear(); 47 | } 48 | 49 | @Override 50 | public void abort() { 51 | aborted = true; 52 | records.clear(); 53 | } 54 | 55 | @SuppressWarnings("unchecked") 56 | @Override 57 | public void emit(T record) { 58 | records.add(record); 59 | } 60 | 61 | boolean wasAborted() { 62 | return aborted; 63 | } 64 | 65 | void emitChanges() { 66 | if (aborted) { 67 | return; 68 | } 69 | for (T record : records) { 70 | context.output(outputTag, record); 71 | } 72 | } 73 | } 74 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/api/AccessType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.api; 18 | 19 | /** 20 | * The AccessType defines whether certain state is only read or written or both. 21 | * 22 | *

The type of access has an impact on whether accesses to the state (and key) by different 23 | * transactions can execute independent of each other, or whether they need to be coordinated. 24 | * For example, multiple transactions can concurrently read a certain state, if no other 25 | * state currently writes the state. 26 | */ 27 | @SuppressWarnings("unused") 28 | public enum AccessType { 29 | 30 | /** 31 | * Indicates that state is only read, but not modified. 32 | * If state is only read, it can never block any other transaction from advancing. 33 | */ 34 | READ(false, false), 35 | 36 | /** 37 | * Indicates that state is modified, but not read before. Modifying state means that coordination 38 | * is involved to ensure consistency for the write back. 39 | */ 40 | WRITE(true, true), 41 | 42 | /** 43 | * Indicates that state is read and later modified. This access type involves coordination to 44 | * ensure consistency for the write back, and possibly additional messages to read state. 45 | */ 46 | READ_WRITE(true, false); 47 | 48 | private final boolean requiresLocking; 49 | private final boolean writeOnly; 50 | 51 | AccessType(boolean requiresLocking, boolean writeOnly) { 52 | this.requiresLocking = requiresLocking; 53 | this.writeOnly = writeOnly; 54 | } 55 | 56 | public boolean requiresLocking() { 57 | return requiresLocking; 58 | } 59 | 60 | public boolean writeOnly() { 61 | return writeOnly; 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/common/union/TaggedElement.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.common.union; 18 | 19 | import java.util.List; 20 | import java.util.Objects; 21 | 22 | /** 23 | * A simple POJO with a dataStreamTag and an element. 24 | * 25 | *

see {@link Union#apply(List)}. 26 | */ 27 | public final class TaggedElement { 28 | 29 | public static final int UNDEFINED_TAG = -1; 30 | 31 | /** 32 | * an index of the original data stream before the union. 33 | */ 34 | private int dataStreamTag; 35 | 36 | /** 37 | * an element from one of the original data streams. 38 | */ 39 | private Object element; 40 | 41 | public TaggedElement(int dataStreamTag, Object element) { 42 | this.dataStreamTag = dataStreamTag; 43 | this.element = element; 44 | } 45 | 46 | public int getDataStreamTag() { 47 | return dataStreamTag; 48 | } 49 | 50 | public void setDataStreamTag(int dataStreamTag) { 51 | this.dataStreamTag = dataStreamTag; 52 | } 53 | 54 | public Object getElement() { 55 | return element; 56 | } 57 | 58 | public void setElement(Object element) { 59 | this.element = element; 60 | } 61 | 62 | @Override 63 | public boolean equals(Object o) { 64 | if (this == o) { 65 | return true; 66 | } 67 | if (o == null || getClass() != o.getClass()) { 68 | return false; 69 | } 70 | TaggedElement that = (TaggedElement) o; 71 | return dataStreamTag == that.dataStreamTag 72 | && Objects.equals(element, that.element); 73 | } 74 | 75 | @Override 76 | public int hashCode() { 77 | int result = 1; 78 | result = 31 * result + (element == null ? 0 : element.hashCode()); 79 | result = 31 * result + Integer.hashCode(dataStreamTag); 80 | return result; 81 | } 82 | } 83 | -------------------------------------------------------------------------------- /da-streamingledger-examples/src/main/java/com/dataartisans/streamingledger/examples/simpletrade/generator/Throttler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.examples.simpletrade.generator; 18 | 19 | import static org.apache.flink.util.Preconditions.checkArgument; 20 | 21 | final class Throttler { 22 | 23 | private final long throttleBatchSize; 24 | private final long nanosPerBatch; 25 | 26 | private long endOfNextBatchNanos; 27 | private int currentBatch; 28 | 29 | Throttler(long maxRecordsPerSecond, int numberOfParallelSubtasks) { 30 | checkArgument(maxRecordsPerSecond == -1 || maxRecordsPerSecond > 0, 31 | "maxRecordsPerSecond must be positive or -1 (infinite)"); 32 | checkArgument(numberOfParallelSubtasks > 0, "numberOfParallelSubtasks must be greater than 0"); 33 | 34 | if (maxRecordsPerSecond == -1) { 35 | // unlimited speed 36 | throttleBatchSize = -1; 37 | nanosPerBatch = 0; 38 | endOfNextBatchNanos = System.nanoTime() + nanosPerBatch; 39 | currentBatch = 0; 40 | return; 41 | } 42 | final float ratePerSubtask = 43 | (float) maxRecordsPerSecond / numberOfParallelSubtasks; 44 | 45 | if (ratePerSubtask >= 10000) { 46 | // high rates: all throttling in intervals of 2ms 47 | throttleBatchSize = (int) ratePerSubtask / 500; 48 | nanosPerBatch = 2_000_000L; 49 | } 50 | else { 51 | throttleBatchSize = ((int) (ratePerSubtask / 20)) + 1; 52 | nanosPerBatch = ((int) (1_000_000_000L / ratePerSubtask)) * throttleBatchSize; 53 | } 54 | this.endOfNextBatchNanos = System.nanoTime() + nanosPerBatch; 55 | this.currentBatch = 0; 56 | } 57 | 58 | void throttle() throws InterruptedException { 59 | if (throttleBatchSize == -1) { 60 | return; 61 | } 62 | if (++currentBatch != throttleBatchSize) { 63 | return; 64 | } 65 | currentBatch = 0; 66 | 67 | final long now = System.nanoTime(); 68 | final int millisRemaining = (int) ((endOfNextBatchNanos - now) / 1_000_000); 69 | 70 | if (millisRemaining > 0) { 71 | endOfNextBatchNanos += nanosPerBatch; 72 | Thread.sleep(millisRemaining); 73 | } 74 | else { 75 | endOfNextBatchNanos = now + nanosPerBatch; 76 | } 77 | } 78 | 79 | } 80 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/test/java/com/dataartisans/streamingledger/sdk/common/union/UnionSerializerTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.common.union; 18 | 19 | import org.apache.flink.api.common.ExecutionConfig; 20 | import org.apache.flink.api.common.typeinfo.BasicTypeInfo; 21 | import org.apache.flink.api.common.typeutils.SerializerTestBase; 22 | import org.apache.flink.api.common.typeutils.TypeSerializer; 23 | import org.apache.flink.util.XORShiftRandom; 24 | 25 | import java.util.ArrayList; 26 | import java.util.List; 27 | 28 | /** 29 | * Test {@link UnionSerializer}. 30 | */ 31 | public class UnionSerializerTest extends SerializerTestBase { 32 | 33 | @Override 34 | protected TypeSerializer createSerializer() { 35 | List> serializers = new ArrayList<>(); 36 | ExecutionConfig config = new ExecutionConfig(); 37 | 38 | serializers.add(BasicTypeInfo.LONG_TYPE_INFO.createSerializer(config)); 39 | serializers.add(BasicTypeInfo.STRING_TYPE_INFO.createSerializer(config)); 40 | serializers.add(BasicTypeInfo.BOOLEAN_TYPE_INFO.createSerializer(config)); 41 | 42 | return new UnionSerializer(serializers); 43 | } 44 | 45 | @Override 46 | protected int getLength() { 47 | return -1; 48 | } 49 | 50 | @Override 51 | protected Class getTypeClass() { 52 | return TaggedElement.class; 53 | } 54 | 55 | @Override 56 | protected TaggedElement[] getTestData() { 57 | XORShiftRandom random = new XORShiftRandom(); 58 | 59 | TaggedElement[] data = new TaggedElement[100]; 60 | for (int i = 0; i < data.length; i++) { 61 | final int tag = random.nextInt(3); 62 | final Object element; 63 | switch (tag) { 64 | case 0: { 65 | element = random.nextLong(); 66 | break; 67 | } 68 | case 1: { 69 | byte[] bytes = new byte[random.nextInt(256)]; 70 | random.nextBytes(bytes); 71 | element = new String(bytes); 72 | break; 73 | } 74 | case 2: { 75 | element = random.nextBoolean(); 76 | break; 77 | } 78 | default: { 79 | element = null; 80 | } 81 | } 82 | data[i] = new TaggedElement(tag, element); 83 | } 84 | return data; 85 | } 86 | } 87 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/api/TransactionProcessFunction.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.api; 18 | 19 | import org.apache.flink.api.common.functions.Function; 20 | 21 | import java.lang.annotation.ElementType; 22 | import java.lang.annotation.Retention; 23 | import java.lang.annotation.RetentionPolicy; 24 | import java.lang.annotation.Target; 25 | 26 | /** 27 | * TransactionProcessFunction. 28 | * 29 | * @param The type of the transaction input events. 30 | * @param The type of the transaction result events. 31 | */ 32 | public abstract class TransactionProcessFunction implements Function { 33 | 34 | // stable serialVersionUID for compatibility across code updates 35 | private static final long serialVersionUID = 1L; 36 | 37 | // ------------------------------------------------------------------------ 38 | 39 | /** 40 | * This interface marks the so called {@code 'process method'}, which processes the 41 | * transaction with the transaction event and the involved states. 42 | * 43 | *

This interface is declared within the TransactionProcessFunction class so that 44 | * users do not need to import anything when writing their process method. 45 | */ 46 | @Target(ElementType.METHOD) 47 | @Retention(RetentionPolicy.RUNTIME) 48 | public @interface ProcessTransaction { 49 | } 50 | 51 | // ------------------------------------------------------------------------ 52 | 53 | /** 54 | * Annotation for state access parameters in the process method. 55 | * 56 | *

This interface is declared within the TransactionProcessFunction class so that 57 | * users do not need to import anything when writing their process method. 58 | */ 59 | @Target(ElementType.PARAMETER) 60 | @Retention(RetentionPolicy.RUNTIME) 61 | public @interface State { 62 | 63 | /** 64 | * The value is the name of the state access. 65 | */ 66 | String value(); 67 | 68 | } 69 | 70 | // ------------------------------------------------------------------------ 71 | 72 | /** 73 | * Transaction Processing Context. 74 | */ 75 | public interface Context { 76 | 77 | /** 78 | * Emits a transaction result. 79 | */ 80 | void emit(T record); 81 | 82 | /** 83 | * Abort the current transaction. 84 | * 85 | *

Calling abort will undo any change made to a {@link StateAccess}, and will not cause a transaction result 86 | * to be emitted. 87 | */ 88 | void abort(); 89 | } 90 | } 91 | -------------------------------------------------------------------------------- /da-streamingledger-examples/src/main/java/com/dataartisans/streamingledger/examples/simpletrade/DepositEvent.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.examples.simpletrade; 18 | 19 | /** 20 | * A simple data object that describes a deposit event. 21 | */ 22 | public class DepositEvent { 23 | 24 | private String accountId; 25 | 26 | private String bookEntryId; 27 | 28 | private long accountTransfer; 29 | 30 | private long bookEntryTransfer; 31 | 32 | /** 33 | * Creates a new DepositEvent. 34 | */ 35 | public DepositEvent( 36 | String accountId, 37 | String bookEntryId, 38 | long accountTransfer, 39 | long bookEntryTransfer) { 40 | this.accountId = accountId; 41 | this.bookEntryId = bookEntryId; 42 | this.accountTransfer = accountTransfer; 43 | this.bookEntryTransfer = bookEntryTransfer; 44 | } 45 | 46 | public DepositEvent() { 47 | } 48 | 49 | 50 | // ------------------------------------------------------------------------ 51 | // properties 52 | // ------------------------------------------------------------------------ 53 | 54 | public String getAccountId() { 55 | return accountId; 56 | } 57 | 58 | public void setAccountId(String accountId) { 59 | this.accountId = accountId; 60 | } 61 | 62 | public String getBookEntryId() { 63 | return bookEntryId; 64 | } 65 | 66 | public void setBookEntryId(String bookEntryId) { 67 | this.bookEntryId = bookEntryId; 68 | } 69 | 70 | public long getAccountTransfer() { 71 | return accountTransfer; 72 | } 73 | 74 | public void setAccountTransfer(long accountTransfer) { 75 | this.accountTransfer = accountTransfer; 76 | } 77 | 78 | public long getBookEntryTransfer() { 79 | return bookEntryTransfer; 80 | } 81 | 82 | public void setBookEntryTransfer(long bookEntryTransfer) { 83 | this.bookEntryTransfer = bookEntryTransfer; 84 | } 85 | 86 | // ------------------------------------------------------------------------ 87 | // miscellaneous 88 | // ------------------------------------------------------------------------ 89 | 90 | @Override 91 | public String toString() { 92 | return "DepositEvent {" 93 | + "accountId=" + accountId 94 | + ", bookEntryId=" + bookEntryId 95 | + ", accountTransfer=" + accountTransfer 96 | + ", bookEntryTransfer=" + bookEntryTransfer 97 | + '}'; 98 | } 99 | } 100 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/api/StateAccess.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.api; 18 | 19 | import com.dataartisans.streamingledger.sdk.api.StreamingLedger.State; 20 | import com.dataartisans.streamingledger.sdk.api.TransactionProcessFunction.Context; 21 | 22 | import java.util.function.Supplier; 23 | 24 | /** 25 | * This interface provides access to values of a {@link State}. 26 | * 27 | *

The state access binds a specific {@link State} value and 28 | * a specific argument to a {@link TransactionProcessFunction} process function. 29 | * 30 | *

For example:

31 |  * {@code
32 |  * void process(InputEvent e, .., @State("age") StateAccess age) { .. }
33 |  * }
34 |  * 
35 | * Would bind {@code age} to a specific value (as identified by the input event {@code e}). 36 | * Calling {@link #read()} would return the bound value, and calling {@link #write(Object)} would 37 | * replace the value with the supplied argument. 38 | * 39 | *

Any change to this state access would made visible to other transactions, atomically as soon as the process 40 | * function completes, unless explicitly aborted via {@link Context#abort()}. 41 | * 42 | * @param value parameter type. 43 | */ 44 | public interface StateAccess { 45 | 46 | /** 47 | * Reads a value. 48 | * 49 | * @return the value bound to this state access. 50 | * @throws StateAccessException if this state access is not readable. i.e. it is not one of {@link AccessType#READ} 51 | * or {@link AccessType#READ_WRITE}. 52 | */ 53 | T read() throws StateAccessException; 54 | 55 | /** 56 | * Reads a value by replacing missing values with a default supplier. 57 | * 58 | * @param defaultSupplier a default value supplier. 59 | * @return the value bound to this state access or default otherwise. 60 | * @throws StateAccessException if this state access is not readable. (only {@link AccessType#WRITE}) 61 | */ 62 | default T readOr(Supplier defaultSupplier) throws StateAccessException { 63 | T read = read(); 64 | if (read != null) { 65 | return read; 66 | } 67 | return defaultSupplier.get(); 68 | } 69 | 70 | /** 71 | * Writes a value. 72 | * 73 | * @param newValue the value to bind with this state access. 74 | * @throws StateAccessException if this state access is not writeable. (only {@link AccessType#READ}). 75 | */ 76 | void write(T newValue) throws StateAccessException; 77 | 78 | /** 79 | * Deletes a value. 80 | * 81 | *

Please note: that it would also delete the key associated with this value. 82 | * 83 | * @throws StateAccessException if this state access is not writeable. (only {@link AccessType#READ}). 84 | */ 85 | void delete() throws StateAccessException; 86 | 87 | /** 88 | * @return the {@link State} name that this state access is referencing. 89 | */ 90 | String getStateName(); 91 | 92 | /** 93 | * @return the bind name of this state access. 94 | */ 95 | String getStateAccessName(); 96 | } 97 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/spi/StreamingLedgerSpec.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.spi; 18 | 19 | import org.apache.flink.api.common.typeinfo.TypeInformation; 20 | 21 | import com.dataartisans.streamingledger.sdk.api.StreamingLedger; 22 | import com.dataartisans.streamingledger.sdk.api.StreamingLedger.StateAccessSpec; 23 | import com.dataartisans.streamingledger.sdk.api.TransactionProcessFunction; 24 | 25 | import java.io.Serializable; 26 | import java.util.ArrayList; 27 | import java.util.Collections; 28 | import java.util.List; 29 | import java.util.Objects; 30 | 31 | import static java.util.Objects.requireNonNull; 32 | 33 | /** 34 | * A {@code StreamingLedgerSpec} contains all the necessary information gathered at 35 | * {@link StreamingLedger#usingStream} to execute the process function at runtime. 36 | * 37 | * @param The type of the transaction data events. 38 | * @param The type of the transaction data results. 39 | */ 40 | public final class StreamingLedgerSpec implements Serializable { 41 | 42 | private static final long serialVersionUID = 1; 43 | 44 | // ------------------------------------------------------------------------ 45 | // Properties 46 | // ------------------------------------------------------------------------ 47 | 48 | public final TransactionProcessFunction processFunction; 49 | public final String processMethodName; 50 | public final List> stateBindings; 51 | public final TypeInformation inputType; 52 | public final TypeInformation resultType; 53 | 54 | StreamingLedgerSpec( 55 | TransactionProcessFunction processFunction, 56 | String processMethodName, 57 | List> stateBindings, 58 | TypeInformation inputType, 59 | TypeInformation resultType) { 60 | this.processFunction = requireNonNull(processFunction); 61 | this.processMethodName = requireNonNull(processMethodName); 62 | this.stateBindings = Collections.unmodifiableList(new ArrayList<>(stateBindings)); 63 | this.inputType = requireNonNull(inputType); 64 | this.resultType = requireNonNull(resultType); 65 | } 66 | 67 | @Override 68 | public boolean equals(Object o) { 69 | if (this == o) { 70 | return true; 71 | } 72 | if (o == null || getClass() != o.getClass()) { 73 | return false; 74 | } 75 | StreamingLedgerSpec that = (StreamingLedgerSpec) o; 76 | return Objects.equals(processFunction, that.processFunction) 77 | && Objects.equals(processMethodName, that.processMethodName) 78 | && Objects.equals(stateBindings, that.stateBindings) 79 | && Objects.equals(inputType, that.inputType) 80 | && Objects.equals(resultType, that.resultType); 81 | } 82 | 83 | @Override 84 | public int hashCode() { 85 | return Objects.hash(processFunction, processMethodName, stateBindings, inputType, resultType); 86 | } 87 | } 88 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/common/union/UnionTypeInfo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.common.union; 18 | 19 | import org.apache.flink.api.common.ExecutionConfig; 20 | import org.apache.flink.api.common.typeinfo.TypeInformation; 21 | import org.apache.flink.api.common.typeutils.TypeSerializer; 22 | 23 | import java.io.Serializable; 24 | import java.util.ArrayList; 25 | import java.util.List; 26 | import java.util.Objects; 27 | 28 | import static java.util.Objects.requireNonNull; 29 | 30 | final class UnionTypeInfo extends TypeInformation implements Serializable { 31 | 32 | private static final long serialVersionUID = 1; 33 | 34 | private final List> underlyingTypes; 35 | 36 | UnionTypeInfo(List> underlyingTypes) { 37 | requireNonNull(underlyingTypes); 38 | this.underlyingTypes = underlyingTypes; 39 | } 40 | 41 | @Override 42 | public boolean isBasicType() { 43 | return false; 44 | } 45 | 46 | @Override 47 | public boolean isTupleType() { 48 | return false; 49 | } 50 | 51 | @Override 52 | public int getArity() { 53 | return 1; 54 | } 55 | 56 | @Override 57 | public int getTotalFields() { 58 | return 1; 59 | } 60 | 61 | @Override 62 | public Class getTypeClass() { 63 | return TaggedElement.class; 64 | } 65 | 66 | @Override 67 | public boolean isKeyType() { 68 | return false; 69 | } 70 | 71 | @Override 72 | public TypeSerializer createSerializer(ExecutionConfig config) { 73 | List> underlyingSerializers = new ArrayList<>(underlyingTypes.size()); 74 | for (TypeInformation underlyingType : underlyingTypes) { 75 | TypeSerializer serializer = underlyingType.createSerializer(config); 76 | underlyingSerializers.add(serializer); 77 | } 78 | return new UnionSerializer(underlyingSerializers); 79 | } 80 | 81 | @Override 82 | public String toString() { 83 | StringBuilder sb = new StringBuilder(); 84 | sb.append("UnionTaggedSerializer {"); 85 | final int size = underlyingTypes.size(); 86 | for (int i = 0; i < size; i++) { 87 | sb.append(underlyingTypes.get(i).toString()); 88 | if (i < size - 1) { 89 | sb.append(", "); 90 | } 91 | } 92 | sb.append(" }"); 93 | return sb.toString(); 94 | } 95 | 96 | @Override 97 | public boolean equals(Object o) { 98 | if (this == o) { 99 | return true; 100 | } 101 | if (o == null || getClass() != o.getClass()) { 102 | return false; 103 | } 104 | UnionTypeInfo that = (UnionTypeInfo) o; 105 | return Objects.equals(underlyingTypes, that.underlyingTypes); 106 | } 107 | 108 | @Override 109 | public int hashCode() { 110 | return Objects.hash(underlyingTypes); 111 | } 112 | 113 | @Override 114 | public boolean canEqual(Object obj) { 115 | return obj instanceof UnionTypeInfo; 116 | } 117 | } 118 | -------------------------------------------------------------------------------- /da-streamingledger-examples/src/main/java/com/dataartisans/streamingledger/examples/simpletrade/generator/SyntheticSources.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.examples.simpletrade.generator; 18 | 19 | import org.apache.flink.api.common.typeinfo.TypeInformation; 20 | import org.apache.flink.streaming.api.datastream.DataStream; 21 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 22 | import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; 23 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 24 | import org.apache.flink.streaming.api.functions.ProcessFunction; 25 | import org.apache.flink.types.Either; 26 | import org.apache.flink.util.Collector; 27 | import org.apache.flink.util.OutputTag; 28 | 29 | import com.dataartisans.streamingledger.examples.simpletrade.DepositEvent; 30 | import com.dataartisans.streamingledger.examples.simpletrade.TransactionEvent; 31 | 32 | /** 33 | * Creates two synthetic sources for {@link DepositEvent} and {@link TransactionEvent}. 34 | */ 35 | public final class SyntheticSources { 36 | 37 | public final DataStream deposits; 38 | public final DataStream transactions; 39 | 40 | /** 41 | * Creates and adds two synthetic sources for {@link DepositEvent} and {@link TransactionEvent}. 42 | * 43 | * @param env the streaming environment to add the sources to. 44 | * @param recordsPerSecond the number of {@link TransactionEvent} per second to generate. 45 | * @return a {@link DataStream} for each event type generated. 46 | */ 47 | public static SyntheticSources create(StreamExecutionEnvironment env, int recordsPerSecond) { 48 | 49 | final DataStreamSource> depositsAndTransactions = env.addSource( 50 | new DepositsThenTransactionsSource(recordsPerSecond)); 51 | 52 | final OutputTag transactionsSideOutput = new OutputTag<>( 53 | "transactions side output", 54 | TypeInformation.of(TransactionEvent.class)); 55 | 56 | final SingleOutputStreamOperator deposits = depositsAndTransactions.process( 57 | new ProcessFunction, DepositEvent>() { 58 | 59 | @Override 60 | public void processElement( 61 | Either depositOrTransaction, 62 | Context context, 63 | Collector out) { 64 | 65 | if (depositOrTransaction.isLeft()) { 66 | out.collect(depositOrTransaction.left()); 67 | } 68 | else { 69 | context.output(transactionsSideOutput, depositOrTransaction.right()); 70 | } 71 | } 72 | }); 73 | 74 | final DataStream transactions = deposits.getSideOutput(transactionsSideOutput); 75 | 76 | return new SyntheticSources(deposits, transactions); 77 | } 78 | 79 | SyntheticSources(DataStream deposits, DataStream transactions) { 80 | this.deposits = deposits; 81 | this.transactions = transactions; 82 | } 83 | } 84 | 85 | -------------------------------------------------------------------------------- /da-streamingledger-examples/src/main/java/com/dataartisans/streamingledger/examples/simpletrade/TransactionResult.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.examples.simpletrade; 18 | 19 | import static java.util.Objects.requireNonNull; 20 | 21 | /** 22 | * Data type describing the result of a processed transaction. 23 | * It describes whether the transaction was successful as well as the resulting account balances. 24 | */ 25 | public class TransactionResult { 26 | 27 | private TransactionEvent transaction; 28 | 29 | private boolean success; 30 | 31 | private long newSourceAccountBalance; 32 | 33 | private long newTargetAccountBalance; 34 | 35 | /** 36 | * Creates a new transaction result. 37 | * 38 | * @param transaction The original transaction event. 39 | * @param success True, if the transaction was successful, false if not. 40 | * @param newSourceAccountBalance The resulting balance of the source account. 41 | * @param newTargetAccountBalance The resulting balance of the target account. 42 | */ 43 | public TransactionResult( 44 | TransactionEvent transaction, 45 | boolean success, 46 | long newSourceAccountBalance, 47 | long newTargetAccountBalance) { 48 | 49 | this.transaction = requireNonNull(transaction); 50 | this.success = success; 51 | this.newSourceAccountBalance = newSourceAccountBalance; 52 | this.newTargetAccountBalance = newTargetAccountBalance; 53 | } 54 | 55 | public TransactionResult() { 56 | } 57 | 58 | // ------------------------------------------------------------------------ 59 | // Properties 60 | // ------------------------------------------------------------------------ 61 | 62 | public TransactionEvent getTransaction() { 63 | return transaction; 64 | } 65 | 66 | public void setTransaction(TransactionEvent transaction) { 67 | this.transaction = transaction; 68 | } 69 | 70 | public boolean isSuccess() { 71 | return success; 72 | } 73 | 74 | public void setSuccess(boolean success) { 75 | this.success = success; 76 | } 77 | 78 | public long getNewSourceAccountBalance() { 79 | return newSourceAccountBalance; 80 | } 81 | 82 | public void setNewSourceAccountBalance(long newSourceAccountBalance) { 83 | this.newSourceAccountBalance = newSourceAccountBalance; 84 | } 85 | 86 | public long getNewTargetAccountBalance() { 87 | return newTargetAccountBalance; 88 | } 89 | 90 | public void setNewTargetAccountBalance(long newTargetAccountBalance) { 91 | this.newTargetAccountBalance = newTargetAccountBalance; 92 | } 93 | 94 | // ------------------------------------------------------------------------ 95 | // Miscellaneous 96 | // ------------------------------------------------------------------------ 97 | 98 | @Override 99 | public String toString() { 100 | return "TransactionResult {" 101 | + "transaction=" + transaction 102 | + ", success=" + success 103 | + ", newSourceAccountBalance=" + newSourceAccountBalance 104 | + ", newTargetAccountBalance=" + newTargetAccountBalance 105 | + '}'; 106 | } 107 | } 108 | -------------------------------------------------------------------------------- /da-streamingledger-runtime-serial/src/main/java/com/dataartisans/streamingledger/runtime/serial/SerialStateAccess.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.runtime.serial; 18 | 19 | import org.apache.flink.api.common.state.MapState; 20 | import org.apache.flink.api.java.functions.KeySelector; 21 | 22 | import com.dataartisans.streamingledger.sdk.api.AccessType; 23 | import com.dataartisans.streamingledger.sdk.api.StateAccess; 24 | import com.dataartisans.streamingledger.sdk.api.StateAccessException; 25 | import com.dataartisans.streamingledger.sdk.api.StateNotReadableException; 26 | import com.dataartisans.streamingledger.sdk.api.StateNotWritableException; 27 | import com.dataartisans.streamingledger.sdk.api.StreamingLedger.StateAccessSpec; 28 | 29 | import static java.util.Objects.requireNonNull; 30 | 31 | final class SerialStateAccess implements StateAccess { 32 | private final StateAccessSpec spec; 33 | private final MapState state; 34 | 35 | private final KeySelector keySelector; 36 | private final boolean writeOnly; 37 | private final boolean readOnly; 38 | 39 | private K key; 40 | private V value; 41 | private boolean changed; 42 | 43 | SerialStateAccess(StateAccessSpec spec, MapState state) { 44 | this.spec = requireNonNull(spec); 45 | this.state = requireNonNull(state); 46 | this.keySelector = requireNonNull(spec.keyAccess); 47 | this.writeOnly = spec.accessType == AccessType.WRITE; 48 | this.readOnly = spec.accessType == AccessType.READ; 49 | } 50 | 51 | @Override 52 | public V read() throws StateAccessException { 53 | if (writeOnly) { 54 | throw new StateNotReadableException(this); 55 | } 56 | return value; 57 | } 58 | 59 | @Override 60 | public void write(V newValue) throws StateAccessException { 61 | if (readOnly) { 62 | throw new StateNotWritableException(this); 63 | } 64 | this.value = newValue; 65 | this.changed = true; 66 | } 67 | 68 | @Override 69 | public void delete() throws StateAccessException { 70 | if (readOnly) { 71 | throw new StateNotWritableException(this); 72 | } 73 | this.value = null; 74 | this.changed = true; 75 | } 76 | 77 | @Override 78 | public String getStateName() { 79 | return spec.state.getName(); 80 | } 81 | 82 | @Override 83 | public String getStateAccessName() { 84 | return spec.bindName; 85 | } 86 | 87 | // ----------------------------------------------------- 88 | // For internal use by SerialTransactor 89 | // ----------------------------------------------------- 90 | 91 | void prepare(InT input) throws Exception { 92 | final K key = keySelector.getKey(input); 93 | this.key = key; 94 | this.changed = false; 95 | if (!writeOnly) { 96 | this.value = state.get(key); 97 | } 98 | } 99 | 100 | void commit(boolean wasAborted) throws Exception { 101 | if (!changed || wasAborted) { 102 | return; 103 | } 104 | if (value == null) { 105 | state.remove(key); 106 | } 107 | else { 108 | state.put(key, value); 109 | } 110 | } 111 | } 112 | -------------------------------------------------------------------------------- /tools/intellij/inspections.xml: -------------------------------------------------------------------------------- 1 | 16 | 17 | 18 | 67 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/spi/StreamingLedgerRuntimeLoader.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.spi; 18 | 19 | import org.apache.flink.util.FlinkRuntimeException; 20 | 21 | import java.util.Iterator; 22 | import java.util.ServiceLoader; 23 | import java.util.concurrent.locks.ReentrantLock; 24 | 25 | import javax.annotation.Nullable; 26 | import javax.annotation.concurrent.GuardedBy; 27 | 28 | /** 29 | * A loader utility to find and load the {@link StreamingLedgerRuntimeProvider} to 30 | * execute the program with. 31 | */ 32 | public class StreamingLedgerRuntimeLoader { 33 | 34 | /** 35 | * The lock to make sure only one transaction runtime provider is ever loaded. 36 | */ 37 | private static final ReentrantLock LOCK = new ReentrantLock(); 38 | 39 | /** 40 | * The transaction runtime provider, null if not yet loaded. 41 | */ 42 | @Nullable 43 | private static StreamingLedgerRuntimeProvider runtimeProvider; 44 | 45 | /** 46 | * Gets the runtime provider to execute the transactions. This method will load 47 | * the runtime provider if it has not been loaded before. 48 | * 49 | *

If more than one runtime provider is found, this method throws an 50 | * exception, because it cannot determine which provider to use. 51 | */ 52 | public static StreamingLedgerRuntimeProvider getRuntimeProvider() { 53 | LOCK.lock(); 54 | try { 55 | if (runtimeProvider == null) { 56 | runtimeProvider = loadRuntimeProvider(); 57 | } 58 | return runtimeProvider; 59 | } 60 | finally { 61 | LOCK.unlock(); 62 | } 63 | } 64 | 65 | @GuardedBy("LOCK") 66 | private static StreamingLedgerRuntimeProvider loadRuntimeProvider() { 67 | try { 68 | ServiceLoader serviceLoader = 69 | ServiceLoader.load(StreamingLedgerRuntimeProvider.class); 70 | 71 | Iterator iter = serviceLoader.iterator(); 72 | 73 | // find the first service implementation 74 | StreamingLedgerRuntimeProvider firstProvider; 75 | if (iter.hasNext()) { 76 | firstProvider = iter.next(); 77 | } 78 | else { 79 | throw new FlinkRuntimeException("No StreamingLedgerRuntimeProvider found. " 80 | + "Please make sure you have a transaction runtime implementation in the classpath."); 81 | } 82 | 83 | // check if there is more than one service implementation 84 | if (iter.hasNext()) { 85 | String secondServiceName = "(could not load service implementation)"; 86 | try { 87 | secondServiceName = iter.next().getClass().getName(); 88 | } 89 | catch (Throwable ignored) { 90 | } 91 | 92 | throw new FlinkRuntimeException("Ambiguous: Found more than one StreamingLedgerRuntimeProvider: " 93 | + firstProvider.getClass().getName() + " and " + secondServiceName); 94 | } 95 | 96 | return firstProvider; 97 | } 98 | catch (FlinkRuntimeException e) { 99 | // simply propagate without further wrapping, for simplicity 100 | throw e; 101 | } 102 | catch (Throwable t) { 103 | throw new FlinkRuntimeException("Could not load StreamingLedgerRuntimeProvider", t); 104 | } 105 | } 106 | } 107 | -------------------------------------------------------------------------------- /da-streamingledger-runtime-serial/src/main/java/com/dataartisans/streamingledger/runtime/serial/SerialStreamingLedgerRuntimeProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.runtime.serial; 18 | 19 | import org.apache.flink.streaming.api.datastream.DataStream; 20 | import org.apache.flink.streaming.api.datastream.KeyedStream; 21 | import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; 22 | import org.apache.flink.util.OutputTag; 23 | 24 | import com.dataartisans.streamingledger.sdk.api.StreamingLedger.ResultStreams; 25 | import com.dataartisans.streamingledger.sdk.common.union.TaggedElement; 26 | import com.dataartisans.streamingledger.sdk.common.union.Union; 27 | import com.dataartisans.streamingledger.sdk.spi.InputAndSpec; 28 | import com.dataartisans.streamingledger.sdk.spi.StreamingLedgerRuntimeProvider; 29 | import com.dataartisans.streamingledger.sdk.spi.StreamingLedgerSpec; 30 | 31 | import java.util.ArrayList; 32 | import java.util.HashMap; 33 | import java.util.List; 34 | import java.util.Map; 35 | import java.util.stream.Collectors; 36 | 37 | /** 38 | * A {@link StreamingLedgerRuntimeProvider} for the default serial transaction implementation. 39 | */ 40 | public final class SerialStreamingLedgerRuntimeProvider implements StreamingLedgerRuntimeProvider { 41 | 42 | private static DataStream union(List> inputAndSpecs) { 43 | List> inputs = inputAndSpecs.stream() 44 | .map(inputAndSpec -> inputAndSpec.inputStream) 45 | .collect(Collectors.toList()); 46 | 47 | return Union.apply(inputs); 48 | } 49 | 50 | // ------------------------------------------------------------------------------------------------------- 51 | // Helpers 52 | // ------------------------------------------------------------------------------------------------------- 53 | 54 | private static List> createSideOutputTags(List> specs) { 55 | List> outputTags = new ArrayList<>(); 56 | for (InputAndSpec streamWithSpec : specs) { 57 | OutputTag outputTag = new OutputTag<>(streamWithSpec.streamName, streamWithSpec.streamSpec.resultType); 58 | outputTags.add(outputTag); 59 | } 60 | return outputTags; 61 | } 62 | 63 | private static List> specs(List> inputAndSpecs) { 64 | return inputAndSpecs.stream() 65 | .map(inputAndSpec -> inputAndSpec.streamSpec) 66 | .collect(Collectors.toList()); 67 | } 68 | 69 | @Override 70 | public ResultStreams translate(String name, List> streamLedgerSpecs) { 71 | List> sideOutputTags = createSideOutputTags(streamLedgerSpecs); 72 | 73 | // the input stream is a union of different streams. 74 | KeyedStream input = union(streamLedgerSpecs) 75 | .keyBy(unused -> true); 76 | 77 | // main pipeline 78 | String serialTransactorName = "SerialTransactor(" + name + ")"; 79 | SingleOutputStreamOperator resultStream = input 80 | .process(new SerialTransactor(specs(streamLedgerSpecs), sideOutputTags)) 81 | .name(serialTransactorName) 82 | .uid(serialTransactorName + "___SERIAL_TX") 83 | .forceNonParallel() 84 | .returns(Void.class); 85 | 86 | // gather the sideOutputs. 87 | Map> output = new HashMap<>(); 88 | for (OutputTag outputTag : sideOutputTags) { 89 | DataStream rs = resultStream.getSideOutput(outputTag); 90 | output.put(outputTag.getId(), rs); 91 | } 92 | return new ResultStreams(output); 93 | } 94 | } 95 | -------------------------------------------------------------------------------- /da-streamingledger-runtime-serial/src/main/java/com/dataartisans/streamingledger/runtime/serial/SerialTransactor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.runtime.serial; 18 | 19 | import org.apache.flink.api.common.functions.RuntimeContext; 20 | import org.apache.flink.configuration.Configuration; 21 | import org.apache.flink.streaming.api.functions.ProcessFunction; 22 | import org.apache.flink.util.Collector; 23 | import org.apache.flink.util.OutputTag; 24 | 25 | import com.dataartisans.streamingledger.sdk.common.union.TaggedElement; 26 | import com.dataartisans.streamingledger.sdk.spi.StreamingLedgerSpec; 27 | 28 | import java.util.List; 29 | import java.util.stream.Collectors; 30 | 31 | import static java.util.Objects.requireNonNull; 32 | 33 | final class SerialTransactor extends ProcessFunction { 34 | 35 | private static final long serialVersionUID = 1; 36 | 37 | private final List> specs; 38 | private final List> sideOutputs; 39 | private final SideOutputContext collector; 40 | 41 | private transient SingleStreamSerialTransactor[] transactors; 42 | 43 | SerialTransactor(List> specs, List> sideOutputTags) { 44 | this.specs = requireNonNull(specs); 45 | this.sideOutputs = castOutputTags(sideOutputTags); 46 | this.collector = new SideOutputContext<>(); 47 | } 48 | 49 | @SuppressWarnings("unchecked") 50 | private static SingleStreamSerialTransactor[] newSingleStreamSerialTransactorArray(int n) { 51 | return (SingleStreamSerialTransactor[]) new SingleStreamSerialTransactor[n]; 52 | } 53 | 54 | @SuppressWarnings("unchecked") 55 | private static SingleStreamSerialTransactor singleStreamSerialTransactorFromSpec( 56 | StreamingLedgerSpec aSpec, 57 | OutputTag aTag, 58 | SideOutputContext collector, 59 | RuntimeContext runtimeContext) { 60 | 61 | OutputTag outputTag = (OutputTag) aTag; 62 | StreamingLedgerSpec spec = (StreamingLedgerSpec) aSpec; 63 | return new SingleStreamSerialTransactor<>(spec, outputTag, collector, runtimeContext); 64 | } 65 | 66 | @SuppressWarnings("unchecked") 67 | private static List> castOutputTags(List> sideOutputTags) { 68 | return sideOutputTags.stream() 69 | .map(outputTag -> (OutputTag) outputTag) 70 | .collect(Collectors.toList()); 71 | } 72 | 73 | @Override 74 | public void open(Configuration parameters) throws Exception { 75 | super.open(parameters); 76 | final RuntimeContext runtimeContext = getRuntimeContext(); 77 | 78 | SingleStreamSerialTransactor[] transactors = 79 | newSingleStreamSerialTransactorArray(specs.size()); 80 | 81 | // initialize the individual transactors 82 | for (int streamTag = 0; streamTag < specs.size(); streamTag++) { 83 | StreamingLedgerSpec aSpec = specs.get(streamTag); 84 | OutputTag aTag = sideOutputs.get(streamTag); 85 | transactors[streamTag] = singleStreamSerialTransactorFromSpec(aSpec, aTag, collector, runtimeContext); 86 | } 87 | this.transactors = transactors; 88 | } 89 | 90 | @Override 91 | public void processElement(TaggedElement input, Context context, Collector unused) throws Exception { 92 | collector.setContext(context); 93 | 94 | final int streamTag = input.getDataStreamTag(); 95 | SingleStreamSerialTransactor transactor = transactors[streamTag]; 96 | transactor.apply(input.getElement()); 97 | } 98 | } 99 | -------------------------------------------------------------------------------- /da-streamingledger-examples/src/main/java/com/dataartisans/streamingledger/examples/simpletrade/TransactionEvent.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.examples.simpletrade; 18 | 19 | /** 20 | * A simple data object that describes a transaction. 21 | */ 22 | public class TransactionEvent { 23 | 24 | private String sourceAccountId; 25 | 26 | private String targetAccountId; 27 | 28 | private String sourceBookEntryId; 29 | 30 | private String targetBookEntryId; 31 | 32 | private long accountTransfer; 33 | 34 | private long bookEntryTransfer; 35 | 36 | private long minAccountBalance; 37 | 38 | /** 39 | * Creates a new TransactionEvent for the given accounts and book entries. 40 | */ 41 | public TransactionEvent( 42 | String sourceAccountId, 43 | String targetAccountId, 44 | String sourceBookEntryId, 45 | String targetBookEntryId, 46 | long accountTransfer, 47 | long bookEntryTransfer, 48 | long minAccountBalance) { 49 | 50 | this.sourceAccountId = sourceAccountId; 51 | this.targetAccountId = targetAccountId; 52 | this.sourceBookEntryId = sourceBookEntryId; 53 | this.targetBookEntryId = targetBookEntryId; 54 | this.accountTransfer = accountTransfer; 55 | this.bookEntryTransfer = bookEntryTransfer; 56 | this.minAccountBalance = minAccountBalance; 57 | } 58 | 59 | public TransactionEvent() { 60 | } 61 | 62 | // ------------------------------------------------------------------------ 63 | // properties 64 | // ------------------------------------------------------------------------ 65 | 66 | public String getSourceAccountId() { 67 | return sourceAccountId; 68 | } 69 | 70 | public void setSourceAccountId(String sourceAccountId) { 71 | this.sourceAccountId = sourceAccountId; 72 | } 73 | 74 | public String getTargetAccountId() { 75 | return targetAccountId; 76 | } 77 | 78 | public void setTargetAccountId(String targetAccountId) { 79 | this.targetAccountId = targetAccountId; 80 | } 81 | 82 | public String getSourceBookEntryId() { 83 | return sourceBookEntryId; 84 | } 85 | 86 | public void setSourceBookEntryId(String sourceBookEntryId) { 87 | this.sourceBookEntryId = sourceBookEntryId; 88 | } 89 | 90 | public String getTargetBookEntryId() { 91 | return targetBookEntryId; 92 | } 93 | 94 | public void setTargetBookEntryId(String targetBookEntryId) { 95 | this.targetBookEntryId = targetBookEntryId; 96 | } 97 | 98 | public long getAccountTransfer() { 99 | return accountTransfer; 100 | } 101 | 102 | public void setAccountTransfer(long accountTransfer) { 103 | this.accountTransfer = accountTransfer; 104 | } 105 | 106 | public long getBookEntryTransfer() { 107 | return bookEntryTransfer; 108 | } 109 | 110 | public void setBookEntryTransfer(long bookEntryTransfer) { 111 | this.bookEntryTransfer = bookEntryTransfer; 112 | } 113 | 114 | public long getMinAccountBalance() { 115 | return minAccountBalance; 116 | } 117 | 118 | public void setMinAccountBalance(long minAccountBalance) { 119 | this.minAccountBalance = minAccountBalance; 120 | } 121 | 122 | // ------------------------------------------------------------------------ 123 | // miscellaneous 124 | // ------------------------------------------------------------------------ 125 | 126 | @Override 127 | public String toString() { 128 | return "TransactionEvent {" 129 | + "sourceAccountId=" + sourceAccountId 130 | + ", targetAccountId=" + targetAccountId 131 | + ", sourceBookEntryId=" + sourceBookEntryId 132 | + ", targetBookEntryId=" + targetBookEntryId 133 | + ", accountTransfer=" + accountTransfer 134 | + ", bookEntryTransfer=" + bookEntryTransfer 135 | + ", minAccountBalance=" + minAccountBalance 136 | + '}'; 137 | } 138 | } 139 | -------------------------------------------------------------------------------- /da-streamingledger-runtime-serial/src/main/java/com/dataartisans/streamingledger/runtime/serial/SingleStreamSerialTransactor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.runtime.serial; 18 | 19 | import org.apache.flink.api.common.functions.RuntimeContext; 20 | import org.apache.flink.api.common.state.MapState; 21 | import org.apache.flink.api.common.state.MapStateDescriptor; 22 | import org.apache.flink.util.OutputTag; 23 | 24 | import com.dataartisans.streamingledger.sdk.api.StreamingLedger.StateAccessSpec; 25 | import com.dataartisans.streamingledger.sdk.common.reflection.ByteBuddyProcessFunctionInvoker; 26 | import com.dataartisans.streamingledger.sdk.common.reflection.ProcessFunctionInvoker; 27 | import com.dataartisans.streamingledger.sdk.spi.StreamingLedgerSpec; 28 | 29 | import static java.util.Objects.requireNonNull; 30 | 31 | /** 32 | * A serially executing transactor. 33 | * 34 | * @param transaction event type 35 | * @param transaction result type 36 | */ 37 | final class SingleStreamSerialTransactor { 38 | private final ProcessFunctionInvoker userFunction; 39 | private final SerialStateAccess[] accesses; 40 | private final OutputTag sideOutputTag; 41 | private final SideOutputContext context; 42 | 43 | SingleStreamSerialTransactor( 44 | StreamingLedgerSpec spec, 45 | OutputTag sideOutputTag, 46 | SideOutputContext context, 47 | RuntimeContext runtimeContext) { 48 | 49 | requireNonNull(spec); 50 | requireNonNull(sideOutputTag); 51 | requireNonNull(runtimeContext); 52 | 53 | this.context = context; 54 | this.sideOutputTag = sideOutputTag; 55 | this.accesses = createStateAccessesFromSpec(spec, runtimeContext); 56 | this.userFunction = ByteBuddyProcessFunctionInvoker.create(spec); 57 | } 58 | 59 | @SuppressWarnings("unchecked") 60 | private static SerialStateAccess[] newStateAccessArray(int n) { 61 | return (SerialStateAccess[]) new SerialStateAccess[n]; 62 | } 63 | 64 | // -------------------------------------------------------------------------------------------------------- 65 | // Internal helpers 66 | // -------------------------------------------------------------------------------------------------------- 67 | 68 | private static MapState fromSpec(StateAccessSpec spec, RuntimeContext context) { 69 | MapStateDescriptor descriptor = new MapStateDescriptor<>( 70 | spec.state.getName(), 71 | spec.state.getKeyType(), 72 | spec.state.getValueType()); 73 | 74 | return context.getMapState(descriptor); 75 | } 76 | 77 | void apply(InT input) throws Exception { 78 | context.setOutputTag(sideOutputTag); 79 | // 80 | // prepare 81 | // 82 | for (SerialStateAccess access : accesses) { 83 | access.prepare(input); 84 | } 85 | context.prepare(); 86 | // 87 | // invoke 88 | // 89 | userFunction.invoke(input, context, accesses); 90 | // 91 | // commit or abort 92 | // 93 | for (SerialStateAccess access : accesses) { 94 | access.commit(context.wasAborted()); 95 | } 96 | context.emitChanges(); 97 | } 98 | 99 | private SerialStateAccess[] createStateAccessesFromSpec( 100 | StreamingLedgerSpec spec, 101 | RuntimeContext ctx) { 102 | 103 | SerialStateAccess[] accesses = newStateAccessArray(spec.stateBindings.size()); 104 | 105 | for (int i = 0; i < accesses.length; i++) { 106 | final StateAccessSpec accessSpec = spec.stateBindings.get(i); 107 | 108 | MapState state = fromSpec(accessSpec, ctx); 109 | 110 | @SuppressWarnings({"rawtypes", "unchecked"}) 111 | SerialStateAccess serialStateAccess = new SerialStateAccess(accessSpec, state); 112 | accesses[i] = serialStateAccess; 113 | } 114 | return accesses; 115 | } 116 | } 117 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/common/union/Union.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.common.union; 18 | 19 | import org.apache.flink.api.common.functions.RichMapFunction; 20 | import org.apache.flink.api.common.typeinfo.TypeInformation; 21 | import org.apache.flink.configuration.Configuration; 22 | import org.apache.flink.streaming.api.datastream.DataStream; 23 | 24 | import java.util.ArrayList; 25 | import java.util.List; 26 | import java.util.stream.Collectors; 27 | 28 | import static org.apache.flink.util.Preconditions.checkArgument; 29 | 30 | /** 31 | * Union differently typed {@link DataStream}s. 32 | */ 33 | public final class Union { 34 | 35 | private Union() { 36 | } 37 | 38 | /** 39 | * Union differently typed {@link DataStream}s into single {@code DataStream}. 40 | * 41 | *

The resulting {@code DataStream} is of type {@link TaggedElement} where 42 | * {@link TaggedElement#getDataStreamTag()} corresponds to the list position of the source {@code DataStream} in 43 | * {@code inputs} that produced that element, and {@link TaggedElement#getElement()} is the element produced. 44 | * 45 | * @param inputs the input data streams to union. 46 | * @return a {@code DataStream} that corresponds to the union of all the input {@link DataStream}s 47 | */ 48 | public static DataStream apply(List> inputs) { 49 | checkArgument(!inputs.isEmpty(), "union requires at least one input data stream."); 50 | 51 | List> taggedInputs = tagInputStreams(inputs); 52 | if (taggedInputs.size() == 1) { 53 | return taggedInputs.get(0); 54 | } 55 | DataStream first = taggedInputs.get(0); 56 | List> restList = taggedInputs.subList(1, taggedInputs.size()); 57 | 58 | @SuppressWarnings({"unchecked", "raw"}) 59 | DataStream[] restArray = (DataStream[]) new DataStream[restList.size()]; 60 | DataStream[] rest = restList.toArray(restArray); 61 | return first.union(rest); 62 | } 63 | 64 | // ------------------------------------------------------------------------------------------------------- 65 | // Internal Helpers 66 | // ------------------------------------------------------------------------------------------------------- 67 | 68 | private static List> tagInputStreams(List> inputs) { 69 | TypeInformation typeInfo = createUnionTypeInfo(inputs); 70 | 71 | List> taggedInputs = new ArrayList<>(); 72 | int dataStreamIndex = 0; 73 | for (DataStream input : inputs) { 74 | 75 | final DataStream transformed = input 76 | .map(new TaggingMap<>(dataStreamIndex)) 77 | .returns(typeInfo); 78 | 79 | dataStreamIndex++; 80 | taggedInputs.add(transformed); 81 | } 82 | return taggedInputs; 83 | } 84 | 85 | private static UnionTypeInfo createUnionTypeInfo(List> inputs) { 86 | List> underlyingTypes = inputs.stream() 87 | .map(DataStream::getType) 88 | .collect(Collectors.toList()); 89 | 90 | return new UnionTypeInfo(underlyingTypes); 91 | } 92 | 93 | // ------------------------------------------------------------------------------------------------------- 94 | // Nested Class 95 | // ------------------------------------------------------------------------------------------------------- 96 | 97 | private static final class TaggingMap extends RichMapFunction { 98 | 99 | private static final long serialVersionUID = 1; 100 | 101 | private final int dataStreamIndex; 102 | private transient TaggedElement union; 103 | 104 | TaggingMap(int dataStreamIndex) { 105 | this.dataStreamIndex = dataStreamIndex; 106 | } 107 | 108 | @Override 109 | public TaggedElement map(InT element) { 110 | union.setElement(element); 111 | return union; 112 | } 113 | 114 | @Override 115 | public void open(Configuration parameters) throws Exception { 116 | super.open(parameters); 117 | this.union = new TaggedElement(this.dataStreamIndex, null); 118 | } 119 | } 120 | } 121 | -------------------------------------------------------------------------------- /da-streamingledger-examples/src/main/java/com/dataartisans/streamingledger/examples/simpletrade/generator/DepositsThenTransactionsSource.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.examples.simpletrade.generator; 18 | 19 | import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; 20 | import org.apache.flink.types.Either; 21 | 22 | import com.dataartisans.streamingledger.examples.simpletrade.DepositEvent; 23 | import com.dataartisans.streamingledger.examples.simpletrade.TransactionEvent; 24 | 25 | import java.util.SplittableRandom; 26 | 27 | import static org.apache.flink.util.Preconditions.checkArgument; 28 | 29 | /** 30 | * A random data generator with data rate throttling logic. 31 | * 32 | *

This source emits two kinds of events {@link DepositEvent} and a {@link TransactionEvent}. First this source emits 33 | * deposit events for each account and book entry, and then starts emitting random transaction events while not 34 | * canceled. 35 | */ 36 | final class DepositsThenTransactionsSource extends RichParallelSourceFunction> { 37 | 38 | private static final int NUM_ROWS = 1_000_000; 39 | private static final String ACCOUNT_ID_PREFIX = "ACCT-"; 40 | private static final String BOOK_ENTRY_ID_PREFIX = "BOOK-"; 41 | private static final long MAX_ACCOUNT_TRANSFER = 10_000; 42 | private static final long MAX_BOOK_TRANSFER = 1_000; 43 | private static final long MIN_BALANCE = 0; 44 | 45 | private static final long serialVersionUID = 1L; 46 | 47 | private final int maxRecordsPerSecond; 48 | 49 | private volatile boolean running = true; 50 | 51 | DepositsThenTransactionsSource(int maxRecordsPerSecond) { 52 | checkArgument(maxRecordsPerSecond == -1 || maxRecordsPerSecond > 0, 53 | "maxRecordsPerSecond must be positive or -1 (infinite)"); 54 | this.maxRecordsPerSecond = maxRecordsPerSecond; 55 | } 56 | 57 | @Override 58 | public void run(SourceContext> context) throws Exception { 59 | final int numberOfParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks(); 60 | final int indexOfThisSubtask = getRuntimeContext().getIndexOfThisSubtask(); 61 | 62 | collectDeposits(context, indexOfThisSubtask, numberOfParallelSubtasks); 63 | collectTransactions(context, numberOfParallelSubtasks); 64 | } 65 | 66 | @Override 67 | public void cancel() { 68 | running = false; 69 | } 70 | 71 | private void collectDeposits( 72 | SourceContext> context, 73 | final int indexOfThisSubtask, 74 | final int numberOfParallelSubtasks) { 75 | 76 | final int startId = (indexOfThisSubtask * NUM_ROWS) / numberOfParallelSubtasks; 77 | final int endId = ((indexOfThisSubtask + 1) * NUM_ROWS) / numberOfParallelSubtasks; 78 | 79 | for (int i = startId; i < endId; i++) { 80 | String accountId = ACCOUNT_ID_PREFIX + i; 81 | String bookEntryId = BOOK_ENTRY_ID_PREFIX + i; 82 | 83 | DepositEvent event = new DepositEvent( 84 | accountId, 85 | bookEntryId, 86 | MAX_ACCOUNT_TRANSFER, 87 | MAX_BOOK_TRANSFER); 88 | 89 | context.collect(Either.Left(event)); 90 | } 91 | } 92 | 93 | private void collectTransactions( 94 | SourceContext> context, 95 | int numberOfParallelSubtasks) throws InterruptedException { 96 | 97 | SplittableRandom random = new SplittableRandom(); 98 | Throttler throttler = new Throttler(maxRecordsPerSecond, numberOfParallelSubtasks); 99 | while (running) { 100 | TransactionEvent event = randomTransactionEvent(random); 101 | context.collect(Either.Right(event)); 102 | throttler.throttle(); 103 | } 104 | } 105 | 106 | private TransactionEvent randomTransactionEvent(SplittableRandom rnd) { 107 | final long accountsTransfer = rnd.nextLong(MAX_ACCOUNT_TRANSFER); 108 | final long transfer = rnd.nextLong(MAX_BOOK_TRANSFER); 109 | while (true) { 110 | final int sourceAcct = rnd.nextInt(NUM_ROWS); 111 | final int targetAcct = rnd.nextInt(NUM_ROWS); 112 | final int sourceBook = rnd.nextInt(NUM_ROWS); 113 | final int targetBook = rnd.nextInt(NUM_ROWS); 114 | 115 | if (sourceAcct == targetAcct || sourceBook == targetBook) { 116 | continue; 117 | } 118 | return new TransactionEvent( 119 | ACCOUNT_ID_PREFIX + sourceAcct, 120 | ACCOUNT_ID_PREFIX + targetAcct, 121 | BOOK_ENTRY_ID_PREFIX + sourceBook, 122 | BOOK_ENTRY_ID_PREFIX + targetBook, 123 | accountsTransfer, 124 | transfer, 125 | MIN_BALANCE); 126 | } 127 | } 128 | 129 | } 130 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 17 | 19 | 20 | 4.0.0 21 | 22 | 23 | com.data-artisans.streamingledger 24 | da-streamingledger 25 | 1.1-SNAPSHOT 26 | .. 27 | 28 | 29 | da-streamingledger-sdk 30 | da-streamingledger :: da-streamingledger-sdk 31 | 32 | jar 33 | 34 | 35 | 36 | org.apache.flink 37 | flink-streaming-java_2.11 38 | ${flink.version} 39 | provided 40 | 41 | 42 | net.bytebuddy 43 | byte-buddy 44 | ${bytebuddy.version} 45 | 46 | 47 | 48 | 49 | org.apache.flink 50 | flink-test-utils-junit 51 | ${flink.version} 52 | test 53 | 54 | 55 | org.apache.flink 56 | flink-core 57 | ${flink.version} 58 | test-jar 59 | test 60 | 61 | 62 | 63 | 64 | 65 | 66 | 67 | 68 | org.apache.maven.plugins 69 | maven-shade-plugin 70 | 3.0.0 71 | 72 | 73 | 74 | package 75 | 76 | shade 77 | 78 | 79 | true 80 | 81 | 82 | net.bytebuddy 83 | com.dataartisans.streamingledger.shaded.net.bytebuddy 84 | 85 | 86 | true 87 | 88 | 89 | net.bytebuddy:* 90 | 91 | 92 | 93 | org.apache.flink:force-shading 94 | com.google.code.findbugs:jsr305 95 | org.slf4j:* 96 | log4j:* 97 | 98 | 99 | 100 | 101 | 103 | *:* 104 | 105 | META-INF/*.SF 106 | META-INF/*.DSA 107 | META-INF/*.RSA 108 | 109 | 110 | 111 | 112 | 114 | 115 | 116 | 117 | 118 | 119 | 120 | 121 | 122 | 123 | 124 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # data Artisans Streaming Ledger 2 | 3 | ### Serializable ACID transactions on streaming data 4 | 5 | data Artisans Streaming Ledger is a library on top of [Apache Flink](https://flink.apache.org/), for processing event streams across multiple shared states/tables with Serializable ACID Semantics. 6 | 7 | Instead of operating on a single key in a single operator at a time (like in vanilla Apache Flink and other stream processors) data Artisans Streaming Ledger allows you to define a set of states, connect streams of events that drive the transactions, and apply flexible business logic that operates transactionally across those states. 8 | 9 | ## This repository contains the following `maven` modules: 10 | * `da-streamingledger-sdk` - The `SDK` needed to define a streaming ledger application. 11 | * `da-streamingledger-runtime-serial` - A simplistic serial runner, to experiment with the `SDK`. 12 | * `da-streamingledger-examples` - Streaming ledger example programs. 13 | 14 | A parallel runner exists as part of the dA platform, 15 | you can learn more about the dA platform here: [dA Platform](https://data-artisans.com/platform-overview) 16 | 17 | ## Example 18 | 19 | Let's create a simple ledger of user accounts. 20 | An account in the ledger is identified by a `String` key, 21 | and has a `Long` value (its balance). 22 | 23 | We start by defining the streaming ledger scope. All state definitions and transaction functions 24 | would be bound to this named scope `"Account Ledger"`. 25 | 26 | ```java 27 | StreamingLedger ledger = StreamingLedger.create("Account Ledger"); 28 | ``` 29 | 30 | Next, we define the accounts state. 31 | 32 | ```java 33 | StreamingLedger.State accounts = ledger.declareState("accounts") 34 | .withKeyType(String.class) 35 | .withValueType(Long.class); 36 | ``` 37 | 38 | Next, let's assume we have a `DataStream` of `TransactionEvent`s, with the following schema: 39 | 40 | ```java 41 | final class TransactionEvent { 42 | 43 | private final String sourceAccountId; 44 | 45 | private final String targetAccountId; 46 | 47 | private final long accountTransfer; 48 | 49 | ... 50 | 51 | public String getSourceAccountId() { 52 | return sourceAccountId; 53 | } 54 | 55 | public String getTargetAccountId() { 56 | return targetAccountId; 57 | } 58 | 59 | public long getAccountTransfer() { 60 | return accountTransfer; 61 | } 62 | 63 | ... 64 | } 65 | 66 | ``` 67 | And we would like to transfer money from the source account to the target account, in response to an incoming `TransactionEvent`. 68 | 69 | ```java 70 | DataStream transactions = ... 71 | 72 | ledger.usingStream(transactions, "transaction stream") 73 | .apply(new TxnHandler()) 74 | .on(accounts, TransactionEvent::getSourceAccountId, "source-account", READ_WRITE) 75 | .on(accounts, TransactionEvent::getTargetAccountId, "target-account", READ_WRITE); 76 | ``` 77 | 78 | Where `TxnHandler` is a `TransactionProcessFunction` defined as: 79 | 80 | ```java 81 | class TxnHandler extends TransactionProcessFunction { 82 | 83 | @ProcessTransaction 84 | public void process( 85 | TransactionEvent txn, 86 | Context transactionContext, 87 | @State("source-account") StateAccess sourceAccount, 88 | @State("target-account") StateAccess targetAccount) { 89 | 90 | final long sourceAccountBalance = sourceAccount.read(); 91 | final long targetAccountBalance = targetAccount.read(); 92 | 93 | // check the preconditions 94 | if (sourceAccountBalance > txn.getAccountTransfer()) { 95 | 96 | // compute the new balances 97 | long newSourceBalance = sourceAccountBalance - txn.getAccountTransfer(); 98 | long newTargetBalance = targetAccountBalance + txn.getAccountTransfer(); 99 | 100 | // write back the updated values 101 | sourceAccount.write(newSourceBalance); 102 | targetAccount.write(newTargetBalance); 103 | } 104 | } 105 | } 106 | ``` 107 | 108 | Note that `TxnHandler` will be executed with the following guaranties: 109 | * *Atomicity:* The transaction applies all changes in an atomic manner. Either all of the modifications that the transaction function performs on the rows happen, or none. 110 | 111 | * *Consistency:* The transaction brings the tables from one consistent state into another consistent state. 112 | 113 | * *Isolation:* Each transaction executes as if it were the only transaction operating on the tables. Databases know different isolation levels with different guarantees. data Artisans Streaming Ledger here offers the best class: serializability. 114 | 115 | * *Durability:* The changes made by a transaction are durable and are not lost. Durability is ensured in the same way as in other Flink applications – through persistent sources and checkpoints. In the asynchronous nature of stream processing, durability of a result can only be assumed after a checkpoint. 116 | 117 | 118 | A more complete example can be found [here](https://github.com/dataArtisans/da-streamingledger/blob/master/da-streamingledger-examples/src/main/java/com/dataartisans/streamingledger/examples/simpletrade/SimpleTradeExample.java) 119 | 120 | ## Building from source 121 | 122 | prerequisites: 123 | 124 | * git 125 | * Maven 126 | * At least Java 8 127 | 128 | ``` 129 | git clone https://github.com/dataArtisans/da-streamingledger.git 130 | cd da-streamingledger 131 | mvn clean install 132 | ``` 133 | 134 | data Artisans Streaming Ledger is now available at your local `.m2` repository. 135 | 136 | ## Obtaining from Maven Central 137 | 138 | Just add the following dependency to start experimenting with the `SDK` 139 | 140 | ``` 141 | 142 | com.data-artisans.streamingledger 143 | da-streamingledger-sdk 144 | 1.0.0 145 | 146 | 147 | com.data-artisans.streamingledger 148 | da-streamingledger-runtime-serial 149 | 1.0.0 150 | 151 | ``` 152 | 153 | ## License 154 | 155 | The code in this repository is under the Apache license, see [license](https://github.com/dataArtisans/da-streamingledger/blob/master/LICENSE) 156 | 157 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/test/java/com/dataartisans/streamingledger/sdk/common/reflection/ByteBuddyProcessFunctionInvokerTest.java: -------------------------------------------------------------------------------- 1 | 2 | /* 3 | * Copyright 2018 Data Artisans GmbH 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.dataartisans.streamingledger.sdk.common.reflection; 19 | 20 | import org.apache.flink.api.common.typeinfo.BasicTypeInfo; 21 | import org.apache.flink.api.common.typeinfo.TypeInformation; 22 | 23 | import com.dataartisans.streamingledger.sdk.api.AccessType; 24 | import com.dataartisans.streamingledger.sdk.api.StateAccess; 25 | import com.dataartisans.streamingledger.sdk.api.StateAccessException; 26 | import com.dataartisans.streamingledger.sdk.api.StreamingLedger.State; 27 | import com.dataartisans.streamingledger.sdk.api.StreamingLedger.StateAccessSpec; 28 | import com.dataartisans.streamingledger.sdk.api.TransactionProcessFunction; 29 | import com.dataartisans.streamingledger.sdk.api.TransactionProcessFunction.Context; 30 | import com.dataartisans.streamingledger.sdk.spi.StreamingLedgerSpec; 31 | import com.dataartisans.streamingledger.sdk.spi.StreamingLedgerSpecFactory; 32 | import org.junit.Before; 33 | import org.junit.Test; 34 | 35 | import java.util.ArrayList; 36 | import java.util.Collections; 37 | import java.util.List; 38 | 39 | import static org.hamcrest.CoreMatchers.hasItem; 40 | import static org.hamcrest.MatcherAssert.assertThat; 41 | 42 | /** 43 | * Test {@link ByteBuddyProcessFunctionInvoker}. 44 | */ 45 | public class ByteBuddyProcessFunctionInvokerTest { 46 | 47 | // -------------------------------------------------------------------------------------------------------------- 48 | // Transaction Function. 49 | // -------------------------------------------------------------------------------------------------------------- 50 | 51 | private ListContext context; 52 | 53 | // -------------------------------------------------------------------------------------------------------------- 54 | // Test Setup. 55 | // -------------------------------------------------------------------------------------------------------------- 56 | private StateAccessStub[] arguments; 57 | private StreamingLedgerSpec specification; 58 | 59 | private static StreamingLedgerSpec createSpecificationUnderTest() { 60 | State state = new State<>( 61 | "state", 62 | BasicTypeInfo.INT_TYPE_INFO, 63 | BasicTypeInfo.LONG_TYPE_INFO); 64 | 65 | StateAccessSpec accessSpec = new StateAccessSpec<>( 66 | "value", 67 | state, 68 | String::hashCode, 69 | AccessType.READ_WRITE); 70 | 71 | List> bindings = Collections.singletonList(accessSpec); 72 | TypeInformation inType = BasicTypeInfo.STRING_TYPE_INFO; 73 | TypeInformation outType = BasicTypeInfo.LONG_TYPE_INFO; 74 | return StreamingLedgerSpecFactory.create(new ReadingTransactionFunction(), bindings, inType, outType); 75 | } 76 | 77 | @Before 78 | public void before() { 79 | specification = createSpecificationUnderTest(); 80 | context = new ListContext<>(); 81 | arguments = new StateAccessStub[]{new StateAccessStub()}; 82 | } 83 | 84 | // -------------------------------------------------------------------------------------------------------------- 85 | // Tests. 86 | // -------------------------------------------------------------------------------------------------------------- 87 | 88 | @Test 89 | public void usageExample() { 90 | ProcessFunctionInvoker generatedCodeThatInvokesUserCode = 91 | ByteBuddyProcessFunctionInvoker.create(specification); 92 | 93 | arguments[0].value = Long.MAX_VALUE; 94 | generatedCodeThatInvokesUserCode.invoke("does not matter", context, arguments); 95 | 96 | assertThat(context.emitted, hasItem(Long.MAX_VALUE)); 97 | } 98 | 99 | // -------------------------------------------------------------------------------------------------------------- 100 | // Test Utils. 101 | // -------------------------------------------------------------------------------------------------------------- 102 | 103 | /** 104 | * ReadingTransactionFunction - simulates a user provided transaction function, that has a single state access. 105 | * This user code will read the provided value and emit it. See: {@link #process(String, Context, StateAccess)}. 106 | */ 107 | private static class ReadingTransactionFunction extends TransactionProcessFunction { 108 | 109 | private static final long serialVersionUID = 1; 110 | 111 | @ProcessTransaction 112 | public void process(String input, Context out, @State("value") StateAccess value) { 113 | Long v = value.read(); 114 | out.emit(v); 115 | } 116 | } 117 | 118 | private final class StateAccessStub implements StateAccess { 119 | T value; 120 | 121 | @Override 122 | public T read() throws StateAccessException { 123 | return value; 124 | } 125 | 126 | @Override 127 | public void write(T newValue) throws StateAccessException { 128 | value = newValue; 129 | } 130 | 131 | @Override 132 | public void delete() throws StateAccessException { 133 | value = null; 134 | } 135 | 136 | @Override 137 | public String getStateName() { 138 | throw new UnsupportedOperationException(); 139 | } 140 | 141 | @Override 142 | public String getStateAccessName() { 143 | throw new UnsupportedOperationException(); 144 | } 145 | } 146 | 147 | private final class ListContext implements Context { 148 | List emitted = new ArrayList<>(); 149 | 150 | @Override 151 | public void emit(T record) { 152 | emitted.add(record); 153 | } 154 | 155 | @Override 156 | public void abort() { 157 | emitted.clear(); 158 | } 159 | } 160 | 161 | } 162 | -------------------------------------------------------------------------------- /da-streamingledger-examples/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 17 | 19 | 20 | 4.0.0 21 | 22 | 23 | com.data-artisans.streamingledger 24 | da-streamingledger 25 | 1.1-SNAPSHOT 26 | .. 27 | 28 | 29 | da-streamingledger-examples 30 | da-streamingledger :: da-streamingledger-examples 31 | 32 | jar 33 | 34 | 35 | 36 | 37 | org.apache.flink 38 | flink-streaming-java_2.11 39 | ${flink.version} 40 | provided 41 | 42 | 43 | com.data-artisans.streamingledger 44 | da-streamingledger-sdk 45 | ${project.version} 46 | 47 | 48 | com.data-artisans.streamingledger 49 | da-streamingledger-runtime-serial 50 | ${project.version} 51 | 52 | 53 | 54 | 55 | org.slf4j 56 | slf4j-log4j12 57 | runtime 58 | 59 | 60 | 61 | log4j 62 | log4j 63 | runtime 64 | 65 | 66 | 67 | 68 | 69 | 70 | 71 | 72 | 73 | org.apache.maven.plugins 74 | maven-shade-plugin 75 | 3.0.0 76 | 77 | 78 | 79 | package 80 | 81 | shade 82 | 83 | 84 | false 85 | 86 | 87 | org.apache.flink:force-shading 88 | com.google.code.findbugs:jsr305 89 | org.slf4j:* 90 | log4j:* 91 | net.bytebuddy:* 92 | 93 | 94 | 95 | 96 | 98 | *:* 99 | 100 | META-INF/*.SF 101 | META-INF/*.DSA 102 | META-INF/*.RSA 103 | 104 | 105 | 106 | 107 | 109 | com.dataartisans.streamingledger.examples.simpletrade.SimpleTradeExample 110 | 111 | 112 | 113 | 114 | 115 | 116 | 117 | 118 | 119 | 120 | 121 | 122 | 123 | 124 | 125 | add-dependencies-for-IDEA 126 | 127 | 128 | 129 | idea.version 130 | 131 | 132 | 133 | 134 | 135 | org.apache.flink 136 | flink-streaming-java_2.11 137 | ${flink.version} 138 | compile 139 | 140 | 141 | org.apache.flink 142 | flink-runtime-web_2.11 143 | ${flink.version} 144 | compile 145 | 146 | 147 | 148 | 149 | 150 | 151 | -------------------------------------------------------------------------------- /da-streamingledger-examples/src/main/java/com/dataartisans/streamingledger/examples/simpletrade/SimpleTradeExample.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.examples.simpletrade; 18 | 19 | import org.apache.flink.runtime.state.StateBackend; 20 | import org.apache.flink.runtime.state.filesystem.FsStateBackend; 21 | import org.apache.flink.streaming.api.datastream.DataStream; 22 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 23 | import org.apache.flink.util.OutputTag; 24 | 25 | import com.dataartisans.streamingledger.examples.simpletrade.generator.SyntheticSources; 26 | import com.dataartisans.streamingledger.sdk.api.StateAccess; 27 | import com.dataartisans.streamingledger.sdk.api.StreamingLedger; 28 | import com.dataartisans.streamingledger.sdk.api.StreamingLedger.ResultStreams; 29 | import com.dataartisans.streamingledger.sdk.api.TransactionProcessFunction; 30 | 31 | import java.net.URI; 32 | import java.nio.file.Paths; 33 | import java.util.function.Supplier; 34 | 35 | import static com.dataartisans.streamingledger.sdk.api.AccessType.READ_WRITE; 36 | 37 | /** 38 | * A simple example illustrating the use of stream ledger. 39 | * 40 | *

The example here uses two states (called "accounts" and "bookEntries") and modifies two keys in each state in one 41 | * joint transaction. 42 | */ 43 | public class SimpleTradeExample { 44 | 45 | private static final Supplier ZERO = () -> 0L; 46 | 47 | /** 48 | * The main entry point to the sample application. This runs the program with a 49 | * built-in data generator and the non-parallel local runtime implementation for 50 | * the transaction logic. 51 | * 52 | * @param args The command line arguments. 53 | */ 54 | public static void main(String[] args) throws Exception { 55 | 56 | // set up the execution environment and the configuration 57 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 58 | 59 | // configure Flink 60 | env.setParallelism(4); 61 | env.getConfig().enableObjectReuse(); 62 | 63 | // enable checkpoints once a minute 64 | env.enableCheckpointing(60_000); 65 | URI uri = Paths.get("./checkpoints").toAbsolutePath().normalize().toUri(); 66 | StateBackend backend = new FsStateBackend(uri, true); 67 | env.setStateBackend(backend); 68 | 69 | // create and add two data sources 70 | SyntheticSources sources = SyntheticSources.create(env, 1); 71 | 72 | // start building the transactional streams 73 | StreamingLedger tradeLedger = StreamingLedger.create("simple trade example"); 74 | 75 | // define the transactional states 76 | StreamingLedger.State accounts = tradeLedger.declareState("accounts") 77 | .withKeyType(String.class) 78 | .withValueType(Long.class); 79 | 80 | StreamingLedger.State books = tradeLedger.declareState("bookEntries") 81 | .withKeyType(String.class) 82 | .withValueType(Long.class); 83 | 84 | // produce the deposits transaction stream 85 | DataStream deposits = sources.deposits; 86 | 87 | // define transactors on states 88 | tradeLedger.usingStream(deposits, "deposits") 89 | .apply(new DepositHandler()) 90 | .on(accounts, DepositEvent::getAccountId, "account", READ_WRITE) 91 | .on(books, DepositEvent::getBookEntryId, "asset", READ_WRITE); 92 | 93 | // produce transactions stream 94 | DataStream transfers = sources.transactions; 95 | 96 | OutputTag transactionResults = tradeLedger.usingStream(transfers, "transactions") 97 | .apply(new TxnHandler()) 98 | .on(accounts, TransactionEvent::getSourceAccountId, "source-account", READ_WRITE) 99 | .on(accounts, TransactionEvent::getTargetAccountId, "target-account", READ_WRITE) 100 | .on(books, TransactionEvent::getSourceBookEntryId, "source-asset", READ_WRITE) 101 | .on(books, TransactionEvent::getTargetBookEntryId, "target-asset", READ_WRITE) 102 | .output(); 103 | 104 | // compute the resulting streams. 105 | ResultStreams resultsStreams = tradeLedger.resultStreams(); 106 | 107 | // output to the console 108 | resultsStreams.getResultStream(transactionResults).print(); 109 | 110 | // trigger program execution 111 | env.execute(); 112 | } 113 | 114 | /** 115 | * The implementation of the logic that executes a deposit. 116 | */ 117 | private static final class DepositHandler extends TransactionProcessFunction { 118 | 119 | private static final long serialVersionUID = 1; 120 | 121 | @ProcessTransaction 122 | public void process( 123 | final DepositEvent event, 124 | final Context ctx, 125 | final @State("account") StateAccess account, 126 | final @State("asset") StateAccess asset) { 127 | 128 | long newAccountValue = account.readOr(ZERO) + event.getAccountTransfer(); 129 | 130 | account.write(newAccountValue); 131 | 132 | long newAssetValue = asset.readOr(ZERO) + event.getBookEntryTransfer(); 133 | asset.write(newAssetValue); 134 | } 135 | } 136 | 137 | /** 138 | * The implementation of the logic that executes the transaction. The logic is given the original 139 | * TransactionEvent plus all states involved in the transaction. 140 | */ 141 | private static final class TxnHandler extends TransactionProcessFunction { 142 | 143 | private static final long serialVersionUID = 1; 144 | 145 | @ProcessTransaction 146 | public void process( 147 | final TransactionEvent txn, 148 | final Context ctx, 149 | final @State("source-account") StateAccess sourceAccount, 150 | final @State("target-account") StateAccess targetAccount, 151 | final @State("source-asset") StateAccess sourceAsset, 152 | final @State("target-asset") StateAccess targetAsset) { 153 | 154 | final long sourceAccountBalance = sourceAccount.readOr(ZERO); 155 | final long sourceAssetValue = sourceAsset.readOr(ZERO); 156 | final long targetAccountBalance = targetAccount.readOr(ZERO); 157 | final long targetAssetValue = targetAsset.readOr(ZERO); 158 | 159 | // check the preconditions 160 | if (sourceAccountBalance > txn.getMinAccountBalance() 161 | && sourceAccountBalance > txn.getAccountTransfer() 162 | && sourceAssetValue > txn.getBookEntryTransfer()) { 163 | 164 | // compute the new balances 165 | final long newSourceBalance = sourceAccountBalance - txn.getAccountTransfer(); 166 | final long newTargetBalance = targetAccountBalance + txn.getAccountTransfer(); 167 | final long newSourceAssets = sourceAssetValue - txn.getBookEntryTransfer(); 168 | final long newTargetAssets = targetAssetValue + txn.getBookEntryTransfer(); 169 | 170 | // write back the updated values 171 | sourceAccount.write(newSourceBalance); 172 | targetAccount.write(newTargetBalance); 173 | sourceAsset.write(newSourceAssets); 174 | targetAsset.write(newTargetAssets); 175 | 176 | // emit result event with updated balances and flag to mark transaction as processed 177 | ctx.emit(new TransactionResult(txn, true, newSourceBalance, newTargetBalance)); 178 | } 179 | else { 180 | // emit result with unchanged balances and a flag to mark transaction as rejected 181 | ctx.emit(new TransactionResult(txn, false, sourceAccountBalance, targetAccountBalance)); 182 | } 183 | } 184 | } 185 | 186 | } 187 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/spi/StreamingLedgerSpecFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.spi; 18 | 19 | import org.apache.flink.annotation.Internal; 20 | import org.apache.flink.api.common.typeinfo.TypeInformation; 21 | 22 | import com.dataartisans.streamingledger.sdk.api.StateAccess; 23 | import com.dataartisans.streamingledger.sdk.api.StreamingLedger.StateAccessSpec; 24 | import com.dataartisans.streamingledger.sdk.api.TransactionProcessFunction; 25 | import com.dataartisans.streamingledger.sdk.api.TransactionProcessFunction.Context; 26 | import com.dataartisans.streamingledger.sdk.api.TransactionProcessFunction.ProcessTransaction; 27 | import com.dataartisans.streamingledger.sdk.api.TransactionProcessFunction.State; 28 | import com.dataartisans.streamingledger.sdk.common.reflection.Methods; 29 | 30 | import java.lang.reflect.Method; 31 | import java.lang.reflect.Parameter; 32 | import java.lang.reflect.ParameterizedType; 33 | import java.lang.reflect.Type; 34 | import java.util.ArrayList; 35 | import java.util.HashMap; 36 | import java.util.Iterator; 37 | import java.util.List; 38 | import java.util.Map; 39 | import java.util.function.Function; 40 | 41 | import static java.util.Objects.requireNonNull; 42 | 43 | /** 44 | * A {@link StreamingLedgerSpec} factory. 45 | */ 46 | @Internal 47 | public final class StreamingLedgerSpecFactory { 48 | 49 | private StreamingLedgerSpecFactory() { 50 | } 51 | 52 | /** 53 | * Creates a new TransactionOperationSpec. 54 | * 55 | * @param The type of the transaction data events. 56 | * @param The type of the transaction data results. 57 | */ 58 | public static StreamingLedgerSpec create( 59 | TransactionProcessFunction processFunction, 60 | List> stateSpecs, 61 | TypeInformation inputType, 62 | TypeInformation outputType) { 63 | 64 | requireNonNull(processFunction); 65 | requireNonNull(stateSpecs); 66 | Iterator annotatedMethods = Methods.findAnnotatedMethods( 67 | processFunction.getClass(), 68 | ProcessTransaction.class); 69 | 70 | if (!annotatedMethods.hasNext()) { 71 | throw missingAnnotation(processFunction); 72 | } 73 | final Method method = annotatedMethods.next(); 74 | if (annotatedMethods.hasNext()) { 75 | throw tooManyAnnotatedMethods(processFunction); 76 | } 77 | final Parameter[] parameters = method.getParameters(); 78 | if (parameters.length != stateSpecs.size() + 2) { 79 | throw wrongParameterCount(processFunction, method.getName(), inputType.getTypeClass(), stateSpecs); 80 | } 81 | // first parameter is the input 82 | if (!isOfSimpleType(parameters[0], inputType)) { 83 | throw wrongParameter(processFunction, method.getName(), parameters[0], "wrong type."); 84 | } 85 | // second parameter is Context 86 | if (!isOfGenericType(parameters[1], Context.class, outputType)) { 87 | throw wrongParameter(processFunction, method.getName(), parameters[1], "wrong type."); 88 | } 89 | // the rest of the parameters are StateAccess annotated with the @State annotation. 90 | // The annotation connects a parameter to the StateAccessSpec via bindName. 91 | Map> bindName2StateSpec = uniqueIndex(stateSpecs, s -> s.bindName); 92 | List> referencedStateSpecs = new ArrayList<>(); 93 | for (int i = 2; i < parameters.length; i++) { 94 | final Parameter parameter = parameters[i]; 95 | final State state = parameter.getAnnotation(State.class); 96 | if (state == null) { 97 | throw wrongParameter( 98 | processFunction, 99 | method.getName(), 100 | parameters[i], 101 | "not annotated with a @State."); 102 | } 103 | StateAccessSpec stateAccess = bindName2StateSpec.get(state.value()); 104 | if (stateAccess == null) { 105 | throw wrongParameter( 106 | processFunction, 107 | method.getName(), 108 | parameters[i], 109 | "unknown state spec '" + state.value() + "'"); 110 | } 111 | // parameter: StateAccess 112 | if (!isOfGenericType(parameter, StateAccess.class, stateAccess.state.getValueType())) { 113 | throw wrongParameter( 114 | processFunction, 115 | method.getName(), 116 | parameters[i], 117 | "state spec '" + state.value() + "' has a value type " 118 | + stateAccess.state.getValueType()); 119 | } 120 | referencedStateSpecs.add(stateAccess); 121 | } 122 | 123 | return new StreamingLedgerSpec<>( 124 | processFunction, 125 | method.getName(), 126 | referencedStateSpecs, 127 | inputType, 128 | outputType); 129 | } 130 | 131 | 132 | // ------------------------------------------------------------------------ 133 | // Static helpers 134 | // ------------------------------------------------------------------------ 135 | 136 | private static IllegalArgumentException missingAnnotation(Object subject) { 137 | String className = subject.getClass().getSimpleName(); 138 | String annotation = ProcessTransaction.class.getSimpleName(); 139 | return new IllegalArgumentException("Could not find any method of " + className + " that is annotated with @" 140 | + annotation 141 | ); 142 | } 143 | 144 | private static IllegalArgumentException tooManyAnnotatedMethods(Object subject) { 145 | String className = subject.getClass().getSimpleName(); 146 | String annotation = ProcessTransaction.class.getSimpleName(); 147 | return new IllegalArgumentException("There multiple methods of " + className + " that are annotated with " 148 | + "@" + annotation 149 | ); 150 | } 151 | 152 | private static IllegalArgumentException wrongParameter( 153 | Object subject, 154 | String methodName, 155 | Parameter parameter, 156 | String message) { 157 | String className = subject.getClass().getSimpleName(); 158 | return new IllegalArgumentException("A problem with the field " + parameter + " of " + className + "." 159 | + methodName + "\t" + message 160 | ); 161 | } 162 | 163 | private static IllegalArgumentException wrongParameterCount( 164 | Object subject, 165 | String methodName, 166 | Class typeClass, 167 | List> stateSpecs) { 168 | String className = subject.getClass().getSimpleName(); 169 | return new IllegalArgumentException(className + "." + methodName + " has wrong argument count." 170 | + " Expected: " + methodName + "(" + typeClass + ", Context<" + typeClass + ">, ... " 171 | + stateSpecs.size() + " state accesses" 172 | ); 173 | } 174 | 175 | private static boolean isOfSimpleType(Parameter parameter, TypeInformation type) { 176 | return parameter.getType() == type.getTypeClass(); 177 | } 178 | 179 | @SuppressWarnings("BooleanMethodIsAlwaysInverted") 180 | private static boolean isOfGenericType(Parameter parameter, Class baseType, TypeInformation type) { 181 | if (!(parameter.getParameterizedType() instanceof ParameterizedType)) { 182 | return false; 183 | } 184 | ParameterizedType parameterizedType = (ParameterizedType) parameter.getParameterizedType(); 185 | if (parameterizedType.getRawType() != baseType) { 186 | return false; 187 | } 188 | Type t = parameterizedType.getActualTypeArguments()[0]; 189 | return t == type.getTypeClass(); 190 | } 191 | 192 | private static Map uniqueIndex(Iterable elements, Function indexExtractor) { 193 | Map key2element = new HashMap<>(); 194 | for (E element : elements) { 195 | K key = indexExtractor.apply(element); 196 | key2element.put(key, element); 197 | } 198 | return key2element; 199 | } 200 | } 201 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/common/union/UnionSerializer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.common.union; 18 | 19 | import org.apache.flink.api.common.typeutils.CompatibilityResult; 20 | import org.apache.flink.api.common.typeutils.CompatibilityUtil; 21 | import org.apache.flink.api.common.typeutils.TypeDeserializer; 22 | import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter; 23 | import org.apache.flink.api.common.typeutils.TypeSerializer; 24 | import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; 25 | import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer; 26 | import org.apache.flink.api.java.tuple.Tuple2; 27 | import org.apache.flink.core.memory.DataInputView; 28 | import org.apache.flink.core.memory.DataOutputView; 29 | 30 | import java.io.IOException; 31 | import java.util.ArrayList; 32 | import java.util.Arrays; 33 | import java.util.List; 34 | 35 | import static java.util.Objects.requireNonNull; 36 | import static org.apache.flink.api.common.typeutils.CompatibilityResult.requiresMigration; 37 | import static org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.base.Preconditions.checkArgument; 38 | 39 | final class UnionSerializer extends TypeSerializer { 40 | 41 | private static final long serialVersionUID = 1; 42 | 43 | private final TypeSerializer[] underlyingSerializers; 44 | 45 | private transient Object[] reusableObjects; 46 | 47 | UnionSerializer(List> underlyingSerializers) { 48 | requireNonNull(underlyingSerializers); 49 | checkArgument(!underlyingSerializers.isEmpty(), "At least one underlying serializer is needed."); 50 | this.underlyingSerializers = toArray(underlyingSerializers); 51 | this.reusableObjects = createReusableObjects(this.underlyingSerializers); 52 | } 53 | 54 | private static Object[] createReusableObjects(TypeSerializer[] underlyingSerializers) { 55 | Object[] reusableObjects = new Object[underlyingSerializers.length]; 56 | for (int i = 0; i < reusableObjects.length; i++) { 57 | reusableObjects[i] = underlyingSerializers[i].createInstance(); 58 | } 59 | return reusableObjects; 60 | } 61 | 62 | @SuppressWarnings("unchecked") 63 | private static TypeSerializer[] toArray(List> underlyingSerializers) { 64 | return underlyingSerializers.toArray(new TypeSerializer[0]); 65 | } 66 | 67 | @Override 68 | public boolean isImmutableType() { 69 | for (TypeSerializer serializer : underlyingSerializers) { 70 | if (!serializer.isImmutableType()) { 71 | return false; 72 | } 73 | } 74 | return true; 75 | } 76 | 77 | @Override 78 | public TypeSerializer duplicate() { 79 | List> duplicates = new ArrayList<>(underlyingSerializers.length); 80 | boolean stateful = false; 81 | for (TypeSerializer serializer : underlyingSerializers) { 82 | TypeSerializer duplicate = serializer.duplicate(); 83 | if (duplicate != serializer) { 84 | stateful = true; 85 | } 86 | duplicates.add(duplicate); 87 | } 88 | if (!stateful) { 89 | return this; 90 | } 91 | return new UnionSerializer(duplicates); 92 | } 93 | 94 | @Override 95 | public TaggedElement createInstance() { 96 | return new TaggedElement(TaggedElement.UNDEFINED_TAG, null); 97 | } 98 | 99 | @Override 100 | public TaggedElement copy(TaggedElement from) { 101 | final int tag = from.getDataStreamTag(); 102 | Object copyOf = underlyingSerializers[tag].copy(from.getElement()); 103 | return new TaggedElement(from.getDataStreamTag(), copyOf); 104 | } 105 | 106 | @Override 107 | public TaggedElement copy(TaggedElement from, TaggedElement reuse) { 108 | final int tag = from.getDataStreamTag(); 109 | final TypeSerializer serializer = underlyingSerializers[tag]; 110 | final Object elementCopy = serializer.copy(from.getElement(), reusableObjects[tag]); 111 | 112 | reuse.setElement(elementCopy); 113 | reuse.setDataStreamTag(tag); 114 | return reuse; 115 | } 116 | 117 | @Override 118 | public int getLength() { 119 | return -1; 120 | } 121 | 122 | @Override 123 | public void serialize(TaggedElement record, DataOutputView target) throws IOException { 124 | final int tag = record.getDataStreamTag(); 125 | target.writeInt(tag); 126 | underlyingSerializers[tag].serialize(record.getElement(), target); 127 | } 128 | 129 | @Override 130 | public TaggedElement deserialize(DataInputView source) throws IOException { 131 | final int tag = source.readInt(); 132 | Object value = underlyingSerializers[tag].deserialize(source); 133 | return new TaggedElement(tag, value); 134 | } 135 | 136 | @Override 137 | public TaggedElement deserialize(TaggedElement reuse, DataInputView source) throws IOException { 138 | final int tag = source.readInt(); 139 | final TypeSerializer serializer = underlyingSerializers[tag]; 140 | final Object element = serializer.deserialize(reusableObjects[tag], source); 141 | 142 | reuse.setDataStreamTag(tag); 143 | reuse.setElement(element); 144 | return reuse; 145 | } 146 | 147 | @Override 148 | public void copy(DataInputView source, DataOutputView target) throws IOException { 149 | final int tag = source.readInt(); 150 | target.writeInt(tag); 151 | underlyingSerializers[tag].copy(source, target); 152 | } 153 | 154 | @Override 155 | public boolean equals(Object o) { 156 | if (this == o) { 157 | return true; 158 | } 159 | if (o == null || getClass() != o.getClass()) { 160 | return false; 161 | } 162 | UnionSerializer that = (UnionSerializer) o; 163 | return Arrays.equals(underlyingSerializers, that.underlyingSerializers); 164 | } 165 | 166 | @Override 167 | public boolean canEqual(Object obj) { 168 | return obj instanceof UnionSerializer; 169 | } 170 | 171 | @Override 172 | public int hashCode() { 173 | return Arrays.hashCode(underlyingSerializers); 174 | } 175 | 176 | // ----------------------------------------------------------------------------------- 177 | // Internal helper methods 178 | // ----------------------------------------------------------------------------------- 179 | 180 | @Override 181 | public TypeSerializerConfigSnapshot snapshotConfiguration() { 182 | return new UnionSerializerConfigSnapshot(Arrays.asList(underlyingSerializers)); 183 | } 184 | 185 | @Override 186 | public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { 187 | if (!(configSnapshot instanceof UnionSerializerConfigSnapshot)) { 188 | return requiresMigration(); 189 | } 190 | UnionSerializerConfigSnapshot config = (UnionSerializerConfigSnapshot) configSnapshot; 191 | List, TypeSerializerConfigSnapshot>> previousSerializersAndConfigs = 192 | config.getNestedSerializersAndConfigs(); 193 | 194 | if (previousSerializersAndConfigs.size() != underlyingSerializers.length) { 195 | return requiresMigration(); 196 | } 197 | 198 | List> migratedSerializers = new ArrayList<>(); 199 | // Adopted from the EitherSerializerConfigSnapshot. 200 | // One (or more) of the serializers here is not compatible with one (or more) of the previous serializers. 201 | boolean requiresMigration = false; 202 | for (int i = 0; i < underlyingSerializers.length; i++) { 203 | final TypeSerializer underlyingSerializer = underlyingSerializers[i]; 204 | 205 | final CompatibilityResult res = CompatibilityUtil.resolveCompatibilityResult( 206 | previousSerializersAndConfigs.get(i).f0, 207 | UnloadableDummyTypeSerializer.class, 208 | previousSerializersAndConfigs.get(i).f1, 209 | underlyingSerializer); 210 | 211 | if (!res.isRequiresMigration()) { 212 | migratedSerializers.add(underlyingSerializer); 213 | continue; 214 | } 215 | requiresMigration = true; 216 | TypeDeserializer deserializer = res.getConvertDeserializer(); 217 | if (deserializer == null) { 218 | return requiresMigration(); 219 | } 220 | migratedSerializers.add(new TypeDeserializerAdapter<>(deserializer)); 221 | } 222 | UnionSerializer migratedSerializer = new UnionSerializer(migratedSerializers); 223 | return requiresMigration ? requiresMigration(migratedSerializer) : CompatibilityResult.compatible(); 224 | } 225 | 226 | private void readObject(java.io.ObjectInputStream in) 227 | throws IOException, ClassNotFoundException { 228 | in.defaultReadObject(); 229 | this.reusableObjects = createReusableObjects(this.underlyingSerializers); 230 | } 231 | } 232 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/test/java/com/dataartisans/streamingledger/sdk/api/StreamingLedgerSpecTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.api; 18 | 19 | import org.apache.flink.api.common.typeinfo.BasicTypeInfo; 20 | import org.apache.flink.api.common.typeinfo.TypeInformation; 21 | 22 | import com.dataartisans.streamingledger.sdk.api.StreamingLedger.State; 23 | import com.dataartisans.streamingledger.sdk.api.StreamingLedger.StateAccessSpec; 24 | import com.dataartisans.streamingledger.sdk.spi.StreamingLedgerSpec; 25 | import com.dataartisans.streamingledger.sdk.spi.StreamingLedgerSpecFactory; 26 | import org.junit.Test; 27 | 28 | import java.util.Collections; 29 | import java.util.List; 30 | 31 | import static org.hamcrest.CoreMatchers.is; 32 | import static org.hamcrest.MatcherAssert.assertThat; 33 | 34 | /** 35 | * Test {@link StreamingLedgerSpec}. 36 | */ 37 | public class StreamingLedgerSpecTest { 38 | 39 | private static StreamingLedgerSpec createSpecificationUnderTest( 40 | TransactionProcessFunction processFunction, 41 | TypeInformation inType, 42 | TypeInformation outType) { 43 | 44 | State state = new State<>( 45 | "state", 46 | BasicTypeInfo.INT_TYPE_INFO, 47 | BasicTypeInfo.LONG_TYPE_INFO); 48 | 49 | StateAccessSpec accessSpec = new StateAccessSpec<>( 50 | "value", 51 | state, 52 | Object::hashCode, 53 | AccessType.READ_WRITE); 54 | 55 | List> bindings = Collections.singletonList(accessSpec); 56 | return StreamingLedgerSpecFactory.create(processFunction, bindings, inType, outType); 57 | } 58 | 59 | @Test 60 | public void example() { 61 | 62 | final class TxnFn extends TransactionProcessFunction { 63 | 64 | private static final long serialVersionUID = 1; 65 | 66 | @ProcessTransaction 67 | public void process(String input, Context out, @State("value") StateAccess value) { 68 | Long v = value.read(); 69 | out.emit(v); 70 | } 71 | } 72 | 73 | StreamingLedgerSpec spec = createSpecificationUnderTest( 74 | new TxnFn(), 75 | BasicTypeInfo.STRING_TYPE_INFO, 76 | BasicTypeInfo.LONG_TYPE_INFO); 77 | 78 | assertThat(spec.processMethodName, is("process")); 79 | } 80 | 81 | @Test(expected = IllegalArgumentException.class) 82 | public void missingProcessTransactionAnnotation() { 83 | 84 | final class MissingProcessTxnAnnotation extends TransactionProcessFunction { 85 | 86 | private static final long serialVersionUID = 1; 87 | 88 | // Ooops: @ProcessTransaction 89 | public void process(Boolean input, Context out, @State("value") StateAccess value) { 90 | Long v = value.read(); 91 | out.emit(v); 92 | } 93 | } 94 | 95 | createSpecificationUnderTest( 96 | new MissingProcessTxnAnnotation(), 97 | BasicTypeInfo.STRING_TYPE_INFO, 98 | BasicTypeInfo.LONG_TYPE_INFO); 99 | } 100 | 101 | @Test(expected = IllegalArgumentException.class) 102 | public void twoFunctionsAnnotated() { 103 | 104 | final class TwoFunctionsAnnotated extends TransactionProcessFunction { 105 | 106 | private static final long serialVersionUID = 1; 107 | 108 | @ProcessTransaction 109 | public void process1(Boolean input, Context out, @State("value") StateAccess value) { 110 | Long v = value.read(); 111 | out.emit(v); 112 | } 113 | 114 | @ProcessTransaction 115 | public void process2(Boolean input, Context out, @State("value") StateAccess value) { 116 | Long v = value.read(); 117 | out.emit(v); 118 | } 119 | } 120 | 121 | createSpecificationUnderTest( 122 | new TwoFunctionsAnnotated(), 123 | BasicTypeInfo.STRING_TYPE_INFO, 124 | BasicTypeInfo.LONG_TYPE_INFO); 125 | } 126 | 127 | @Test 128 | public void subClassingWorks() { 129 | 130 | class ParentClass extends TransactionProcessFunction { 131 | 132 | private static final long serialVersionUID = 1; 133 | 134 | @ProcessTransaction 135 | public void process(String input, Context out, @State("value") StateAccess value) { 136 | Long v = value.read(); 137 | out.emit(v); 138 | } 139 | 140 | } 141 | 142 | final class SubClass extends ParentClass { 143 | } 144 | 145 | createSpecificationUnderTest( 146 | new SubClass(), 147 | BasicTypeInfo.STRING_TYPE_INFO, 148 | BasicTypeInfo.LONG_TYPE_INFO); 149 | } 150 | 151 | @Test(expected = IllegalArgumentException.class) 152 | public void wrongInputParameter() { 153 | 154 | final class WrongInputParameter extends TransactionProcessFunction { 155 | 156 | private static final long serialVersionUID = 1; 157 | 158 | @ProcessTransaction 159 | public void process(Boolean input, Context out, @State("value") StateAccess value) { 160 | Long v = value.read(); 161 | out.emit(v); 162 | } 163 | } 164 | 165 | createSpecificationUnderTest( 166 | new WrongInputParameter(), 167 | BasicTypeInfo.STRING_TYPE_INFO, 168 | BasicTypeInfo.LONG_TYPE_INFO); 169 | } 170 | 171 | @Test(expected = IllegalArgumentException.class) 172 | public void wrongContextTypeParameter() { 173 | 174 | final class WrongContextParameter extends TransactionProcessFunction { 175 | 176 | private static final long serialVersionUID = 1; 177 | 178 | @ProcessTransaction 179 | public void process(String input, Context out, @State("value") StateAccess value) { 180 | out.emit(true); 181 | } 182 | } 183 | 184 | createSpecificationUnderTest( 185 | new WrongContextParameter(), 186 | BasicTypeInfo.STRING_TYPE_INFO, 187 | BasicTypeInfo.LONG_TYPE_INFO); 188 | } 189 | 190 | @Test(expected = IllegalArgumentException.class) 191 | public void unknownStateBindingName() { 192 | 193 | final class UnknownStateBindingName extends TransactionProcessFunction { 194 | 195 | private static final long serialVersionUID = 1; 196 | 197 | @ProcessTransaction 198 | public void process(String input, Context out, @State("xyz") StateAccess value) { 199 | out.emit(1L); 200 | } 201 | } 202 | 203 | createSpecificationUnderTest( 204 | new UnknownStateBindingName(), 205 | BasicTypeInfo.STRING_TYPE_INFO, 206 | BasicTypeInfo.LONG_TYPE_INFO); 207 | } 208 | 209 | @Test(expected = IllegalArgumentException.class) 210 | public void wrongStateAccessTypeParameter() { 211 | 212 | final class UnknownStateBindingName extends TransactionProcessFunction { 213 | 214 | private static final long serialVersionUID = 1; 215 | 216 | @ProcessTransaction 217 | public void process(String input, Context out, @State("value") StateAccess value) { 218 | out.emit(1L); 219 | } 220 | } 221 | 222 | createSpecificationUnderTest( 223 | new UnknownStateBindingName(), 224 | BasicTypeInfo.STRING_TYPE_INFO, 225 | BasicTypeInfo.LONG_TYPE_INFO); 226 | } 227 | 228 | @Test(expected = IllegalArgumentException.class) 229 | public void stateBindingIsNotAnnotatedWithStateAnnotation() { 230 | 231 | final class MissingStateAnnotation extends TransactionProcessFunction { 232 | 233 | private static final long serialVersionUID = 1; 234 | 235 | @ProcessTransaction 236 | public void process(String input, Context out, StateAccess value) { 237 | out.emit(1L); 238 | } 239 | } 240 | 241 | createSpecificationUnderTest( 242 | new MissingStateAnnotation(), 243 | BasicTypeInfo.STRING_TYPE_INFO, 244 | BasicTypeInfo.LONG_TYPE_INFO); 245 | } 246 | 247 | @Test(expected = IllegalArgumentException.class) 248 | public void missingStateAccess() { 249 | 250 | final class MissingStateAccess extends TransactionProcessFunction { 251 | 252 | private static final long serialVersionUID = 1; 253 | 254 | @ProcessTransaction 255 | public void process(String input, Context out, @State("value") Long value) { 256 | out.emit(1L); 257 | } 258 | } 259 | 260 | createSpecificationUnderTest( 261 | new MissingStateAccess(), 262 | BasicTypeInfo.STRING_TYPE_INFO, 263 | BasicTypeInfo.LONG_TYPE_INFO); 264 | } 265 | 266 | } 267 | -------------------------------------------------------------------------------- /da-streamingledger-sdk/src/main/java/com/dataartisans/streamingledger/sdk/common/reflection/ByteBuddyProcessFunctionInvoker.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 Data Artisans GmbH 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.dataartisans.streamingledger.sdk.common.reflection; 18 | 19 | import com.dataartisans.streamingledger.sdk.api.TransactionProcessFunction; 20 | import com.dataartisans.streamingledger.sdk.api.TransactionProcessFunction.ProcessTransaction; 21 | import com.dataartisans.streamingledger.sdk.spi.StreamingLedgerSpec; 22 | import net.bytebuddy.ByteBuddy; 23 | import net.bytebuddy.NamingStrategy; 24 | import net.bytebuddy.TypeCache; 25 | import net.bytebuddy.TypeCache.Sort; 26 | import net.bytebuddy.description.method.MethodDescription; 27 | import net.bytebuddy.description.method.MethodDescription.ForLoadedConstructor; 28 | import net.bytebuddy.description.method.MethodDescription.ForLoadedMethod; 29 | import net.bytebuddy.description.modifier.FieldManifestation; 30 | import net.bytebuddy.description.modifier.Visibility; 31 | import net.bytebuddy.description.type.TypeDefinition; 32 | import net.bytebuddy.description.type.TypeDescription; 33 | import net.bytebuddy.description.type.TypeDescription.Generic; 34 | import net.bytebuddy.dynamic.DynamicType; 35 | import net.bytebuddy.dynamic.DynamicType.Builder; 36 | import net.bytebuddy.dynamic.DynamicType.Unloaded; 37 | import net.bytebuddy.dynamic.loading.ClassLoadingStrategy.Default; 38 | import net.bytebuddy.dynamic.scaffold.subclass.ConstructorStrategy; 39 | import net.bytebuddy.implementation.FieldAccessor; 40 | import net.bytebuddy.implementation.MethodCall; 41 | import net.bytebuddy.implementation.bytecode.assign.Assigner; 42 | import net.bytebuddy.implementation.bytecode.assign.Assigner.Typing; 43 | import net.bytebuddy.matcher.ElementMatchers; 44 | 45 | import java.lang.reflect.Constructor; 46 | import java.lang.reflect.InvocationTargetException; 47 | import java.lang.reflect.Method; 48 | import java.lang.reflect.Modifier; 49 | import java.util.Iterator; 50 | import java.util.concurrent.atomic.AtomicLong; 51 | 52 | import static java.util.Objects.requireNonNull; 53 | 54 | /** 55 | * A {@link ByteBuddy} backed factory of {@link ProcessFunctionInvoker}. 56 | */ 57 | public class ByteBuddyProcessFunctionInvoker { 58 | 59 | private static final TypeCache> CACHE = new TypeCache<>(Sort.SOFT); 60 | 61 | public static ProcessFunctionInvoker create(StreamingLedgerSpec spec) { 62 | final ClassLoader userClassLoader = classLoader(spec); 63 | final Class generatedClass = CACHE.findOrInsert(userClassLoader, spec, () -> generateAndLoadClass(spec)); 64 | 65 | try { 66 | return createInstance(generatedClass, spec); 67 | } 68 | catch (NoSuchMethodException | IllegalAccessException | InstantiationException | InvocationTargetException e) { 69 | throw new RuntimeException("Unable to create a new instance for " + spec.processFunction, e); 70 | } 71 | } 72 | 73 | private static Class> generateAndLoadClass( 74 | StreamingLedgerSpec spec) throws NoSuchMethodException { 75 | Unloaded unloaded = createDynamicTypeFromSpec(spec); 76 | return loadClass(unloaded, classLoader(spec)); 77 | } 78 | 79 | private static ProcessFunctionInvoker createInstance( 80 | Class generatedClass, 81 | StreamingLedgerSpec spec) 82 | throws NoSuchMethodException, IllegalAccessException, InvocationTargetException, InstantiationException { 83 | 84 | Constructor constructor = generatedClass.getDeclaredConstructor(spec.processFunction.getClass()); 85 | constructor.setAccessible(true); 86 | 87 | @SuppressWarnings("unchecked") 88 | ProcessFunctionInvoker instance = (ProcessFunctionInvoker) 89 | constructor.newInstance(spec.processFunction); 90 | 91 | return instance; 92 | } 93 | 94 | private static DynamicType.Unloaded createDynamicTypeFromSpec(StreamingLedgerSpec spec) 95 | throws NoSuchMethodException { 96 | PackageLocalNamingStrategy generatedTypeName = new PackageLocalNamingStrategy(spec.processFunction.getClass()); 97 | 98 | TypeDefinition generatedType = Generic.Builder.parameterizedType( 99 | ProcessFunctionInvoker.class, 100 | spec.inputType.getTypeClass(), 101 | spec.resultType.getTypeClass() 102 | ).build(); 103 | 104 | TypeDefinition processFunctionType = new TypeDescription.ForLoadedType(spec.processFunction.getClass()); 105 | ForLoadedConstructor superTypeConstructor = 106 | new ForLoadedConstructor(ProcessFunctionInvoker.class.getDeclaredConstructor()); 107 | MethodDescription processMethodType = processMethodTypeFromSpec(spec); 108 | 109 | Builder builder = configureByteBuddyBuilder( 110 | generatedTypeName, 111 | generatedType, 112 | processFunctionType, 113 | superTypeConstructor, 114 | processMethodType, 115 | spec.stateBindings.size()); 116 | 117 | return builder.make(); 118 | } 119 | 120 | private static Builder configureByteBuddyBuilder( 121 | PackageLocalNamingStrategy generatedTypeName, 122 | TypeDefinition generatedType, 123 | TypeDefinition processFunctionType, 124 | ForLoadedConstructor superTypeConstructor, 125 | MethodDescription processMethodType, 126 | int numberOfStateBindings) { 127 | 128 | return new ByteBuddy() 129 | // final class extends { 130 | .with(generatedTypeName) 131 | .subclass(generatedType, ConstructorStrategy.Default.NO_CONSTRUCTORS).modifiers(Modifier.FINAL) 132 | // private final delegate; 133 | .defineField("delegate", processFunctionType, Visibility.PRIVATE, FieldManifestation.FINAL) 134 | // public ( delegate) { 135 | // super(); 136 | // this.delegate = delegate; 137 | // } 138 | .defineConstructor(Modifier.PUBLIC) 139 | .withParameters(processFunctionType) 140 | .intercept(MethodCall.invoke(superTypeConstructor) 141 | .andThen(FieldAccessor.ofField("delegate").setsArgumentAt(0)) 142 | ) 143 | // invoke(input, context, StateAccess[] arguments) { 144 | // this.delegate.invoke(input, context, arguments[0], arguments[1], .. arguments[n - 1]); 145 | // } 146 | .method(ElementMatchers.named("invoke")) 147 | .intercept(MethodCall.invoke(processMethodType) 148 | .onField("delegate") 149 | .withArgument(0, 1) // event & context 150 | .withArgumentArrayElements(2, numberOfStateBindings) // StateAccess 151 | .withAssigner(Assigner.DEFAULT, Typing.STATIC) 152 | ); 153 | } 154 | 155 | private static MethodDescription processMethodTypeFromSpec(StreamingLedgerSpec spec) { 156 | Class processClass = spec.processFunction.getClass(); 157 | Iterator methods = Methods.findAnnotatedMethods(processClass, ProcessTransaction.class); 158 | if (!methods.hasNext()) { 159 | throw new IllegalArgumentException("Unable to find an annotated method on " + processClass.getSimpleName()); 160 | } 161 | Method method = methods.next(); 162 | if (methods.hasNext()) { 163 | throw new IllegalArgumentException( 164 | "Was expecting a single method annotated with a ProcessTransaction, but found more."); 165 | } 166 | return new ForLoadedMethod(method); 167 | } 168 | 169 | @SuppressWarnings("unchecked") 170 | private static Class> loadClass( 171 | DynamicType.Unloaded unloaded, 172 | ClassLoader classLoader) { 173 | return (Class>) 174 | unloaded 175 | .load(classLoader, Default.INJECTION) 176 | .getLoaded(); 177 | } 178 | 179 | private static ClassLoader classLoader(StreamingLedgerSpec spec) { 180 | final Class userClass = spec.processFunction.getClass(); 181 | return userClass.getClassLoader(); 182 | } 183 | 184 | /** 185 | * A naming strategy for generated classes. 186 | * 187 | *

The following name format is produced: user-package . superClass $ user-className $ sequence number 188 | */ 189 | private static final class PackageLocalNamingStrategy extends NamingStrategy.AbstractBase { 190 | private static final AtomicLong sequence = new AtomicLong(); 191 | private final String packageName; 192 | private final String className; 193 | 194 | PackageLocalNamingStrategy(Class superType) { 195 | requireNonNull(superType); 196 | Package aPackage = superType.getPackage(); 197 | if (aPackage == null) { 198 | this.packageName = ""; 199 | } 200 | else { 201 | this.packageName = aPackage.getName(); 202 | } 203 | this.className = superType.getSimpleName(); 204 | } 205 | 206 | @Override 207 | protected String name(TypeDescription superClass) { 208 | StringBuilder sb = new StringBuilder(); 209 | if (!packageName.isEmpty()) { 210 | sb.append(packageName); 211 | sb.append('.'); 212 | } 213 | sb.append(superClass.getSimpleName()); 214 | sb.append("$"); 215 | sb.append(className); 216 | sb.append("$"); 217 | sb.append(sequence.incrementAndGet()); 218 | return sb.toString(); 219 | } 220 | } 221 | 222 | } 223 | -------------------------------------------------------------------------------- /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 | 203 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 17 | 19 | 20 | 4.0.0 21 | 22 | com.data-artisans.streamingledger 23 | da-streamingledger 24 | 1.1-SNAPSHOT 25 | 26 | da-streamingledger 27 | pom 28 | 29 | Serializable ACID transactions on streaming data 30 | http://data-artisans.com 31 | 32 | https://github.com/dataArtisans/da-streamingledger 33 | 34 | 35 | 36 | 37 | jdoe 38 | info@data-artisans.com 39 | http://data-artisans.com/ 40 | data Artisans 41 | 42 | 43 | 44 | 45 | 46 | 47 | 48 | 49 | The Apache Software License, Version 2.0 50 | http://www.apache.org/licenses/LICENSE-2.0.txt 51 | repo 52 | 53 | 54 | 55 | 56 | 57 | 58 | UTF-8 59 | UTF-8 60 | 1.8 61 | 1.6.0 62 | 1.8.0 63 | 3.0.1 64 | 65 | 66 | 67 | 68 | 69 | da-streamingledger-sdk 70 | da-streamingledger-runtime-serial 71 | da-streamingledger-examples 72 | 73 | 74 | 75 | 76 | 77 | 78 | 79 | 80 | org.slf4j 81 | slf4j-api 82 | 83 | 84 | 85 | com.google.code.findbugs 86 | jsr305 87 | 88 | 89 | 90 | 91 | 92 | junit 93 | junit 94 | 4.12 95 | jar 96 | test 97 | 98 | 99 | 100 | 101 | 102 | org.slf4j 103 | slf4j-log4j12 104 | jar 105 | test 106 | 107 | 108 | 109 | log4j 110 | log4j 111 | jar 112 | test 113 | 114 | 115 | 116 | 117 | 118 | 119 | com.google.code.findbugs 120 | jsr305 121 | 1.3.9 122 | 123 | 124 | org.slf4j 125 | slf4j-api 126 | 1.7.7 127 | 128 | 129 | org.slf4j 130 | slf4j-log4j12 131 | 1.7.7 132 | 133 | 134 | log4j 135 | log4j 136 | 1.2.17 137 | 138 | 139 | 140 | 141 | 142 | 143 | 144 | 145 | 146 | 147 | 148 | org.apache.maven.plugins 149 | maven-compiler-plugin 150 | 3.1 151 | 152 | ${java.version} 153 | ${java.version} 154 | -Xlint:all 155 | 156 | 157 | 158 | 159 | 160 | org.apache.maven.plugins 161 | maven-jar-plugin 162 | 2.4 163 | 164 | 165 | 166 | true 167 | true 168 | 169 | 170 | 171 | 172 | 173 | 174 | 175 | org.apache.maven.plugins 176 | maven-source-plugin 177 | 2.2.1 178 | 179 | 180 | attach-sources 181 | 182 | jar 183 | 184 | 185 | 186 | 187 | 188 | org.apache.maven.plugins 189 | maven-javadoc-plugin 190 | 2.9.1 191 | 192 | true 193 | -Xdoclint:none 194 | false 195 | 196 | 197 | 198 | attach-javadocs 199 | 200 | jar 201 | 202 | 203 | 204 | 205 | 206 | 207 | 208 | org.apache.maven.plugins 209 | maven-surefire-plugin 210 | 211 | 2.18.1 212 | 213 | 2C 214 | 215 | 0${surefire.forkNumber} 216 | log4j-test.properties 217 | 218 | -Xms256m -Xmx2048m -Dmvn.forkNumber=${surefire.forkNumber} -XX:+UseG1GC 219 | 220 | 221 | 222 | unit-tests 223 | test 224 | 225 | test 226 | 227 | 228 | 229 | **/*Test.* 230 | 231 | true 232 | 233 | 234 | 235 | integration-tests 236 | integration-test 237 | 238 | test 239 | 240 | 241 | 242 | **/*ITCase.* 243 | 244 | false 245 | 246 | 247 | 248 | 249 | 250 | 251 | 252 | org.apache.rat 253 | apache-rat-plugin 254 | 0.12 255 | false 256 | 257 | 258 | verify 259 | 260 | check 261 | 262 | 263 | 264 | 265 | false 266 | false 267 | 0 268 | 269 | 270 | Data Artisans GmbH 271 | 272 | 273 | 274 | 275 | Apache License Version 2.0 276 | 277 | 278 | 279 | README.md 280 | **/.*/** 281 | **/target/** 282 | tools/maven/** 283 | tools/intellij/** 284 | **/.idea/** 285 | **/*.iml 286 | **/dependency-reduced-pom.xml 287 | 288 | 289 | 290 | 291 | 292 | 293 | org.apache.maven.plugins 294 | maven-checkstyle-plugin 295 | 2.17 296 | 297 | 298 | com.puppycrawl.tools 299 | checkstyle 300 | 8.4 301 | 302 | 303 | 304 | 305 | validate 306 | validate 307 | 308 | check 309 | 310 | 311 | 312 | 313 | tools/maven/checkstyle.xml 314 | tools/maven/checkstyle-suppressions.xml 315 | true 316 | true 317 | true 318 | 319 | 320 | 321 | 322 | 323 | com.github.spotbugs 324 | spotbugs-maven-plugin 325 | 3.1.1 326 | 327 | 328 | com.github.spotbugs 329 | spotbugs 330 | 3.1.1 331 | 332 | 333 | 334 | 335 | analyze-compile 336 | compile 337 | 338 | check 339 | 340 | 341 | 342 | 343 | Max 344 | Low 345 | tools/maven/spotbugs-exclude.xml 346 | 347 | 348 | 349 | 350 | 351 | org.apache.maven.plugins 352 | maven-enforcer-plugin 353 | 3.0.0-M1 354 | 355 | 356 | enforce-min-versions 357 | 358 | enforce 359 | 360 | 361 | 362 | 363 | [3.0.3,) 364 | 365 | 366 | ${java.version} 367 | 368 | 369 | 370 | 371 | 372 | dependency-convergence 373 | 374 | enforce 375 | 376 | 377 | 378 | 379 | 380 | 381 | 382 | 383 | 384 | 385 | 386 | 387 | 388 | 389 | 390 | 391 | ossrh 392 | https://oss.sonatype.org/content/repositories/snapshots 393 | 394 | 395 | ossrh 396 | https://oss.sonatype.org/service/local/staging/deploy/maven2/ 397 | 398 | 399 | 400 | 401 | 402 | release 403 | 404 | 405 | 406 | org.apache.maven.plugins 407 | maven-gpg-plugin 408 | 1.5 409 | 410 | 411 | sign-artifacts 412 | verify 413 | 414 | sign 415 | 416 | 417 | 418 | 419 | 420 | 421 | 422 | 423 | 424 | 425 | -------------------------------------------------------------------------------- /tools/maven/checkstyle.xml: -------------------------------------------------------------------------------- 1 | 2 | 20 | 21 | 24 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | 35 | 36 | 37 | 38 | 39 | 40 | 41 | 42 | 43 | 44 | 45 | 46 | 47 | 48 | 49 | 50 | 51 | 52 | 53 | 54 | 55 | 56 | 57 | 58 | 59 | 60 | 61 | 62 | 63 | 64 | 65 | 66 | 67 | 68 | 69 | 70 | 71 | 72 | 73 | 74 | 75 | 76 | 77 | 78 | 80 | 81 | 82 | 83 | 84 | 85 | 86 | 87 | 88 | 89 | 90 | 91 | 92 | 93 | 94 | 95 | 96 | 97 | 98 | 99 | 100 | 101 | 102 | 103 | 104 | 105 | 106 | 107 | 108 | 109 | 110 | 111 | 112 | 113 | 114 | 115 | 116 | 117 | 118 | 119 | 121 | 122 | 123 | 124 | 125 | 126 | 127 | 128 | 129 | 130 | 131 | 132 | 133 | 134 | 135 | 136 | 137 | 138 | 139 | 140 | 141 | 142 | 143 | 144 | 145 | 147 | 148 | 149 | 150 | 151 | 152 | 153 | 154 | 155 | 156 | 157 | 158 | 159 | 160 | 161 | 162 | 163 | 164 | 165 | 166 | 167 | 168 | 169 | 170 | 171 | 172 | 173 | 174 | 175 | 176 | 177 | 178 | 179 | 180 | 181 | 182 | 183 | 184 | 185 | 186 | 187 | 188 | 190 | 191 | 192 | 193 | 194 | 195 | 196 | 197 | 198 | 199 | 200 | 201 | 202 | 203 | 204 | 205 | 206 | 207 | 208 | 209 | 210 | 211 | 212 | 213 | 214 | 215 | 216 | 217 | 218 | 219 | 220 | 221 | 222 | 223 | 224 | 225 | 226 | 227 | 228 | 229 | 230 | 231 | 232 | 233 | 234 | 235 | 236 | 237 | 238 | 239 | 240 | 241 | 242 | 243 | 244 | 245 | 246 | 247 | 248 | 249 | 250 | 251 | 252 | 253 | 254 | 255 | 256 | 257 | 258 | 259 | 260 | 261 | 262 | 263 | 264 | 265 | 266 | 267 | 268 | 269 | 270 | 271 | 272 | 273 | 274 | 275 | 276 | 277 | 278 | 279 | 281 | 282 | 283 | 284 | 285 | 286 | 293 | 294 | 295 | 296 | 297 | 298 | 299 | 300 | 301 | 302 | 303 | 304 | 305 | 306 | 307 | 308 | 309 | 310 | 311 | 312 | 313 | 314 | 315 | 316 | 317 | 318 | 320 | 321 | 322 | 323 | 324 | 325 | 326 | 327 | 328 | 329 | 330 | 331 | 332 | 333 | 334 | 335 | 336 | 337 | 338 | 339 | 340 | 341 | 342 | 343 | 344 | 345 | 346 | 347 | 348 | 349 | 350 | 351 | 352 | 353 | 354 | 355 | 356 | 357 | 358 | 359 | 360 | 361 | 362 | 363 | 364 | 365 | 366 | 367 | 368 | 369 | 370 | 371 | 372 | 373 | --------------------------------------------------------------------------------