You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rocketmq.apache.org by du...@apache.org on 2021/08/03 06:50:53 UTC
[rocketmq-streams] 25/35: add script module add common module
This is an automated email from the ASF dual-hosted git repository.
duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-streams.git
commit 60409d597e503e7fb2d382ef07c367efb46200f0
Author: yuanxiaodong <ch...@alibaba-inc.com>
AuthorDate: Mon Aug 2 17:14:46 2021 +0800
add script module
add common module
---
rocketmq-streams-commons/pom.xml | 76 ++
.../streams/common/cache/CompressTable.java | 305 +++++++
.../common/cache/compress/AdditionStore.java | 161 ++++
.../streams/common/cache/compress/ByteArray.java | 103 +++
.../common/cache/compress/ByteArrayValueKV.java | 106 +++
.../streams/common/cache/compress/CacheKV.java | 371 ++++++++
.../streams/common/cache/compress/ICacheKV.java | 63 ++
.../streams/common/cache/compress/KVElement.java | 123 +++
.../cache/compress/impl/FixedLenRowCacheKV.java | 148 ++++
.../common/cache/compress/impl/IntValueKV.java | 195 +++++
.../streams/common/cache/compress/impl/KeySet.java | 109 +++
.../common/cache/compress/impl/ListValueKV.java | 66 ++
.../common/cache/compress/impl/MultiValueKV.java | 114 +++
.../common/cache/compress/impl/MutilValueKV.java | 116 +++
.../common/cache/compress/impl/SplitCache.java | 86 ++
.../common/cache/compress/impl/StringValueKV.java | 137 +++
.../streams/common/cache/softreference/ICache.java | 58 ++
.../cache/softreference/RebuildCacheElement.java | 26 +
.../softreference/impl/SoftReferenceCache.java | 127 +++
.../streams/common/calssscaner/AbstractScan.java | 257 ++++++
.../streams/common/channel/AbstractChannel.java | 220 +++++
.../rocketmq/streams/common/channel/IChannel.java | 34 +
.../AbstractSupportShuffleChannelBuilder.java | 31 +
.../common/channel/builder/IChannelBuilder.java | 70 ++
.../channel/builder/IShuffleChannelBuilder.java | 39 +
.../common/channel/impl/OutputPrintChannel.java | 47 +
.../common/channel/impl/PrintChannelBuilder.java | 47 +
.../channel/impl/file/FileChannelBuilder.java | 96 ++
.../streams/common/channel/impl/file/FileSink.java | 164 ++++
.../common/channel/impl/file/FileSource.java | 232 +++++
.../common/channel/impl/file/FileSplit.java | 64 ++
.../common/channel/impl/memory/MemoryCache.java | 83 ++
.../common/channel/impl/memory/MemoryChannel.java | 79 ++
.../channel/impl/memory/MemoryChannelBuilder.java | 71 ++
.../common/channel/impl/memory/MemorySink.java | 107 +++
.../common/channel/impl/memory/MemorySource.java | 92 ++
.../common/channel/impl/memory/MemorySplit.java | 52 ++
.../streams/common/channel/sink/AbstractSink.java | 243 +++++
.../channel/sink/AbstractSupportShuffleSink.java | 80 ++
.../streams/common/channel/sink/ISink.java | 104 +++
.../channel/sinkcache/DataSourceAutoFlushTask.java | 66 ++
.../common/channel/sinkcache/IMessageCache.java | 79 ++
.../channel/sinkcache/IMessageFlushCallBack.java | 31 +
.../impl/AbstractMutilSplitMessageCache.java | 137 +++
.../channel/sinkcache/impl/MessageCache.java | 142 +++
.../sinkcache/impl/MultiSplitMessageCache.java | 33 +
.../sinkcache/impl/MutilSplitMessageCache.java | 33 +
.../common/channel/source/AbstractBatchSource.java | 202 +++++
.../common/channel/source/AbstractSource.java | 536 ++++++++++++
.../source/AbstractSupportOffsetResetSource.java | 21 +
.../channel/source/AbstractUnreliableSource.java | 164 ++++
.../streams/common/channel/source/ISource.java | 90 ++
.../channel/source/ISplitChangedListener.java | 41 +
.../channel/source/systemmsg/NewSplitMessage.java | 26 +
.../source/systemmsg/RemoveSplitMessage.java | 26 +
.../source/systemmsg/SplitChangedMessage.java | 73 ++
.../streams/common/channel/split/ISplit.java | 49 ++
.../streams/common/checkpoint/CheckPoint.java | 78 ++
.../common/checkpoint/CheckPointManager.java | 279 ++++++
.../common/checkpoint/CheckPointMessage.java | 77 ++
.../streams/common/checkpoint/CheckPointState.java | 65 ++
.../streams/common/checkpoint/ICheckPoint.java | 31 +
.../common/classloader/ByteClassLoader.java | 30 +
.../common/classloader/FileClassLoader.java | 83 ++
.../common/classloader/IsolationClassLoader.java | 78 ++
.../common/compiler/CustomJavaCompiler.java | 248 ++++++
.../common/component/AbstractComponent.java | 208 +++++
.../streams/common/component/ComponentCreator.java | 293 +++++++
.../common/component/ConfigureDescriptor.java | 98 +++
.../streams/common/component/IComponent.java | 73 ++
.../streams/common/component/IgnoreNameSpace.java | 24 +
.../PropertyConfigureDescriptorManager.java | 104 +++
.../PropertyConfigureDiscriptorManager.java | 110 +++
.../common/configurable/AbstractConfigurable.java | 216 +++++
.../common/configurable/BasedConfigurable.java | 249 ++++++
.../IAfterConfiguableRefreshListerner.java | 25 +
.../streams/common/configurable/IConfigurable.java | 84 ++
.../configurable/IConfigurableIdentification.java | 40 +
.../configurable/IConfigurableListerner.java | 20 +
.../common/configurable/IConfigurableService.java | 136 +++
.../common/configurable/IFieldProcessor.java | 24 +
.../common/configurable/annotation/Changeable.java | 31 +
.../configurable/annotation/ENVDependence.java | 30 +
.../configurable/annotation/NoSerialized.java | 29 +
.../streams/common/configure/ConfigureFileKey.java | 142 +++
.../streams/common/context/AbstractContext.java | 364 ++++++++
.../streams/common/context/BatchMessageOffset.java | 123 +++
.../rocketmq/streams/common/context/Context.java | 33 +
.../rocketmq/streams/common/context/IMessage.java | 85 ++
.../streams/common/context/IgnoreMessage.java | 23 +
.../rocketmq/streams/common/context/Message.java | 129 +++
.../streams/common/context/MessageHeader.java | 360 ++++++++
.../streams/common/context/MessageOffset.java | 208 +++++
.../streams/common/context/UserDefinedMessage.java | 502 +++++++++++
.../streams/common/datatype/ArrayDataType.java | 146 ++++
.../streams/common/datatype/BaseDataType.java | 229 +++++
.../streams/common/datatype/BooleanDataType.java | 142 +++
.../streams/common/datatype/ByteDataType.java | 107 +++
.../common/datatype/ConfigurableDataType.java | 127 +++
.../streams/common/datatype/DataJsonable.java | 36 +
.../rocketmq/streams/common/datatype/DataType.java | 109 +++
.../streams/common/datatype/DateDataType.java | 153 ++++
.../streams/common/datatype/DoubleDataType.java | 156 ++++
.../streams/common/datatype/FloatDataType.java | 121 +++
.../common/datatype/GenericParameterDataType.java | 122 +++
.../common/datatype/GenericParamterDataType.java | 123 +++
.../streams/common/datatype/IJsonable.java | 34 +
.../streams/common/datatype/IntDataType.java | 122 +++
.../streams/common/datatype/JavaBeanDataType.java | 92 ++
.../streams/common/datatype/JsonableDataType.java | 104 +++
.../streams/common/datatype/ListDataType.java | 223 +++++
.../streams/common/datatype/LongDataType.java | 173 ++++
.../streams/common/datatype/MapDataType.java | 286 ++++++
.../common/datatype/NotSupportDataType.java | 74 ++
.../streams/common/datatype/NumberDataType.java | 84 ++
.../streams/common/datatype/SetDataType.java | 257 ++++++
.../streams/common/datatype/ShortDataType.java | 118 +++
.../streams/common/datatype/StringDataType.java | 83 ++
.../streams/common/dboperator/IDBDriver.java | 79 ++
.../common/disruptor/BufferFullFunction.java | 27 +
.../streams/common/disruptor/DisruptorEvent.java | 34 +
.../common/disruptor/DisruptorEventFactory.java | 26 +
.../common/disruptor/DisruptorProducer.java | 51 ++
.../streams/common/functions/FilterFunction.java | 24 +
.../streams/common/functions/ForEachFunction.java | 31 +
.../common/functions/ForEachMessageFunction.java | 30 +
.../streams/common/functions/Function.java | 19 +
.../streams/common/functions/MapFunction.java | 24 +
.../common/functions/MultiTableSplitFunction.java | 29 +
.../streams/common/functions/ReduceFunction.java | 24 +
.../streams/common/functions/SplitFunction.java | 30 +
.../common/functions/TableSplitFunction.java | 46 +
.../common/interfaces/IBaseStreamOperator.java | 33 +
.../streams/common/interfaces/IFilterService.java | 57 ++
.../streams/common/interfaces/IJDBCOperator.java | 77 ++
.../common/interfaces/IJsonobjectProcessor.java | 25 +
.../common/interfaces/ILineMessageProcessor.java | 32 +
.../streams/common/interfaces/IPropertyEnable.java | 39 +
.../common/interfaces/IScheduleExecutor.java | 27 +
.../streams/common/interfaces/IStreamOperator.java | 29 +
.../streams/common/interfaces/ISystemMessage.java | 20 +
.../common/interfaces/ISystemMessageProcessor.java | 23 +
.../streams/common/interfaces/ITransport.java | 32 +
.../streams/common/logger/LoggerCreator.java | 57 ++
.../streams/common/logger/LoggerOutputChannel.java | 79 ++
.../common/logger/MyDailyRollingFileAppender.java | 545 ++++++++++++
.../streams/common/metadata/AbstractMetaData.java | 247 ++++++
.../rocketmq/streams/common/metadata/MetaData.java | 227 +++++
.../streams/common/metadata/MetaDataAdapter.java | 48 +
.../streams/common/metadata/MetaDataField.java | 167 ++++
.../rocketmq/streams/common/model/BeanHolder.java | 36 +
.../rocketmq/streams/common/model/Entity.java | 49 ++
.../rocketmq/streams/common/model/NameCreator.java | 63 ++
.../rocketmq/streams/common/model/ServiceName.java | 28 +
.../streams/common/model/ThreadContext.java | 31 +
.../rocketmq/streams/common/monitor/IMonitor.java | 171 ++++
.../streams/common/monitor/MonitorFactory.java | 133 +++
.../common/monitor/TopologyFilterMonitor.java | 66 ++
.../common/monitor/group/GroupedMonitorInfo.java | 119 +++
.../common/monitor/group/MonitorCommander.java | 200 +++++
.../streams/common/monitor/impl/DipperMonitor.java | 510 +++++++++++
.../streams/common/monitor/impl/MonitorItem.java | 177 ++++
.../common/monitor/impl/NothingMonitorItem.java | 59 ++
.../common/monitor/impl/NothingMontior.java | 112 +++
.../optimization/CalculationResultCache.java | 64 ++
.../common/optimization/HyperscanRegex.java | 104 +++
.../streams/common/optimization/LikeRegex.java | 175 ++++
.../common/optimization/LogFingerprintFilter.java | 152 ++++
.../common/optimization/MessageGloableTrace.java | 80 ++
.../common/optimization/OptimizationRegex.java | 308 +++++++
.../optimization/SQLLogFingerprintFilter.java | 53 ++
.../AbstractMutilPipelineChainPipline.java | 193 ++++
.../streams/common/topology/ChainPipeline.java | 447 ++++++++++
.../streams/common/topology/ChainStage.java | 189 ++++
.../common/topology/builder/IStageBuilder.java | 39 +
.../common/topology/builder/PipelineBuilder.java | 370 ++++++++
.../common/topology/model/AbstractRule.java | 181 ++++
.../common/topology/model/AbstractScript.java | 110 +++
.../common/topology/model/AbstractStage.java | 298 +++++++
.../streams/common/topology/model/ILifeCycle.java | 33 +
.../common/topology/model/IStageHandle.java | 44 +
.../streams/common/topology/model/IWindow.java | 92 ++
.../streams/common/topology/model/Pipeline.java | 291 ++++++
.../topology/model/PipelineSourceJoiner.java | 48 +
.../streams/common/topology/model/Union.java | 63 ++
.../stages/AbstractStatelessChainStage.java | 41 +
.../topology/stages/AbstractWindowStage.java | 90 ++
.../common/topology/stages/FilterChainStage.java | 357 ++++++++
.../common/topology/stages/JoinChainStage.java | 160 ++++
.../common/topology/stages/NewSQLChainStage.java | 82 ++
.../common/topology/stages/OpenAPIChainStage.java | 80 ++
.../common/topology/stages/OutputChainStage.java | 231 +++++
.../common/topology/stages/PythonChainStage.java | 80 ++
.../topology/stages/RightJoinChainStage.java | 92 ++
.../common/topology/stages/SQLChainStage.java | 81 ++
.../common/topology/stages/ScriptChainStage.java | 124 +++
.../topology/stages/SubPiplineChainStage.java | 410 +++++++++
.../common/topology/stages/UnionChainStage.java | 51 ++
.../common/topology/stages/WindowChainStage.java | 83 ++
.../common/topology/stages/udf/FilterOperator.java | 37 +
.../topology/stages/udf/FlatMapOperator.java | 61 ++
.../common/topology/stages/udf/IReducer.java | 35 +
.../common/topology/stages/udf/IRedurce.java | 36 +
.../common/topology/stages/udf/MapOperator.java | 35 +
.../common/topology/stages/udf/StageBuilder.java | 92 ++
.../common/topology/stages/udf/UDFChainStage.java | 74 ++
.../topology/stages/udf/UDFUnionChainStage.java | 92 ++
.../common/transport/AbstractFileTransport.java | 41 +
.../streams/common/transport/IFileTransport.java | 68 ++
.../rocketmq/streams/common/utils/AESUtil.java | 108 +++
.../rocketmq/streams/common/utils/Base64Utils.java | 100 +++
.../rocketmq/streams/common/utils/ClassUtil.java | 157 ++++
.../streams/common/utils/CollectionUtil.java | 65 ++
.../rocketmq/streams/common/utils/CommandUtil.java | 184 ++++
.../streams/common/utils/CompressUtil.java | 88 ++
.../streams/common/utils/ConfigurableUtil.java | 248 ++++++
.../streams/common/utils/ContantsUtil.java | 355 ++++++++
.../rocketmq/streams/common/utils/DESUtils.java | 142 +++
.../streams/common/utils/DataTypeUtil.java | 504 +++++++++++
.../rocketmq/streams/common/utils/DateUtil.java | 603 +++++++++++++
.../common/utils/DipperThreadLocalUtil.java | 35 +
.../rocketmq/streams/common/utils/ENVUtile.java | 55 ++
.../rocketmq/streams/common/utils/FileUtil.java | 973 +++++++++++++++++++++
.../rocketmq/streams/common/utils/IOUtil.java | 99 +++
.../rocketmq/streams/common/utils/IPUtil.java | 201 +++++
.../streams/common/utils/InstantiationUtil.java | 118 +++
.../streams/common/utils/Ip2LongUtils.java | 296 +++++++
.../rocketmq/streams/common/utils/JarUtil.java | 147 ++++
.../streams/common/utils/JsonableUtil.java | 61 ++
.../streams/common/utils/LogParserUtil.java | 241 +++++
.../rocketmq/streams/common/utils/MapKeyUtil.java | 155 ++++
.../rocketmq/streams/common/utils/MessageUtil.java | 102 +++
.../streams/common/utils/NameCreatorUtil.java | 61 ++
.../rocketmq/streams/common/utils/NumberUtils.java | 179 ++++
.../rocketmq/streams/common/utils/PrintUtil.java | 205 +++++
.../streams/common/utils/PropertiesUtils.java | 144 +++
.../streams/common/utils/RandomStrUtil.java | 38 +
.../rocketmq/streams/common/utils/ReflectUtil.java | 882 +++++++++++++++++++
.../rocketmq/streams/common/utils/RegexUtil.java | 21 +
.../rocketmq/streams/common/utils/RuntimeUtil.java | 63 ++
.../streams/common/utils/SQLFormatterUtil.java | 362 ++++++++
.../rocketmq/streams/common/utils/SQLUtil.java | 413 +++++++++
.../streams/common/utils/ScheduleUtil.java | 52 ++
.../rocketmq/streams/common/utils/ShellUtil.java | 189 ++++
.../rocketmq/streams/common/utils/StringUtil.java | 369 ++++++++
.../streams/common/utils/SyslogParserUtil.java | 25 +
.../rocketmq/streams/common/utils/ThreadUtil.java | 29 +
.../rocketmq/streams/common/utils/TraceUtil.java | 118 +++
.../common/cache/compress/IntValueKVTest.java | 66 ++
.../softreference/SoftReferenceCacheTest.java | 55 ++
.../rocketmq/streams/common/channel/SinkTest.java | 73 ++
.../streams/common/channel/SourceTest.java | 75 ++
.../streams/common/classscan/ClassScanTest.java | 34 +
.../common/compiler/CustomJavaCompilerTest.java | 35 +
.../streams/common/datatype/DataTypeTest.java | 39 +
.../streams/common/stages/SelfChainStageTest.java | 36 +
.../src/test/resources/log4j.xml | 16 +
rocketmq-streams-script/pom.xml | 33 +
.../rocketmq/streams/script/ScriptComponent.java | 76 ++
.../streams/script/annotation/Function.java | 31 +
.../streams/script/annotation/FunctionMethod.java | 51 ++
.../script/annotation/FunctionMethodAilas.java | 37 +
.../script/annotation/FunctionParamter.java | 44 +
.../streams/script/annotation/UDAFFunction.java | 34 +
.../streams/script/context/FunctionContext.java | 87 ++
.../function/aggregation/AverageAccumulator.java | 98 +++
.../function/aggregation/ConcatAccumulator.java | 96 ++
.../aggregation/ConcatDistinctAccumulator.java | 98 +++
.../function/aggregation/CountAccumulator.java | 79 ++
.../function/aggregation/DistinctAccumulator.java | 75 ++
.../function/aggregation/MaxAccumulator.java | 111 +++
.../function/aggregation/MinAccumulator.java | 111 +++
.../function/aggregation/SumAccumulator.java | 82 ++
.../script/function/impl/bool/BooleanFunction.java | 39 +
.../script/function/impl/common/EchoFunction.java | 47 +
.../function/impl/condition/EqualsFunction.java | 155 ++++
.../impl/condition/ExecuteByConditionFuction.java | 36 +
.../impl/condition/GreateEqualsFunction.java | 74 ++
.../function/impl/condition/GreaterFunction.java | 69 ++
.../script/function/impl/condition/IFFunction.java | 39 +
.../impl/condition/InConditionFunction.java | 99 +++
.../impl/condition/LessEqualsFunction.java | 71 ++
.../function/impl/condition/LessFunction.java | 69 ++
.../function/impl/context/ContextFunction.java | 44 +
.../function/impl/date/AddMonthFunction.java | 73 ++
.../impl/date/CurrentTimestampFunction.java | 63 ++
.../script/function/impl/date/DateAddFunction.java | 146 ++++
.../function/impl/date/DateDiffFunction.java | 124 +++
.../function/impl/date/DateFormatFunction.java | 92 ++
.../function/impl/date/DatePartFunction.java | 218 +++++
.../function/impl/date/DateToCharFunction.java | 93 ++
.../function/impl/date/DateTruncFunction.java | 143 +++
.../function/impl/date/DateUtilFunction.java | 130 +++
.../function/impl/date/DateUtileFunction.java | 134 +++
.../script/function/impl/date/DayFunction.java | 62 ++
.../function/impl/date/FromUnixTimeFunction.java | 72 ++
.../script/function/impl/date/GetDateFunction.java | 43 +
.../script/function/impl/date/IsDateFunction.java | 54 ++
.../script/function/impl/date/LastDayFunction.java | 69 ++
.../script/function/impl/date/NextDayFunction.java | 100 +++
.../script/function/impl/date/NowFuntion.java | 46 +
.../script/function/impl/date/QuarterFuction.java | 75 ++
.../script/function/impl/date/QuarterFunction.java | 76 ++
.../function/impl/date/UnixTimeStampFunction.java | 77 ++
.../function/impl/date/WeekOfYearFunction.java | 59 ++
.../script/function/impl/date/WeekdayFunction.java | 60 ++
.../script/function/impl/env/ENVFunction.java | 47 +
.../impl/field/AdditionalFiledFunction.java | 318 +++++++
.../function/impl/field/CoalesceFunction.java | 51 ++
.../function/impl/field/ExtraFieldFunction.java | 67 ++
.../script/function/impl/field/FieldFunction.java | 155 ++++
.../function/impl/field/RemoveFieldFunction.java | 59 ++
.../function/impl/field/RetainFieldFunction.java | 51 ++
.../script/function/impl/filter/BreakFunction.java | 41 +
.../function/impl/filter/FilterFunction.java | 74 ++
.../function/impl/flatmap/SplitArrayFunction.java | 164 ++++
.../function/impl/header/HeaderFunction.java | 48 +
.../function/impl/json/JsonArrayFunction.java | 230 +++++
.../function/impl/json/JsonCreatorFunction.java | 218 +++++
.../function/impl/json/JsonValueFunction.java | 108 +++
.../script/function/impl/math/AbsFunction.java | 118 +++
.../script/function/impl/math/AcosFunction.java | 109 +++
.../function/impl/math/AdditionFunction.java | 62 ++
.../script/function/impl/math/AsinFunction.java | 109 +++
.../script/function/impl/math/AtanFunction.java | 88 ++
.../script/function/impl/math/BinFunction.java | 54 ++
.../script/function/impl/math/BitFunction.java | 201 +++++
.../script/function/impl/math/BroundFunction.java | 113 +++
.../function/impl/math/CardinalityFunction.java | 49 ++
.../script/function/impl/math/CbrtFunction.java | 67 ++
.../script/function/impl/math/CeilFunction.java | 109 +++
.../script/function/impl/math/ConvFunction.java | 244 ++++++
.../script/function/impl/math/CosFunction.java | 109 +++
.../script/function/impl/math/CosHFunction.java | 109 +++
.../script/function/impl/math/CotFunction.java | 109 +++
.../script/function/impl/math/DegreesFunction.java | 90 ++
.../function/impl/math/DivisionFunction.java | 62 ++
.../script/function/impl/math/EFunction.java | 42 +
.../script/function/impl/math/ExpFunction.java | 109 +++
.../function/impl/math/FactorialFunction.java | 79 ++
.../script/function/impl/math/FloorFunction.java | 115 +++
.../script/function/impl/math/HexFunction.java | 69 ++
.../script/function/impl/math/LnFunction.java | 109 +++
.../script/function/impl/math/Log10Function.java | 105 +++
.../script/function/impl/math/Log2Function.java | 105 +++
.../script/function/impl/math/LogFunction.java | 114 +++
.../script/function/impl/math/MathFunction.java | 132 +++
.../function/impl/math/MultiplicationFunction.java | 63 ++
.../function/impl/math/OperationFunction.java | 123 +++
.../script/function/impl/math/PIFunction.java | 42 +
.../script/function/impl/math/PowerFunction.java | 114 +++
.../script/function/impl/math/RadiansFunction.java | 48 +
.../script/function/impl/math/RandFunction.java | 62 ++
.../script/function/impl/math/RoundFunction.java | 117 +++
.../function/impl/math/ShiftLeftFunction.java | 52 ++
.../function/impl/math/ShiftRightFunction.java | 52 ++
.../impl/math/ShiftRightUnSignedFunction.java | 52 ++
.../script/function/impl/math/SignFunction.java | 50 ++
.../script/function/impl/math/SinFunction.java | 109 +++
.../script/function/impl/math/SinHFunction.java | 109 +++
.../script/function/impl/math/SqrtFunction.java | 109 +++
.../function/impl/math/SubtractionFunction.java | 62 ++
.../script/function/impl/math/TanFunction.java | 108 +++
.../script/function/impl/math/TanHFunction.java | 109 +++
.../script/function/impl/math/TruncFunction.java | 118 +++
.../script/function/impl/math/UnHexFunction.java | 64 ++
.../script/function/impl/mock/MockFunction.java | 39 +
.../function/impl/offset/OffsetFunction.java | 67 ++
.../script/function/impl/parser/GrokFunction.java | 91 ++
.../function/impl/parser/LogParserFunction.java | 85 ++
.../function/impl/parser/Paser2JsonFunction.java | 169 ++++
.../function/impl/parser/PaserBySplitFunction.java | 172 ++++
.../function/impl/parser/RegexParserFunction.java | 85 ++
.../function/impl/platform/BlinkUDFFunction.java | 33 +
.../function/impl/python/GroovyFunction.java | 52 ++
.../function/impl/python/PythonFunction.java | 49 ++
.../script/function/impl/router/RouteFunction.java | 37 +
.../script/function/impl/sql/SQLFunction.java | 56 ++
.../script/function/impl/string/AsciiFunction.java | 52 ++
.../function/impl/string/Base64Function.java | 86 ++
.../function/impl/string/BlinkStringFunction.java | 53 ++
.../impl/string/CharMatchCountFunction.java | 61 ++
.../script/function/impl/string/ChrFunction.java | 49 ++
.../function/impl/string/HashCodeFunction.java | 50 ++
.../function/impl/string/InitCapFunction.java | 53 ++
.../script/function/impl/string/InstrFunction.java | 136 +++
.../function/impl/string/InstrtFunction.java | 134 +++
.../function/impl/string/Ip2RegionFunction.java | 43 +
.../function/impl/string/IsEncodingFunction.java | 62 ++
.../function/impl/string/KeyValueFunction.java | 120 +++
.../script/function/impl/string/LTrimFunction.java | 49 ++
.../function/impl/string/LengthBFunction.java | 50 ++
.../function/impl/string/LengthFunction.java | 50 ++
.../script/function/impl/string/Md5Function.java | 84 ++
.../script/function/impl/string/NullFunction.java | 43 +
.../function/impl/string/ParseUrlFunction.java | 102 +++
.../script/function/impl/string/RTrimFunction.java | 50 ++
.../function/impl/string/RegexCountFunction.java | 113 +++
.../script/function/impl/string/RegexFunction.java | 159 ++++
.../function/impl/string/RegexInstrFunction.java | 180 ++++
.../function/impl/string/RegexReplaceFunction.java | 92 ++
.../function/impl/string/RegexSubStrFunction.java | 127 +++
.../function/impl/string/RepeatFunction.java | 60 ++
.../function/impl/string/ReplaceFunction.java | 56 ++
.../function/impl/string/ReverseFunction.java | 50 ++
.../function/impl/string/SoundxFunction.java | 112 +++
.../script/function/impl/string/SpaceFunction.java | 60 ++
.../function/impl/string/SplitPartFunction.java | 133 +++
.../impl/string/SubStringIndexFunction.java | 128 +++
.../function/impl/string/ToCharFunction.java | 119 +++
.../function/impl/string/ToLowerFunction.java | 49 ++
.../function/impl/string/ToUpperFunction.java | 49 ++
.../function/impl/string/TransLateFunction.java | 57 ++
.../script/function/impl/string/TrimFunction.java | 50 ++
.../script/function/impl/string/UUIDFunction.java | 39 +
.../function/impl/string/UrlDecodeFunction.java | 85 ++
.../function/impl/string/UrlEncodeFunction.java | 125 +++
.../script/function/impl/type/CastFunction.java | 73 ++
.../script/function/impl/udtf/UDTFFunction.java | 61 ++
.../function/impl/window/WindowFunction.java | 64 ++
.../script/function/model/FunctionConfigure.java | 573 ++++++++++++
.../function/model/FunctionConfigureMap.java | 109 +++
.../script/function/model/FunctionInfo.java | 101 +++
.../script/function/model/FunctionInfoMap.java | 84 ++
.../script/function/model/FunctionType.java | 45 +
.../function/service/IDipperInterfaceAdpater.java | 46 +
.../script/function/service/IFunctionService.java | 130 +++
.../service/impl/DefaultFunctionServiceImpl.java | 396 +++++++++
.../function/service/impl/ScanFunctionService.java | 96 ++
.../script/operator/AbstractScriptOperator.java | 82 ++
.../operator/expression/GroupScriptExpression.java | 239 +++++
.../operator/expression/ScriptExpression.java | 327 +++++++
.../operator/expression/ScriptParameter.java | 135 +++
.../script/operator/impl/AggregationScript.java | 211 +++++
.../operator/impl/FileGroovyScriptOperator.java | 33 +
.../operator/impl/FileJPythonScriptOperatore.java | 32 +
.../script/operator/impl/FunctionScript.java | 247 ++++++
.../script/operator/impl/GroovyScriptOperator.java | 108 +++
.../operator/impl/JPythonScriptOperator.java | 95 ++
.../script/operator/impl/JavaScriptOperator.java | 63 ++
.../script/operator/impl/ScriptOperator.java | 29 +
.../optimization/AbstractFunctionOptimization.java | 47 +
.../script/optimization/CompileParameter.java | 132 +++
.../optimization/CompileScriptExpression.java | 121 +++
.../script/optimization/EqualsOptimization.java | 44 +
.../script/optimization/IFunctionOptimization.java | 38 +
.../optimization/OptimizationScriptExpression.java | 67 ++
.../script/optimization/RegexOptimization.java | 43 +
.../script/optimization/ScriptOptimization.java | 407 +++++++++
.../script/parser/IScriptExpressionParser.java | 41 +
.../parser/ScriptExpressionParserFactory.java | 70 ++
.../parser/imp/ConditionExpressionParser.java | 227 +++++
.../parser/imp/FunctionExpressionParser.java | 361 ++++++++
.../streams/script/parser/imp/FunctionParser.java | 166 ++++
.../streams/script/service/IAccumulator.java | 67 ++
.../script/service/IFunctionFieldDependent.java | 40 +
.../streams/script/service/IScriptExpression.java | 59 ++
.../streams/script/service/IScriptParamter.java | 43 +
.../streams/script/service/IScriptService.java | 84 ++
.../streams/script/service/IScriptUDFInit.java | 37 +
.../script/service/impl/ScriptServiceImpl.java | 121 +++
.../streams/script/service/udf/UDAFScript.java | 230 +++++
.../streams/script/service/udf/UDFScript.java | 296 +++++++
.../streams/script/service/udf/UDTFScript.java | 113 +++
.../streams/script/utils/FunctionUtils.java | 254 ++++++
.../rocketmq/streams/script/utils/MatchUtil.java | 82 ++
.../streams/script/utils/RandomStrUtil.java | 38 +
.../streams/script/utils/ScriptParserUtil.java | 23 +
.../streams/script/utils/StringMatchUtil.java | 87 ++
.../rocketmq/streams/script/utils/UniqID.java | 66 ++
.../src/main/resources/log4j.xml | 20 +
.../streams/script/function/FunctionTest.java | 89 ++
.../streams/script/function/GrokFunctionTest.java | 44 +
.../script/function/GroovyFunctionTest.java | 52 ++
.../script/function/JPythonFunctionTest.java | 48 +
.../function/aggregation/AccumulatorTest.java | 235 +++++
.../script/operator/ScriptOperatorTest.java | 105 +++
.../streams/script/parser/ScriptParserTest.java | 43 +
.../src/test/resources/log4j.xml | 20 +
.../src/test/resources/python_script.py | 22 +
480 files changed, 54314 insertions(+)
diff --git a/rocketmq-streams-commons/pom.xml b/rocketmq-streams-commons/pom.xml
new file mode 100755
index 0000000..e152477
--- /dev/null
+++ b/rocketmq-streams-commons/pom.xml
@@ -0,0 +1,76 @@
+<?xml version="1.0" encoding="utf-8"?>
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <groupId>org.apache.rocketmq</groupId>
+ <artifactId>rocketmq-streams</artifactId>
+ <version>2.0.0-SNAPSHOT</version>
+ </parent>
+
+ <artifactId>rocketmq-streams-commons</artifactId>
+ <name>ROCKETMQ STREAMS :: commons</name>
+ <packaging>jar</packaging>
+ <dependencies>
+ <!-- http -->
+ <dependency>
+ <groupId>org.apache.commons</groupId>
+ <artifactId>commons-lang3</artifactId>
+ <version>3.11</version>
+ </dependency>
+
+ <!-- 测试依赖 -->
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.alibaba</groupId>
+ <artifactId>fastjson</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>commons-logging</groupId>
+ <artifactId>commons-logging</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>commons-io</groupId>
+ <artifactId>commons-io</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>log4j</groupId>
+ <artifactId>log4j</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.google.code.gson</groupId>
+ <artifactId>gson</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.google.auto.service</groupId>
+ <artifactId>auto-service</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>com.lmax</groupId>
+ <artifactId>disruptor</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.gliwka.hyperscan</groupId>
+ <artifactId>hyperscan</artifactId>
+ <version>5.4.0-2.0.0</version>
+ </dependency>
+
+ <dependency>
+ <groupId>net.java.dev.jna</groupId>
+ <artifactId>platform</artifactId>
+ <version>3.5.2</version>
+ </dependency>
+
+ </dependencies>
+</project>
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/CompressTable.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/CompressTable.java
new file mode 100644
index 0000000..bdc1d41
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/CompressTable.java
@@ -0,0 +1,305 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.cache;
+
+import org.apache.rocketmq.streams.common.datatype.DataType;
+import org.apache.rocketmq.streams.common.datatype.NotSupportDataType;
+import org.apache.rocketmq.streams.common.datatype.StringDataType;
+import org.apache.rocketmq.streams.common.utils.DataTypeUtil;
+
+import java.util.*;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * 压缩表,行数据以byte[][]存放
+ */
+public class CompressTable {
+
+ /**
+ * 列名和位置,根据列名生成一个位置
+ */
+ protected Map<String, Integer> cloumnName2Index = new HashMap<>();
+
+ /**
+ * 位置和列名,根据位置获取列名
+ */
+ protected Map<Integer, String> index2ColumnName = new HashMap<>();
+
+ /**
+ * 列名和类型的映射关系
+ */
+ protected Map<String, DataType> cloumnName2DatatType = new HashMap<>();
+
+ /**
+ * 列名当前索引值。根据获取的列的先后顺序,建立序号
+ */
+ protected AtomicInteger index = new AtomicInteger(0);
+
+ /**
+ * 表的全部数据,一行是一个byte[][]
+ */
+ protected List<byte[][]> rows = new ArrayList<>();
+
+ /**
+ * 总字节数
+ */
+ protected AtomicInteger byteCount = new AtomicInteger(0);
+
+ /**
+ * 创建迭代器,可以循环获取全部数据,一行数据是Map<String, Object>
+ *
+ * @return
+ */
+ public Iterator<Map<String, Object>> newIterator() {
+ return new Iterator<Map<String, Object>>() {
+
+ protected int rowIndex = 0;
+
+ @Override
+ public boolean hasNext() {
+ return rowIndex < rows.size();
+ }
+
+ @Override
+ public Map<String, Object> next() {
+ Map<String, Object> row = byte2Row(rows.get(rowIndex));
+ rowIndex++;
+ return row;
+ }
+ };
+ }
+
+ /**
+ * 增加一行数据,会进行序列化成二进制数组存储。数字类型会有较好的压缩效果,字符串未做压缩
+ *
+ * @param row
+ * @return
+ */
+ public Integer addRow(Map<String, Object> row) {
+ CountHolder countHolder = new CountHolder();
+ byte[][] values = row2Byte(row, countHolder);
+ byteCount.addAndGet(countHolder.count);
+ return saveRowByte(values, countHolder.count);
+ }
+
+ /**
+ * 获取一行数据,会反序列化为Map<String, Object>
+ *
+ * @param index 行号, List<byte[][]> 的下标
+ * @return
+ */
+ public Map<String, Object> getRow(Integer index) {
+ byte[][] bytes = loadRowByte(index);
+ if (bytes == null) {
+ return null;
+ }
+ return byte2Row(bytes);
+ }
+
+ /**
+ * 保存row到list
+ *
+ * @param values
+ * @return
+ */
+ protected Integer saveRowByte(byte[][] values, int byteSize) {
+ this.rows.add(values);
+ return rows.size() - 1;
+ }
+
+ /**
+ * 从list中加载行
+ *
+ * @param index
+ * @return
+ */
+ protected byte[][] loadRowByte(Integer index) {
+ return this.rows.get(index);
+ }
+
+ /**
+ * 把一个row行转换成byte[][]数组
+ *
+ * @param row
+ * @return
+ */
+ public byte[][] row2Byte(Map<String, Object> row) {
+ return row2Byte(row, null);
+ }
+
+ protected class CountHolder {
+ int count = 0;
+ }
+
+ /**
+ * 把一个row行转换成byte[][]数组
+ *
+ * @param row
+ * @return
+ */
+ protected byte[][] row2Byte(Map<String, Object> row, CountHolder countHolder) {
+
+ // byte[][] byteRows=new byte[row.size()][];
+ List<byte[]> list = new ArrayList<>();
+ Iterator<Map.Entry<String, Object>> it = row.entrySet().iterator();
+ while (it.hasNext()) {
+ Map.Entry<String, Object> entry = it.next();
+ int index = getColumnIndex(entry.getKey());
+ byte[] byteValue = createBytes(entry.getKey(), entry.getValue());
+ if (byteValue == null) {
+ list.add(new byte[0]);
+ // byteRows[value]=new byte[0];
+ } else {
+ list.add(byteValue);
+ // byteRows[value]=byteValue;
+ }
+ if (countHolder != null) {
+ int length = 0;
+ if (byteValue != null) {
+ length = byteValue.length;
+ }
+ countHolder.count = countHolder.count + length;
+ }
+ }
+ byte[][] byteRows = new byte[list.size()][];
+ for (int i = 0; i < list.size(); i++) {
+ byte[] temp = list.get(i);
+ byteRows[i] = temp;
+ }
+ return byteRows;
+ }
+
+ /**
+ * 把序列化的字节数组转换成map
+ *
+ * @param bytes 一行的字节数组
+ * @return
+ */
+ public Map<String, Object> byte2Row(byte[][] bytes) {
+ Map<String, Object> row = new HashMap<>();
+ for (int i = 0; i < bytes.length; i++) {
+ byte[] columnValue = bytes[i];
+ String columnName = index2ColumnName.get(i);
+ DataType dataType = cloumnName2DatatType.get(columnName);
+ Object object = null;
+ if (dataType != null) {
+ object = dataType.byteToValue(columnValue);
+ }
+ row.put(columnName, object);
+ }
+ return row;
+ }
+
+ /**
+ * 把数据转换成字节
+ *
+ * @param key
+ * @param value
+ * @return
+ */
+ private byte[] createBytes(String key, Object value) {
+ if (value == null) {
+ return null;
+ }
+ Object tmp = value;
+ DataType dataType = cloumnName2DatatType.get(key);
+ if (dataType == null) {
+ dataType = DataTypeUtil.getDataTypeFromClass(value.getClass());
+ if (dataType == null || dataType.getClass().getName().equals(NotSupportDataType.class.getName())) {
+ dataType = new StringDataType();
+ tmp = value.toString();
+ }
+ cloumnName2DatatType.put(key, dataType);
+ }
+ Object object = dataType.convert(tmp);
+ return dataType.toBytes(object, true);
+ }
+
+ /**
+ * 给每个列一个index,方便把数据放入数组中
+ *
+ * @param key
+ * @return
+ */
+ public int getColumnIndex(String key) {
+ Integer columnIndex = cloumnName2Index.get(key);
+ if (columnIndex == null) {
+ columnIndex = index.incrementAndGet() - 1;
+ cloumnName2Index.put(key, columnIndex);
+ index2ColumnName.put(columnIndex, key);
+ }
+ return columnIndex;
+ }
+
+ public static void main(String[] args) {
+ Map<String, Object> row = new HashMap<>();
+ row.put("name", "chris");
+ row.put("age", 18);
+ row.put("address", "huilongguafdffdsfdfdsn");
+ row.put("country", "china");
+ row.put("abc", "fdfd");
+ row.put("dxxx", "fdfd");
+ row.put("date", new Date());
+ row.put("ip", "192.168.0.1");
+ CompressTable tableCompress = new CompressTable();
+ byte[][] value = tableCompress.row2Byte(row);
+ int count = 0;
+ for (int i = 0; i < value.length; i++) {
+ byte[] tmp = value[i];
+ count += tmp.length;
+ }
+ System.out.println(count);
+ Map<String, Object> values = tableCompress.byte2Row(value);
+ Iterator<Map.Entry<String, Object>> it = values.entrySet().iterator();
+ while (it.hasNext()) {
+ Map.Entry<String, Object> entry = it.next();
+ System.out.println(entry.getKey() + ":" + entry.getValue());
+ }
+ }
+
+ public Map<String, Integer> getCloumnName2Index() {
+ return cloumnName2Index;
+ }
+
+ public Map<Integer, String> getIndex2ColumnName() {
+ return index2ColumnName;
+ }
+
+ public Map<String, DataType> getCloumnName2DatatType() {
+ return cloumnName2DatatType;
+ }
+
+ public void setCloumnName2Index(Map<String, Integer> cloumnName2Index) {
+ this.cloumnName2Index = cloumnName2Index;
+ }
+
+ public void setIndex2ColumnName(Map<Integer, String> index2ColumnName) {
+ this.index2ColumnName = index2ColumnName;
+ }
+
+ public void setCloumnName2DatatType(Map<String, DataType> cloumnName2DatatType) {
+ this.cloumnName2DatatType = cloumnName2DatatType;
+ }
+
+ public int getByteCount() {
+ return byteCount.get();
+ }
+
+ public int getRowCount() {
+ return rows.size();
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/AdditionStore.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/AdditionStore.java
new file mode 100644
index 0000000..0b441bf
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/AdditionStore.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.cache.compress;
+
+import java.util.ArrayList;
+import java.util.List;
+
+
+public class AdditionStore {
+
+ /**
+ * 每个冲突域列表,每个列表的最大值
+ */
+ public static final int CONFLICT_UNIT_SIZE = 1024 * 1024;
+
+ /**
+ * 如果value是非int值,可以通过这个值存储。原来value部分存储地址
+ */
+ private List<byte[]> values = new ArrayList<>();
+
+ /**
+ * 当前冲突元素存放在list的哪个index中
+ */
+ private int conflictIndex = -1;
+
+ /**
+ * 当前冲突的元素在byte中的可用位置
+ */
+ private int conflictOffset = 0;
+
+ /**
+ * 如果元素是固定大小,则这个值表示元素字节个数;
+ */
+ private int elementSize = -1;
+
+ /**
+ * 值是否是变长的,如果是变长的,需要额外两个字段存储长度
+ */
+ private boolean isVarLen = true;
+
+ /**
+ * 每个存储单元的大小
+ */
+ protected int blockSize = CONFLICT_UNIT_SIZE;
+
+ public AdditionStore() {
+
+ }
+
+ public AdditionStore(int elementSize, int blockSize) {
+ this.elementSize = elementSize;
+ if (elementSize > 0) {
+ isVarLen = false;
+ }
+ if (blockSize > 0) {
+ this.blockSize = blockSize;
+ }
+ }
+
+ public AdditionStore(int elementSize) {
+ this(elementSize, CONFLICT_UNIT_SIZE);
+ }
+
+ /**
+ * 把一个字节数组插入到存储中,并返回当前地址
+ *
+ * @param value
+ */
+ public CacheKV.MapAddress add2Store(byte[] value) {
+ if (conflictIndex == -1 || values.size() <= conflictIndex) {
+ byte[] bytes = new byte[blockSize];
+ values.add(bytes);
+ conflictOffset = 0;
+ }
+ if (conflictIndex == -1) {
+ conflictIndex = 0;
+ }
+ int length = value.length;
+ if (isVarLen) {
+ length = length + 2;
+ }
+ if (conflictOffset + length > blockSize) {
+ byte[] bytes = new byte[blockSize];
+ values.add(bytes);
+ conflictOffset = 0;
+ conflictIndex++;
+ }
+ byte[] bytes = values.get(conflictIndex);
+
+ CacheKV.MapAddress address = new CacheKV.MapAddress(conflictIndex, conflictOffset);
+ if (isVarLen) {
+ int size = value.length;
+ bytes[conflictOffset] = (byte)(size & 0xff);
+ bytes[conflictOffset + 1] = (byte)(size >> 8 & 0xff);
+ conflictOffset = conflictOffset + 2;
+ }
+ for (int i = 0; i < value.length; i++) {
+ try {
+ bytes[i + conflictOffset] = value[i];
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+
+ }
+ conflictOffset += value.length;
+ return address;
+ }
+
+ /**
+ * 根据当前地址获取对应的byte值
+ *
+ * @param mapAddress
+ * @return
+ */
+ public ByteArray getValue(CacheKV.MapAddress mapAddress) {
+ byte[] bytes = values.get(mapAddress.conflictIndex);
+ if (bytes == null) {
+ return null;
+ }
+ if (isVarLen == false) {
+ return new ByteArray(bytes, mapAddress.offset, elementSize);
+ } else {
+ int len = new ByteArray(bytes, mapAddress.offset, 2).castInt(0, 2);
+ return new ByteArray(bytes, mapAddress.offset + 2, len);
+ }
+ }
+
+ public int getConflictIndex() {
+ return conflictIndex;
+ }
+
+ public int getConflictOffset() {
+ return conflictOffset;
+ }
+
+ public int getBlockSize() {
+ return blockSize;
+ }
+
+ public void setConflictIndex(int conflictIndex) {
+ this.conflictIndex = conflictIndex;
+ }
+
+ public void setConflictOffset(int conflictOffset) {
+ this.conflictOffset = conflictOffset;
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArray.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArray.java
new file mode 100644
index 0000000..969d299
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArray.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.cache.compress;
+
+import java.util.Arrays;
+
+public class ByteArray {
+
+ private byte[] bytes;
+ private int startIndex;
+ private int size;
+
+ public ByteArray(ByteArray byteArray, int startIndex, int size) {
+ this.bytes = byteArray.bytes;
+ this.startIndex = byteArray.startIndex + startIndex;
+ this.size = size;
+ }
+
+ public ByteArray(byte[] bytes) {
+ this.bytes = bytes;
+ this.startIndex = 0;
+ this.size = bytes.length;
+ }
+
+ public ByteArray(byte[] bytes, int startIndex, int size) {
+ this.bytes = bytes;
+ this.startIndex = startIndex;
+ this.size = size;
+ }
+
+ public ByteArray subByteArray(int offset, int size) {
+ return new ByteArray(bytes, startIndex + offset, size);
+ }
+
+ public byte[] getByteArray() {
+ return Arrays.copyOfRange(bytes, startIndex, startIndex + size);
+ }
+
+ public int castInt(int offset, int size) {
+ int index = startIndex + offset;
+ int res = 0;
+ for (int i = 0; i < size; i++) {
+ res += (bytes[i + index] & 0xff) << (i * 8);
+ }
+ return res;
+ }
+
+ public byte getByte(int offset) {
+ int index = startIndex + offset;
+ return bytes[index];
+ }
+
+ public int getSize() {
+ return size;
+ }
+
+ public void flush(CacheKV.MapAddress address) {
+ if (address == null) {
+ return;
+ }
+ if (address.isConflict == false) {
+ flush(address.offset);
+ } else {
+ flush(address.createBytes());
+ }
+ }
+
+ protected void flush(ByteArray bytes) {
+ for (int i = 0; i < bytes.size; i++) {
+ this.bytes[i + this.startIndex] = bytes.getByte(i);
+ }
+ }
+
+ protected void flush(byte[] bytes) {
+ for (int i = 0; i < bytes.length; i++) {
+ this.bytes[i + this.startIndex] = bytes[i];
+ }
+ }
+
+ protected void flush(int value) {
+ for (int i = 0; i < 4; i++) {
+ if (i == 0) {
+ this.bytes[i + this.startIndex] = (byte)(value & 0xff);
+ } else {
+ this.bytes[i + this.startIndex] = (byte)(value >> (i * 8) & 0xff);
+ }
+ }
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArrayValueKV.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArrayValueKV.java
new file mode 100644
index 0000000..0e3d789
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArrayValueKV.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.cache.compress;
+
+import org.apache.rocketmq.streams.common.utils.NumberUtils;
+
+/**
+ * 支持key是string,value是int的场景,支持size不大于10000000.只支持int,long,boolean,string类型 只能一次行load,不能进行更新
+ */
+public class ByteArrayValueKV extends CacheKV<byte[]> {
+
+ protected final static String CODE = "UTF-8";
+ protected AdditionStore values;
+ protected boolean isFixedLength = false;//只有是定长时,才允许更新
+
+ public ByteArrayValueKV(int capacity, boolean isFixedLength) {
+ super(capacity);
+ this.isFixedLength = isFixedLength;
+ if (isFixedLength) {
+ values = new AdditionStore(capacity);
+ }
+ }
+
+ /**
+ * 直接存取byte数组
+ *
+ * @param key
+ * @return
+ */
+ @Override
+ public byte[] get(String key) {
+ ByteArray value = super.getInner(key);
+ if (value == null) {
+ return null;
+ }
+ MapAddress mapAddress = new MapAddress(value);
+ ByteArray byteArray = values.getValue(mapAddress);
+ return byteArray.getByteArray();
+ }
+
+ /**
+ * 如果是定长的字节,则判断已经有的value字节数和当前字节数相同,否则不允许插入
+ *
+ * @param key
+ * @param value
+ */
+ @Override
+ public void put(String key, byte[] value) {
+ if (key == null || value == null) {
+ return;
+ }
+
+ if (isFixedLength) {
+ byte[] oriValue = get(key);
+ if (oriValue != null) {
+ if (oriValue.length != value.length) {
+ throw new RuntimeException("the string must length equals ,but not。 the key is " + key + ", the ori value is " + oriValue + ", the put value is " + value);
+ }
+ }
+ }
+ MapAddress address = null;
+
+ int index = values.getConflictIndex();
+ int offset = values.getConflictOffset();
+ address = values.add2Store(value);
+ byte[] bytes = address.createBytes();
+ boolean success = super.putInner(key, NumberUtils.toInt(bytes), isFixedLength);
+ if (!success) {//不支持更新,如果存在已经有的key,则不插入,并回退刚插入的数据
+ values.setConflictOffset(offset);
+ values.setConflictIndex(index);
+ throw new RuntimeException("can not update value, the key has exist");
+ }
+
+ }
+
+ @Override
+ public int calMemory() {
+ int value = super.calMemory() + (this.conflicts.getConflictIndex() + 1) * this.conflicts
+ .getBlockSize();
+ return value;
+ }
+
+ @Override
+ public boolean contains(String key) {
+ byte[] bytes = get(key);
+ if (bytes == null) {
+ return false;
+ }
+ return true;
+ }
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/CacheKV.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/CacheKV.java
new file mode 100644
index 0000000..08bb056
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/CacheKV.java
@@ -0,0 +1,371 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.cache.compress;
+
+import org.apache.rocketmq.streams.common.utils.AESUtil;
+import org.apache.rocketmq.streams.common.utils.NumberUtils;
+import org.apache.rocketmq.streams.common.utils.StringUtil;
+
+/**
+ * 支持key是string,value是int的场景,支持size不大于10000000.只支持int,long,boolean,string类型
+ */
+public abstract class CacheKV<T> implements ICacheKV<T> {
+ /**
+ * 如果存储int作为值,最大值不能超过这个值,如果超过,会改成冲突链的模式
+ */
+ protected static final int MAX_INT_VALUE = 2147483647;
+
+ /**
+ * 发生冲突的次数
+ */
+ protected int conflictCount;
+
+ protected int elementSize = 24;
+
+ /**
+ * 建议设置,如果不设置,默认容量为最大容量
+ */
+ protected int capacity = 8000000;
+
+ /**
+ * 元素个数
+ */
+ protected int size = 0;
+
+ /**
+ * 每一个元素分24个字节部分(下一个地址4个字节,key的md5值16个字节,int的值4个字节)
+ */
+ protected byte[] map;
+
+ /**
+ * 当发生hash冲突时,map值存储在这里
+ */
+ protected AdditionStore conflicts;
+
+ public CacheKV(int capacity, int elementSize) {
+ this.capacity = capacity;
+ this.elementSize = elementSize;
+ map = new byte[this.capacity * elementSize];
+ conflicts = new AdditionStore(elementSize);
+
+ }
+
+ public CacheKV(int capacity) {
+ this(capacity, 24);
+ }
+
+ @Override
+ public abstract T get(String key);
+
+ @Override
+ public abstract void put(String key, T value);
+
+ public T remove(String key) {
+ if (StringUtil.isEmpty(key)) { return null; }
+ MapElementContext context = queryMapElementByHashCode(key);
+ /**
+ * TODO:
+ * 1 清空MD5
+ * 2 前驱的后继=后继
+ * 3 当前链接到冲突域的最后一个节点后
+ */
+ return null;
+ }
+
+ public ByteArray getInner(String key) {
+ if (key == null) {
+ return null;
+ }
+ MapElementContext context = queryMapElementByHashCode(key);
+ if (context.isMatchKey) {
+ return context.mapElement.value;
+ }
+ return null;
+ }
+
+ /**
+ * address:是否冲突(1位),conflict index(7位),3个字节地址域名
+ *
+ * @param key
+ * @param value
+ */
+ public boolean putInner(String key, int value, boolean supportUpdate) {
+ if (key == null) {
+ return false;
+ }
+ MapElementContext context = null;
+ //符合hash值的最后一个next=null的节点或者是和当前key匹配的节点
+ context = queryMapElementByHashCode(key);
+
+ KVElement mapElement = context.mapElement;
+ //如果没有发生冲突,说明当前节点无被占用,直接写入
+ if (context.isOccurConflict == false) {
+ size++;
+
+ mapElement.keyHashCode.flush(mapElement.getKeyHashCode());
+ if (mapElement.isNoValue() == false) {
+ mapElement.value.flush(value);
+ }
+
+ NumberUtils.putSubByte2ByteArray(map, context.mapAddress.offset, mapElement.getBytes());
+ } else {
+ //如果key已经存在,覆盖value
+ if (context.isMatchKey) {
+ if (mapElement.isNoValue() == false) {
+ if (!supportUpdate) {
+ return false;
+ }
+ mapElement.value.flush(value);
+ }
+ } else {//如果是新key,且有hash冲突,把新值写入冲突域,把冲突域地址更新next地址
+ size++;
+ conflictCount++;
+ byte[] bytes = KVElement.createByteArray(new MapAddress(), context.keyMd5, value, elementSize);
+ MapAddress mapAddress = conflicts.add2Store(bytes);
+ context.mapElement.nextAddressByte.flush(mapAddress);
+
+ }
+ }
+ return true;
+ }
+
+ /**
+ * 查找同hashcode的冲突链,如果key的值等于当前key值,直接返回。如果key值不同,返回冲突链最后一个元素
+ *
+ * @param key
+ * @return
+ */
+ public MapElementContext queryMapElementByHashCode(String key) {
+ int offset = getElementIndex(key);
+ byte[] hashCodes = AESUtil.stringToMD5(key);
+
+ MapAddress address = new MapAddress(offset);//先从map中查找
+
+ MapElementContext context = getMapElementByAddress(hashCodes, address);
+
+ if (context.mapElement.isEmpty()) {//如果map中无值,直接返回
+ context.keyMd5 = hashCodes;
+ context.mapElement.setKeyHashCode(hashCodes);
+ return context;
+ } else {
+ while (true) {//通过链表找到最后一个节点
+ context.keyMd5 = hashCodes;
+ if (context.isMatchKey) {
+ return context;
+ } else {
+ // System.out.println("occure conflict");
+ context.mapAddress.isConflict = true;
+ if (context.mapElement.nextAddress.isEmpty()) {
+ return context;
+ } else {
+ context = getMapElementByAddress(hashCodes, context.mapElement.nextAddress);
+ }
+ }
+ }
+ }
+
+ }
+
+ @Override
+ public int calMemory() {//计算总共分配的内存
+ int totalMemory = this.capacity * elementSize + (this.conflicts.getConflictIndex() + 1) * this.conflicts
+ .getBlockSize();
+
+ return totalMemory / 1024 / 1024;
+ }
+
+ private boolean equalsByte(byte[] hashCodes, ByteArray keyHashCode) {//判读两个hash code是否相等
+ if (hashCodes == null || keyHashCode == null) {
+ return false;
+ }
+ if (hashCodes.length != keyHashCode.getSize()) {
+ return false;
+ }
+ for (int i = 0; i < hashCodes.length; i++) {
+ if (hashCodes[i] != keyHashCode.getByte(i)) {
+ return false;
+ }
+ }
+ return true;
+
+ }
+
+ @Override
+ public int getSize() {
+ return size;
+ }
+
+ /**
+ * 查找某个地址对应的map element
+ *
+ * @param hashCodes
+ * @param address
+ * @return
+ */
+ protected MapElementContext getMapElementByAddress(byte[] hashCodes, MapAddress address) {
+ if (address.isConflict) {
+ ByteArray valueBytes = this.conflicts.getValue(address);
+ KVElement mapElement = new KVElement(valueBytes);
+ mapElement.setElementSize(this.elementSize);
+ return new MapElementContext(mapElement, address, equalsByte(hashCodes, mapElement.keyHashCode));
+ } else {
+ KVElement mapElement = new KVElement(new ByteArray(map, address.offset, elementSize));
+ mapElement.setElementSize(this.elementSize);
+ return new MapElementContext(mapElement, address, equalsByte(hashCodes, mapElement.keyHashCode));
+ }
+ }
+
+ /**
+ * 获取map的索引,通过hashcode获取
+ *
+ * @param key
+ * @return
+ */
+ protected int getElementIndex(String key) {
+ if (key == null) {
+ return 0;
+ }
+ int hashCode;
+ int value = (hashCode = key.hashCode()) ^ (hashCode >>> 16);
+ value = String.valueOf(value).hashCode();
+ if (value < 0) {
+ value = -value;
+ }
+ int index = value % capacity;
+ return index * elementSize;
+ }
+
+ protected class MapElementContext {
+
+ /**
+ * 当前的元素
+ */
+ protected KVElement mapElement;
+
+ /**
+ * 元素所在的地址
+ */
+ protected MapAddress mapAddress;
+
+ /**
+ * 是否和key的hashcode 匹配
+ */
+ protected boolean isMatchKey = false;
+
+ /**
+ * 是否发生冲突
+ */
+ protected boolean isOccurConflict = false;
+
+ /**
+ * 本次查询key对应的hash code
+ */
+ protected byte[] keyMd5;
+
+ public MapElementContext(KVElement mapElement, MapAddress mapAddress, boolean isMatchKey) {
+ this.mapAddress = mapAddress;
+ this.mapElement = mapElement;
+ this.isMatchKey = isMatchKey;
+ if (mapElement.isEmpty() == false) {
+ isOccurConflict = true;
+ }
+ }
+
+ public boolean isMatchKey() {
+ return isMatchKey;
+ }
+ }
+
+ /**
+ * 地址表示,如果放在list<byte[]>中,isConflict=true,7个bit位构成list的index,后面3个字节代表具体byte[]的位置 如果直接放到byte[]中,isConflict=false,后面7个bit+3个字节构成byte数组的offset
+ */
+ protected static class MapAddress {
+
+ /**
+ * 是否在冲突域
+ */
+ protected boolean isConflict = false;
+
+ /**
+ * 冲突域的list index
+ */
+ protected int conflictIndex = 0;
+
+ /**
+ * list byte【】对应的offset
+ */
+ protected int offset = 0;
+
+ public MapAddress(int conflictIndex, int offset) {
+ this.isConflict = true;
+ this.conflictIndex = conflictIndex;
+ this.offset = offset;
+ }
+
+ public MapAddress(int offset) {
+ this.isConflict = false;
+ this.conflictIndex = -1;
+ this.offset = offset;
+ }
+
+ public MapAddress() {
+
+ }
+
+ public boolean isEmpty() {
+ return isConflict == false && conflictIndex == 0 && offset == 0;
+ }
+
+ /**
+ * 高位是0则全部四位代表在map中的偏移量,高位是1则最高位代表冲突域的索引,第三位代表在冲突域链表中的地址
+ *
+ * @param byteArray
+ */
+ public MapAddress(ByteArray byteArray) {
+ byte firstByte = 0;
+ firstByte = byteArray.getByte(byteArray.getSize() - 1);
+ // byte firstByte=byteArray.getByte(byteArray.getSize()-1);
+ int conflictValue = NumberUtils.toInt(firstByte);
+ int conflictFlag = conflictValue >> 7;
+ if (conflictFlag == 1) {
+ isConflict = true;
+ } else {
+ isConflict = false;
+ this.offset = byteArray.castInt(0, 4);
+ return;
+ }
+ //TODO 这个地址是不是每次相同?
+ this.conflictIndex = conflictValue & 127;
+ this.offset = byteArray.castInt(0, 3);
+ }
+
+ public byte[] createBytes() {
+ byte[] bytes = NumberUtils.toByte(offset);
+ int value = 0;
+ byte fisrtByte = (byte)(conflictIndex & 0xff);
+ if (isConflict) {
+ value = (fisrtByte | (1 << 7));//把第一位变成1
+ } else {
+ return bytes;
+ }
+
+ bytes[bytes.length - 1] = (byte)(value & 0xff);
+ return bytes;
+ }
+ }
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ICacheKV.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ICacheKV.java
new file mode 100644
index 0000000..9a766e0
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ICacheKV.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.cache.compress;
+
+/**
+ * kv提供的对外接口,通过二进制实现存储,减少java对象带来的头部开销。 需要指定初始容量,会在创建对象时分配内存。
+ *
+ */
+public interface ICacheKV<T> {
+
+ /**
+ * 根据key获取value
+ *
+ * @param key
+ * @return
+ */
+ T get(String key);
+
+ /**
+ * 把key,value插入到kv中
+ *
+ * @param key
+ * @param value
+ */
+ void put(String key, T value);
+
+ /**
+ * 是否包含
+ *
+ * @param key
+ * @return
+ */
+ boolean contains(String key);
+
+ /**
+ * 一共有多少行
+ *
+ * @return
+ */
+ int getSize();
+
+ /**
+ * 占用多少内存
+ *
+ * @return
+ */
+ int calMemory();
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVElement.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVElement.java
new file mode 100644
index 0000000..60f2f0b
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVElement.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.cache.compress;
+
+import org.apache.rocketmq.streams.common.utils.NumberUtils;
+
+public class KVElement {
+
+ protected ByteArray nextAddressByte;
+
+ protected CacheKV.MapAddress nextAddress;
+
+ protected ByteArray keyHashCode;
+
+ /**
+ * value的值
+ */
+ protected ByteArray value;
+
+ /**
+ * 是否没有value
+ */
+ protected boolean noValue = false;
+
+ /**
+ * 元素个数
+ */
+ protected int elementSize = 24;
+
+ public KVElement(ByteArray byteArray) {
+ this.nextAddressByte = byteArray.subByteArray(0, 4);
+ nextAddress = new CacheKV.MapAddress(this.nextAddressByte);
+ this.keyHashCode = byteArray.subByteArray(4, 16);
+ if (!noValue) {
+ value = byteArray.subByteArray(20, 4);
+ }
+ }
+
+ public static byte[] createByteArray(CacheKV.MapAddress nextAddress, byte[] keyHashCode, int value,
+ int elementSize) {
+ KVElement element = new KVElement(nextAddress, keyHashCode, value);
+ element.setElementSize(elementSize);
+ return element.getBytes();
+ }
+
+ private KVElement(CacheKV.MapAddress nextAddress, byte[] keyHashCode, int value) {
+ this.nextAddress = nextAddress;
+ this.keyHashCode = new ByteArray(keyHashCode, 0, keyHashCode.length);
+ if (!noValue) {
+ this.value = new ByteArray(NumberUtils.toByte(value));
+ }
+
+ }
+
+ public boolean isEmpty() {
+ boolean empytHashCode = true;
+ for (int i = 0; i < keyHashCode.getSize(); i++) {
+ if (keyHashCode.getByte(i) != 0) {
+ empytHashCode = false;
+ break;
+ }
+ }
+ return (empytHashCode && nextAddress.isEmpty()) ? true : false;
+ }
+
+ public ByteArray getKeyHashCode() {
+ return keyHashCode;
+ }
+
+ public void setKeyHashCode(ByteArray keyHashCode) {
+ this.keyHashCode = keyHashCode;
+ }
+
+ public void setKeyHashCode(byte[] keyHashCode) {
+ this.keyHashCode = new ByteArray(keyHashCode);
+ }
+
+ public byte[] getBytes() {
+ byte[] bytes = new byte[elementSize];
+ NumberUtils.putSubByte2ByteArray(bytes, 0, nextAddress.createBytes());
+ NumberUtils.putSubByte2ByteArray(bytes, 4, keyHashCode.getByteArray());
+ if (!noValue) {
+ NumberUtils.putSubByte2ByteArray(bytes, 20, value.getByteArray());
+ }
+
+ return bytes;
+ }
+
+ public void setValue(int value) {
+ this.value = new ByteArray(NumberUtils.toByte(value));
+ }
+
+ public boolean isNoValue() {
+ return noValue;
+ }
+
+ public int getElementSize() {
+ return elementSize;
+ }
+
+ public KVElement setElementSize(int elementSize) {
+ this.elementSize = elementSize;
+ if (this.elementSize == 20) {
+ this.noValue = true;
+ }
+ return this;
+ }
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/FixedLenRowCacheKV.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/FixedLenRowCacheKV.java
new file mode 100644
index 0000000..71e8062
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/FixedLenRowCacheKV.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.cache.compress.impl;
+
+import org.apache.rocketmq.streams.common.cache.compress.ByteArrayValueKV;
+import org.apache.rocketmq.streams.common.cache.compress.ICacheKV;
+import org.apache.rocketmq.streams.common.datatype.*;
+import org.apache.rocketmq.streams.common.utils.NumberUtils;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * 常用类型的存储,目前只支持,float,int,long,double,set<String>,date,byte,short,boolean 存入一行数据,且必须是定长的行
+ */
+public class FixedLenRowCacheKV {
+ protected static final int MAX_ELIMENT = 3000000;//最大的元素个数
+ protected DataType[] datatypes;
+ protected int byteLength = -1;
+ protected MultiValueKV<byte[]> cache;//每块存储固定,超过后,用多块存储
+
+ protected Map<Integer, MultiValueKV> keySetMap = new HashMap<>();//存对应的set值。每个KeySet有最大值,超过最大值需要多个对象存储
+
+ public FixedLenRowCacheKV(DataType... datatypes) {
+ this.datatypes = datatypes;
+ if (datatypes != null) {
+ int index = 0;
+ int byteLength = 0;
+ for (DataType datatype : datatypes) {
+ if (datatype instanceof SetDataType) {
+ keySetMap.put(index, new MultiValueKV(MAX_ELIMENT) {
+ @Override
+ protected ICacheKV create() {
+ return new KeySet(this.capacity);
+ }
+ });
+ byteLength++;
+ } else if (datatype instanceof FloatDataType || datatype instanceof DoubleDataType || datatype instanceof LongDataType || datatype instanceof DateDataType) {
+ byteLength = byteLength + 8;
+ } else if (datatype instanceof IntDataType) {
+ byteLength = byteLength + 4;
+ } else if (datatype instanceof ByteDataType || datatype instanceof BooleanDataType) {
+ byteLength++;
+ } else if (datatype instanceof ShortDataType) {
+ byteLength = byteLength + 2;
+ }
+ index++;
+ }
+ }
+ cache = new MultiValueKV<byte[]>(MAX_ELIMENT) {
+ @Override
+ protected ICacheKV<byte[]> create() {
+ return new ByteArrayValueKV(this.capacity, true);
+ }
+ };
+ }
+
+ /**
+ * 写入一行数据
+ *
+ * @param key
+ * @param values
+ */
+ public void put(String key, Object... values) {
+ byte[] value = createBytes(values);
+ cache.put(key, value);
+ }
+
+ /**
+ * 获取一行数据
+ *
+ * @param key
+ * @return
+ */
+ public Object[] get(String key) {
+ byte[] value = cache.get(key);
+ if (value == null) {
+ return null;
+ }
+ Object[] result = new Object[datatypes.length];
+ AtomicInteger offset = new AtomicInteger(0);
+ for (int i = 0; i < result.length; i++) {
+ DataType dataType = datatypes[i];
+ if (dataType instanceof SetDataType) {
+ MultiValueKV multiValueKV = keySetMap.get(i);
+ result[i] = multiValueKV;
+ } else {
+ result[i] = dataType.byteToValue(value, offset);
+ }
+
+ }
+ return result;
+ }
+
+ /**
+ * 把具体的值,转化成固定长度的byte数组
+ *
+ * @param values
+ * @return
+ */
+ public byte[] createBytes(Object... values) {
+ if (values == null || values.length == 0) {
+ return null;
+ }
+ byte[] bytes = new byte[byteLength];
+ int index = 0;
+ for (int i = 0; i < values.length; i++) {
+ DataType dataType = datatypes[i];
+ Object o = values[i];
+ byte[] byteValue = null;
+ if (dataType instanceof SetDataType) {
+ byteValue = new byte[] {(byte)i};
+ add2Set(i, (Set<String>)o);
+ } else {
+ byteValue = dataType.toBytes(o, false);
+ }
+ NumberUtils.putSubByte2ByteArray(bytes, index, byteValue);
+ index = index + byteValue.length;
+ }
+ return bytes;
+ }
+
+ /**
+ * 插入数据到set中
+ *
+ * @param i 第几个变量
+ * @param setValue
+ */
+ protected void add2Set(int i, Set<String> setValue) {
+ }
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/IntValueKV.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/IntValueKV.java
new file mode 100644
index 0000000..f2ce1f7
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/IntValueKV.java
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.cache.compress.impl;
+
+import org.apache.rocketmq.streams.common.cache.compress.AdditionStore;
+import org.apache.rocketmq.streams.common.cache.compress.ByteArray;
+import org.apache.rocketmq.streams.common.cache.compress.CacheKV;
+import org.apache.rocketmq.streams.common.utils.NumberUtils;
+import org.junit.Assert;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.InputStreamReader;
+import java.util.*;
+import java.util.Map.Entry;
+
+/**
+ * 支持key是string,value是int的场景,支持size不大于10000000.只支持int,long,boolean,string类型
+ *
+ */
+public class IntValueKV extends CacheKV<Integer> {
+
+ protected AdditionStore conflicts = new AdditionStore(4);
+
+ @Override
+ public Integer get(String key) {
+ ByteArray byteArray = super.getInner(key);
+ if (byteArray == null) {
+ return null;
+ }
+ int value = byteArray.castInt(0, 4);
+ if (value < MAX_INT_VALUE) {
+ return value;
+ }
+ MapAddress mapAddress = new MapAddress(byteArray);
+ ByteArray bytes = conflicts.getValue(mapAddress);
+ return bytes.castInt(0, 4);
+ }
+
+ @Override
+ public void put(String key, Integer value) {
+ if (value < MAX_INT_VALUE) {
+ super.putInner(key, value, true);
+ return;
+ }
+ MapAddress address = conflicts.add2Store(NumberUtils.toByte(value));
+ super.putInner(key, NumberUtils.toInt(address.createBytes()), true);
+
+ }
+
+ @Override
+ public boolean contains(String key) {
+ Integer value = get(key);
+ if (value == null) {
+ return false;
+ }
+ return true;
+
+ }
+
+ @Override
+ public int calMemory() {
+ int value = super.calMemory() + (this.conflicts.getConflictIndex() + 1) * this.conflicts
+ .getBlockSize();
+ return value;
+ }
+
+ /**
+ * TODO remove the key from the sinkcache and return the removed value
+ *
+ * @return
+ */
+ //public Integer remove(String key) {
+ // return null;
+ //}
+ public IntValueKV(int capacity) {
+ super(capacity);
+ }
+
+ public static void main(String[] args) throws Exception {
+ IntValueKV cache = new IntValueKV(5);
+ cache.put("A", 0);
+ cache.put("B", 1);
+ cache.put("C", 2);
+ cache.put("D", 3);
+ cache.put("E", 4);
+ cache.put("F", 5);
+ cache.put("G", 6);
+
+ System.exit(0);
+
+ int size = 10000000;
+ int sampleSize = 1024;
+ int dataSize = 3974534;
+ IntValueKV compressByteMap = new IntValueKV(size);
+ Map<String, Integer> dataMap = new HashMap<>(size);
+ Set<Integer> whiteSet = new HashSet<>(1024);
+ Map<String, Integer> sample1Map = new HashMap<>(1024);
+ Map<String, Integer> sample2Map = new HashMap<>(1024);
+ //init data
+ Random random = new Random();
+ while (true) {
+ if (whiteSet.size() >= sampleSize) {
+ break;
+ }
+ int seed = random.nextInt(dataSize);
+ if (!whiteSet.contains(seed)) {
+ whiteSet.add(seed);
+ }
+ }
+
+ long originWriteCounter = 0;
+ long compressWriteCounter = 0;
+ String path = "/Users/arthur/Downloads/";
+ String blackFile = "2020-11-11-14-08-32_EXPORT_CSV_16231630_392_0.csv";
+ try (BufferedReader reader = new BufferedReader(new InputStreamReader(new FileInputStream(path + blackFile)))) {
+ reader.readLine();
+ String line = null;
+ int counter = 0;
+ while ((line = reader.readLine()) != null) {
+ line = line.replaceAll("\"", "");
+ String[] parts = line.split(",", 2);
+ long begin = System.nanoTime();
+ dataMap.put(parts[1].trim(), Integer.parseInt(parts[0]));
+ originWriteCounter += (System.nanoTime() - begin);
+ if (whiteSet.contains(counter++)) {
+ sample1Map.put(parts[1].trim(), Integer.parseInt(parts[0]));
+ }
+ }
+ }
+ for (int i = 0; i < sampleSize * 100; i++) {
+ sample2Map.put(UUID.randomUUID().toString(), -1);
+ }
+ System.out.println("sample1 size:\t" + sample1Map.size());
+ System.out.println("sample2 size:\t" + sample2Map.size());
+ //System.out.println(
+ // "origin map size(computed by third party):\t" + RamUsageEstimator.humanSizeOf(dataMap) + "\tline's\t"
+ // + dataMap.size());
+ //
+ Iterator<Entry<String, Integer>> iterator = dataMap.entrySet().iterator();
+ while (iterator.hasNext()) {
+ Entry<String, Integer> entry = iterator.next();
+ long begin = System.nanoTime();
+ compressByteMap.put(entry.getKey(), entry.getValue());
+ compressWriteCounter += (System.nanoTime() - begin);
+ }
+ //System.out.println(
+ // "compressed map size(computed by third party):\t" + RamUsageEstimator.humanSizeOf(compressByteMap)
+ // + "\tline's\t"
+ // + compressByteMap.size);
+ System.out.println("compressed map size(computed by it's self)\t" + compressByteMap.calMemory() + " MB");
+ System.out.println(
+ "origin write cost:\t" + originWriteCounter / 1000 + "\tcompress write cost:\t"
+ + compressWriteCounter / 1000);
+ //
+ long originSearchCounter = 0;
+ long compressCounter = 0;
+ Iterator<Entry<String, Integer>> iterator1 = sample1Map.entrySet().iterator();
+ Iterator<Entry<String, Integer>> iterator2 = sample2Map.entrySet().iterator();
+ while (iterator1.hasNext() && iterator2.hasNext()) {
+ Entry<String, Integer> entry1 = iterator1.next();
+ String key1 = entry1.getKey();
+ Integer value1 = entry1.getValue();
+ Entry<String, Integer> entry2 = iterator2.next();
+ String key2 = entry2.getKey();
+ Integer value2 = entry2.getValue();
+ long begin = System.nanoTime();
+ Assert.assertEquals(value1, dataMap.get(key1));
+ Assert.assertNotEquals(value2, dataMap.get(key2));
+ originSearchCounter += (System.nanoTime() - begin);
+ begin = System.nanoTime();
+ Assert.assertEquals(value1, compressByteMap.get(key1));
+ Assert.assertNotEquals(value2, compressByteMap.get(key2));
+ compressCounter += (System.nanoTime() - begin);
+ }
+ System.out.println(
+ "origin search cost:\t" + originSearchCounter / 1000 + "\tcompress search cost:\t"
+ + compressCounter / 1000);
+ }
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/KeySet.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/KeySet.java
new file mode 100644
index 0000000..0e5f278
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/KeySet.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.cache.compress.impl;
+
+import org.apache.rocketmq.streams.common.cache.compress.CacheKV;
+import org.apache.rocketmq.streams.common.cache.compress.ICacheKV;
+
+/**
+ * 支持key是string,value是int的场景,支持size不大于10000000.只支持int,long,boolean,string类型
+ */
+public class KeySet implements ICacheKV {
+
+ protected CacheKV cacheKV;
+
+ public KeySet(int capacity) {
+ cacheKV = new CacheKV(capacity, 20) {
+ @Override
+ public Object get(String key) {
+ boolean exist = contains(key);
+ if (exist) {
+ return key;
+ }
+ return null;
+ }
+
+ @Override
+ public void put(String key, Object value) {
+ putInner(key, 1, true);
+ }
+
+ @Override
+ public boolean contains(String key) {
+ MapElementContext context = queryMapElementByHashCode(key);
+ if (context.isMatchKey()) {
+ return true;
+ }
+ return false;
+ }
+
+ };
+ }
+
+ public void add(String key) {
+ cacheKV.putInner(key, 1, true);
+ }
+
+ @Override
+ public Object get(String key) {
+ return cacheKV.get(key);
+ }
+
+ @Override
+ public void put(String key, Object value) {
+ cacheKV.put(key, value);
+ }
+
+ @Override
+ public boolean contains(String key) {
+ return cacheKV.contains(key);
+ }
+
+ @Override
+ public int getSize() {
+ return cacheKV.getSize();
+ }
+
+ @Override
+ public int calMemory() {
+ return cacheKV.calMemory();
+ }
+
+ public static void main(String[] args) throws InterruptedException {
+ int count = 10000000;
+ KeySet map = new KeySet(count);
+ long start = System.currentTimeMillis();
+ // Set<String> values=new HashSet<>();
+ //Map<String,Integer> kv=new HashMap<>();
+ for (int i = 0; i < count; i++) {
+ map.add("sdfsdfdds" + i);
+ ;
+ }
+ System.out.println("cost memory:\t" + map.calMemory() + "M");
+ // System.out.println(values.size());
+ System.out.println("cost is:\t" + (System.currentTimeMillis() - start));
+ start = System.currentTimeMillis();
+
+ for (int i = 0; i < count; i++) {
+ boolean v = map.contains("sdfsdfdds" + i);
+ if (!v) {
+ throw new RuntimeException("");
+ }
+ }
+ System.out.println("cost is:\t" + (System.currentTimeMillis() - start));
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/ListValueKV.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/ListValueKV.java
new file mode 100644
index 0000000..40b54bd
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/ListValueKV.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.cache.compress.impl;
+
+import org.apache.rocketmq.streams.common.cache.compress.CacheKV;
+import org.apache.rocketmq.streams.common.datatype.DataType;
+import org.apache.rocketmq.streams.common.datatype.ListDataType;
+import org.apache.rocketmq.streams.common.utils.DataTypeUtil;
+
+import java.util.List;
+
+/**
+ * 支持key是string,value是int的场景,支持size不大于10000000.只支持int,long,boolean,string类型
+ */
+public class ListValueKV<T> extends CacheKV<List<T>> {
+
+ protected StringValueKV stringValueKV;
+ protected DataType<T> datatype;
+ protected ListDataType listDataType;
+ protected transient Class elementClass;
+
+ public ListValueKV(int capacity, Class elementClass) {
+ super(0);
+ stringValueKV = new StringValueKV(capacity, false);
+ datatype = DataTypeUtil.getDataTypeFromClass(elementClass);
+ listDataType = new ListDataType(datatype);
+ this.elementClass = elementClass;
+ }
+
+ @Override
+ public List<T> get(String key) {
+ String value = stringValueKV.get(key);
+ if (value == null) {
+ return null;
+ }
+
+ return listDataType.getData(value);
+ }
+
+ @Override
+ public void put(String key, List<T> values) {
+ String value = listDataType.toDataJson(values);
+ ;
+ stringValueKV.put(key, value);
+ }
+
+ @Override
+ public boolean contains(String key) {
+ return stringValueKV.contains(key);
+ }
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MultiValueKV.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MultiValueKV.java
new file mode 100644
index 0000000..7881733
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MultiValueKV.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.cache.compress.impl;
+
+import org.apache.rocketmq.streams.common.cache.compress.ICacheKV;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public abstract class MultiValueKV<T> implements ICacheKV<T> {
+ //按固定大小分割存储
+ protected List<ICacheKV<T>> valueKVS = new ArrayList<>();
+ //当前存储的索引
+ protected int currentIndex = 0;
+ //每个分片的大小
+ protected int capacity;
+
+ public MultiValueKV(int capacity) {
+ this.capacity = capacity;
+ }
+
+ @Override
+ public T get(String key) {
+ if (valueKVS == null) {
+ return null;
+ }
+ for (ICacheKV<T> cacheKV : valueKVS) {
+ if (cacheKV != null) {
+ T value = cacheKV.get(key);
+ if (value != null) {
+ return value;
+ }
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public void put(String key, T value) {
+ if (valueKVS == null) {
+ return;
+ }
+ ICacheKV<T> cacheKV = valueKVS.get(currentIndex);
+ if (cacheKV.getSize() >= capacity) {
+ synchronized (this) {
+ cacheKV = valueKVS.get(currentIndex);
+ if (cacheKV.getSize() >= capacity) {
+ cacheKV = create();
+ valueKVS.add(cacheKV);
+ currentIndex++;
+ }
+ }
+ }
+ cacheKV.put(key, value);
+ }
+
+ @Override
+ public boolean contains(String key) {
+ if (valueKVS == null) {
+ return false;
+ }
+ for (ICacheKV<T> cacheKV : valueKVS) {
+ if (cacheKV != null) {
+ boolean isMatch = cacheKV.contains(key);
+ if (isMatch) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ protected abstract ICacheKV<T> create();
+
+ @Override
+ public int getSize() {
+ return 0;
+ }
+
+ @Override
+ public int calMemory() {
+ return 0;
+ }
+
+ public int getCurrentIndex() {
+ return currentIndex;
+ }
+
+ public void setCurrentIndex(int currentIndex) {
+ this.currentIndex = currentIndex;
+ }
+
+ public int getCapacity() {
+ return capacity;
+ }
+
+ public void setCapacity(int capacity) {
+ this.capacity = capacity;
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MutilValueKV.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MutilValueKV.java
new file mode 100644
index 0000000..3065a46
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MutilValueKV.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.streams.common.cache.compress.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.rocketmq.streams.common.cache.compress.ICacheKV;
+
+public abstract class MutilValueKV<T> implements ICacheKV<T> {
+ //按固定大小分割存储
+ protected List<ICacheKV<T>> valueKVS=new ArrayList<>();
+ //当前存储的索引
+ protected int currentIndex=0;
+ //每个分片的大小
+ protected int capacity;
+
+ public MutilValueKV(int capacity){
+ this.capacity=capacity;
+ }
+
+
+ @Override
+ public T get(String key) {
+ if(valueKVS==null){
+ return null;
+ }
+ for(ICacheKV<T> cacheKV:valueKVS){
+ if(cacheKV!=null){
+ T value=cacheKV.get(key);
+ if(value!=null){
+ return value;
+ }
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public void put(String key, T value) {
+ if(valueKVS==null){
+ return;
+ }
+ ICacheKV<T> cacheKV= valueKVS.get(currentIndex);
+ if(cacheKV.getSize()>=capacity){
+ synchronized (this){
+ cacheKV= valueKVS.get(currentIndex);
+ if(cacheKV.getSize()>=capacity){
+ cacheKV=create();
+ valueKVS.add(cacheKV);
+ currentIndex++;
+ }
+ }
+ }
+ cacheKV.put(key,value);
+ }
+
+ @Override
+ public boolean contains(String key) {
+ if(valueKVS==null){
+ return false;
+ }
+ for(ICacheKV<T> cacheKV:valueKVS){
+ if(cacheKV!=null){
+ boolean isMatch=cacheKV.contains(key);
+ if(isMatch){
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+
+ protected abstract ICacheKV<T> create();
+
+ @Override
+ public int getSize() {
+ return 0;
+ }
+
+ @Override
+ public int calMemory() {
+ return 0;
+ }
+
+ public int getCurrentIndex() {
+ return currentIndex;
+ }
+
+ public void setCurrentIndex(int currentIndex) {
+ this.currentIndex = currentIndex;
+ }
+
+ public int getCapacity() {
+ return capacity;
+ }
+
+ public void setCapacity(int capacity) {
+ this.capacity = capacity;
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/SplitCache.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/SplitCache.java
new file mode 100644
index 0000000..59aa08a
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/SplitCache.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.cache.compress.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * sinkcache(key:String,value:String) for special scene which read operation more than write operation, it's not thread safe
+ */
+public class SplitCache<T> {
+
+ private int capacity;
+
+ /**
+ * TODO: try huffman code?
+ */
+ private List<byte[]> values;
+
+ private IntValueKV keys;
+
+ private SplitCache() {
+
+ }
+
+ public SplitCache(int capacity) {
+ this.capacity = capacity;
+ values = new ArrayList<>(capacity);
+ keys = new IntValueKV(capacity);
+ }
+
+ public void put(String key, String value) {
+ int index = Optional.ofNullable(keys.get(key)).orElse(-1);
+ if (index > -1 && index < capacity) {
+ values.set(index, value.getBytes());
+ } else {
+ values.add(value.getBytes());
+ keys.put(key, values.size() - 1);
+ }
+ }
+
+ public String get(String key) {
+ int index = Optional.ofNullable(keys.get(key)).orElse(-1);
+ return (index > -1 && index < capacity) ? new String(values.get(index)) : null;
+ }
+
+ public String remove(String key) {
+ int index = Optional.ofNullable(keys.get(key)).orElse(-1);
+ String value = null;
+ if (index > -1 && index < capacity) {
+ value = new String(values.get(index));
+ //TODO remove from keys
+ values.remove(index);
+ }
+ return value;
+ }
+
+ public int getSize() {
+ return values.size();
+ }
+
+ public void clear() {
+ values = new ArrayList<>(capacity);
+ keys = new IntValueKV(capacity);
+ }
+
+ public int getCapacity() {
+ return capacity;
+ }
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/StringValueKV.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/StringValueKV.java
new file mode 100644
index 0000000..341cfbe
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/StringValueKV.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.cache.compress.impl;
+
+import org.apache.rocketmq.streams.common.cache.compress.ByteArrayValueKV;
+import org.apache.rocketmq.streams.common.cache.compress.ICacheKV;
+import org.junit.Assert;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * 支持key是string,value是int的场景,支持size不大于10000000.只支持int,long,boolean,string类型 只能一次行load,不能进行更新
+ */
+public class StringValueKV implements ICacheKV<String> {
+
+ protected final static String CODE = "UTF-8";
+ protected ByteArrayValueKV values;
+
+ public StringValueKV(int capacity, boolean isFixedLength) {
+ values = new ByteArrayValueKV(capacity, isFixedLength);
+ }
+
+ @Override
+ public String get(String key) {
+ byte[] bytes = values.get(key);
+ if (bytes == null) {
+ return null;
+ }
+ try {
+ return new String(bytes, CODE);
+ } catch (Exception e) {
+ throw new RuntimeException("can not convert byte 2 string ", e);
+ }
+ }
+
+ @Override
+ public void put(String key, String value) {
+
+ try {
+ byte[] bytes = value.getBytes(CODE);
+ values.put(key, bytes);
+ } catch (UnsupportedEncodingException e) {
+ throw new RuntimeException("can not convert byte 2 string ", e);
+ }
+
+ }
+
+ @Override
+ public boolean contains(String key) {
+ return values.contains(key);
+ }
+
+ @Override
+ public int getSize() {
+ return values.getSize();
+ }
+
+ @Override
+ public int calMemory() {
+ return values.calMemory();
+ }
+
+ public static void main(String[] args) throws InterruptedException {
+ int count = 10000000;
+
+ StringValueKV map = new StringValueKV(count, false);
+ long start = System.currentTimeMillis();
+ for (int i = 0; i < count; i++) {
+ map.put("sdfsdfdds" + i, i + "");
+ }
+ System.out.println("fixed value size: " + map.getSize());
+ //System.out.println("fixed value memory: " + RamUsageEstimator.humanSizeOf(map));
+ System.out.println("fixed value write cost: " + (System.currentTimeMillis() - start));
+
+ start = System.currentTimeMillis();
+ SplitCache splitCache = new SplitCache(count);
+ for (int i = 0; i < count; i++) {
+ splitCache.put("sdfsdfdds" + i, i + "");
+ }
+ System.out.println("free value size: " + splitCache.getSize());
+ // System.out.println("free value memory: " + RamUsageEstimator.humanSizeOf(splitCache));
+ System.out.println("free value cost: " + (System.currentTimeMillis() - start));
+
+ start = System.currentTimeMillis();
+ Map<String, String> originMap = new HashMap<>(count);
+ for (int i = 0; i < count; i++) {
+ originMap.put("sdfsdfdds" + i, i + "");
+ }
+ System.out.println("origin map size: " + originMap.size());
+ // System.out.println("origin map memory: " + RamUsageEstimator.humanSizeOf(originMap));
+ System.out.println("origin map cost: " + (System.currentTimeMillis() - start));
+
+ start = System.currentTimeMillis();
+ for (int i = 0; i < count; i++) {
+ String v = map.get("sdfsdfdds" + i);
+ Assert.assertEquals(v, i + "");
+ v = map.get("asdfasdf" + i);
+ Assert.assertNull(v);
+ }
+ System.out.println("fix value read cost: " + (System.currentTimeMillis() - start));
+
+ start = System.currentTimeMillis();
+ for (int i = 0; i < count; i++) {
+ String v = splitCache.get("sdfsdfdds" + i);
+ Assert.assertEquals(v, i + "");
+ v = splitCache.get("asdfasdf" + i);
+ Assert.assertNull(v);
+ }
+ System.out.println("free value read cost: " + (System.currentTimeMillis() - start));
+
+ start = System.currentTimeMillis();
+ for (int i = 0; i < count; i++) {
+ String v = originMap.get("sdfsdfdds" + i);
+ Assert.assertEquals(v, i + "");
+ v = originMap.get("asdfasdf" + i);
+ Assert.assertNull(v);
+ }
+ System.out.println("origin map read cost: " + (System.currentTimeMillis() - start));
+ }
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/softreference/ICache.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/softreference/ICache.java
new file mode 100644
index 0000000..7f97a34
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/softreference/ICache.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.cache.softreference;
+
+import java.util.List;
+
+/**
+ * 基于软引用实现的缓存,可以在内存不足时回收,尽量最大限度使用内存的场景使用
+ */
+public interface ICache<K, V> {
+
+ /**
+ * get driver
+ *
+ * @param k
+ * @return
+ */
+ V get(K k);
+
+ /**
+ * put driver
+ *
+ * @param k
+ * @param v
+ */
+ void put(K k, V v);
+
+ /**
+ * remove driver which return the removed value
+ *
+ * @param k
+ * @return
+ */
+ V remove(K k);
+
+ /**
+ * remove all keys which has same prefix and return the removed value in the list
+ *
+ * @param k
+ * @return
+ */
+ List<V> removeByPrefix(K k);
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/softreference/RebuildCacheElement.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/softreference/RebuildCacheElement.java
new file mode 100644
index 0000000..5f923dd
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/softreference/RebuildCacheElement.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.cache.softreference;
+
+/**
+ * 缓存失效后,重新创建对象
+ */
+public interface RebuildCacheElement<K, V> {
+
+ V create(K k);
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/softreference/impl/SoftReferenceCache.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/softreference/impl/SoftReferenceCache.java
new file mode 100644
index 0000000..6a930fd
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/softreference/impl/SoftReferenceCache.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.cache.softreference.impl;
+
+import org.apache.rocketmq.streams.common.cache.softreference.ICache;
+import org.apache.rocketmq.streams.common.cache.softreference.RebuildCacheElement;
+
+import java.lang.ref.ReferenceQueue;
+import java.lang.ref.SoftReference;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * 基于软引用实现的缓存,可以在内存不足时回收,尽量最大限度使用内存的场景使用
+ */
+public class SoftReferenceCache<K, V> implements ICache<K, V> {
+
+ protected RebuildCacheElement<K, V> rebuildCacheElement;
+
+ public SoftReferenceCache(RebuildCacheElement<K, V> rebuildCacheElement) {
+ this.rebuildCacheElement = rebuildCacheElement;
+ }
+
+ public SoftReferenceCache() {
+ this.rebuildCacheElement = new RebuildCacheElement<K, V>() {
+ @Override
+ public V create(K k) {
+ return null;
+ }
+ };
+ }
+
+ protected class ExtraInfoReference<T> extends SoftReference<T> {
+
+ private Object info;
+
+ public ExtraInfoReference(Object info, T t, ReferenceQueue<T> refQueue) {
+ super(t, refQueue);
+ this.info = info;
+ }
+
+ public Object getExtraInfo() {
+ return this.info;
+ }
+ }
+
+ // 缓存,用软引用记录
+ private ConcurrentHashMap<K, ExtraInfoReference<V>> cache = new ConcurrentHashMap<K, ExtraInfoReference<V>>();
+ private ReferenceQueue<V> refQueue = new ReferenceQueue<V>();
+
+ @Override
+ public V get(K key) {
+ V value = null;
+
+ if (cache.containsKey(key)) {
+ SoftReference<V> refValue = cache.get(key);
+ if (refValue != null) {
+ value = refValue.get();
+ }
+
+ }
+ // 如果软引用被回收
+ if (value == null) {
+ // 清除软引用队列
+ clearRefQueue();
+ // 创建值并放入缓存
+ value = rebuildCacheElement.create(key);
+ if (value != null) {
+ ExtraInfoReference<V> refValue = new ExtraInfoReference<V>(key, value, refQueue);
+ cache.put(key, refValue);
+ }
+ }
+
+ return value;
+ }
+
+ @Override
+ public void put(K k, V v) {
+ if (v == null) {
+ cache.remove(k);
+ }
+ ExtraInfoReference<V> refValue = new ExtraInfoReference<V>(k, v, refQueue);
+ cache.put(k, refValue);
+ }
+
+ @Override
+ public V remove(K k) {
+ return null;
+ }
+
+ @Override
+ public List<V> removeByPrefix(K k) {
+ return null;
+ }
+
+ public void clear() {
+ clearRefQueue();
+ cache.clear();
+ }
+
+ /**
+ * 从软引用队列中移除无效引用, 同时从cache中删除无效缓存
+ */
+ @SuppressWarnings("unchecked")
+ protected void clearRefQueue() {
+ ExtraInfoReference<V> refValue = null;
+ while ((refValue = (ExtraInfoReference<V>)refQueue.poll()) != null) {
+ K key = (K)refValue.getExtraInfo();
+ cache.remove(key);
+ }
+ }
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/calssscaner/AbstractScan.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/calssscaner/AbstractScan.java
new file mode 100644
index 0000000..5b9f58e
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/calssscaner/AbstractScan.java
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.calssscaner;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.rocketmq.streams.common.utils.FileUtil;
+import org.apache.rocketmq.streams.common.utils.StringUtil;
+
+import java.io.*;
+import java.net.URL;
+import java.util.*;
+import java.util.jar.JarEntry;
+import java.util.jar.JarFile;
+
+public abstract class AbstractScan {
+
+ private static final Log LOG = LogFactory.getLog(AbstractScan.class);
+
+ private static final String CLASS_REAR = ".class";
+
+ protected Set<String> scanDirs = new HashSet<>();
+
+ public void scanClassDir(File jarFile, String packageName, ClassLoader classLoader) {
+ scanClassInJar(jarFile.getAbsolutePath(), packageName, classLoader);
+ }
+
+ public void scanClassDir(String dir, String packageName, ClassLoader classLoader) {
+ if (StringUtil.isEmpty(dir)) {
+ return;
+ }
+ if (classLoader == null) {
+ classLoader = this.getClass().getClassLoader();
+ }
+ if (!dir.endsWith(File.separator)) {
+ dir += File.separator;
+ }
+ if (packageName.contains(".")) {
+ dir += packageName.replace(".", File.separator);
+ }
+ File dirs = new File(dir);
+ if (!dirs.exists()) {
+ return;
+ }
+
+ File[] files = dirs.listFiles();
+ if (files.length == 0) {
+ return;
+ }
+ for (File file : files) {
+ try {
+ String className = file.getName();
+ if (className.endsWith(CLASS_REAR)) {
+ Class clazz = classLoader.loadClass(packageName + "." + className.replace(CLASS_REAR, ""));
+ doProcessor(clazz);
+ }
+ } catch (ClassNotFoundException e) {
+ LOG.error("load class error " + file.getName(), e);
+ continue;
+ }
+ }
+ }
+
+ protected void scanDir(String dir) {
+ if (inJar(dir)) {
+ scanClassInJar(dir);
+ } else {
+ scanClassInDir(dir);
+ }
+ }
+
+ public void scanPackages(String... packageNames) {
+ if (packageNames != null) {
+ for (String packageName : packageNames) {
+ scanPackage(packageName);
+ }
+ }
+ }
+
+ public void scanPackage(String packageName) {
+ if (scanDirs.contains(packageName)) {
+ return;
+ }
+ scanDirs.add(packageName);
+ String packageDir = "/" + packageName.replace(".", "/");
+ Set<String> hasScan = new HashSet<>();
+ if (inJar(packageDir)) {
+ scanClassInJar(packageDir);
+ } else {
+ List<String> dirs = scanPackageDir(packageDir);
+ if (dirs == null) {
+ return;
+ }
+
+ for (String dir : dirs) {
+ if (hasScan.contains(dir)) {
+ continue;
+ } else {
+ scanDir(dir);
+ hasScan.add(dir);
+ }
+
+ }
+ }
+
+ }
+
+ protected List<String> scanPackageDir(String packageDir) {
+ try {
+
+ URL url = this.getClass().getResource(packageDir);
+ if (url == null) {
+ return null;
+ }
+ BufferedReader br = new BufferedReader(new InputStreamReader(url.openStream()));
+ String line = br.readLine();
+ boolean hasScane = false;
+ List<String> dirs = new ArrayList<>();
+ while (line != null) {
+ if (!line.endsWith(CLASS_REAR)) {
+ String dir = packageDir + "/" + line;
+ File fileDir = new File(url.getPath() + "/" + line);
+ if (fileDir.isDirectory()) {
+ List<String> result = scanPackageDir(dir);
+ if (result != null) {
+ dirs.addAll(result);
+ }
+ }
+ } else {
+ if (!hasScane) {
+ dirs.add(packageDir);
+ hasScane = true;
+ }
+ }
+ line = br.readLine();
+ }
+ return dirs;
+ } catch (Exception e) {
+ LOG.error("包扫描异常:", e);
+ // throw new RuntimeException("scan error "+packageDir,e);
+ }
+ return null;
+
+ }
+
+ protected boolean inJar(String dir) {
+ return FileUtil.inJar(dir, this.getClass());
+ }
+
+ protected void scanClassInJar(String dirName) {
+ URL url = null;
+ try {
+ url = this.getClass().getResource(dirName);
+ if (url == null) {
+ return;
+ }
+ } catch (Exception e) {
+ LOG.error("ScanFunctionService scanClassInJar error", e);
+ }
+
+ // jar:file:/Users/yuanxiaodong/alibaba/rule-engine-feature/5/rules-engine/engine/target/ruleengine
+ // .jar!/com/aliyun/filter/function/expression
+
+ String jarUrl = url.toString().replace("jar:file:", "");
+ int index = jarUrl.indexOf("!/");
+ String packageName = createPackageName(dirName);
+ jarUrl = jarUrl.substring(0, index);
+ scanClassInJar(jarUrl, packageName, this.getClass().getClassLoader());
+
+ }
+
+ protected void scanClassInJar(String jarPath, String packageName, ClassLoader classLoader) {
+ try {
+ if (classLoader == null) {
+ classLoader = this.getClass().getClassLoader();
+ }
+ JarFile jarFile = new JarFile(jarPath);
+ Enumeration<JarEntry> entries = jarFile.entries();
+
+ while (entries.hasMoreElements()) {
+ String className = entries.nextElement().getName().replace("/", ".");
+ if (className.startsWith(packageName) && className.endsWith(".class")) {
+ className = className.replace(CLASS_REAR, "");
+ doRegisterFunction(className, classLoader);
+ }
+
+ }
+ } catch (Exception e) {
+ LOG.error("ScanFunctionService scanClassInJar JarFile error", e);
+ }
+ }
+
+ protected void scanClassInDir(String dirName) {
+
+ InputStream in = this.getClass().getResourceAsStream(dirName);
+ if (in == null) {
+ return;
+ }
+ BufferedReader br = new BufferedReader(new InputStreamReader(in));
+ String packageName = createPackageName(dirName);
+ try {
+ String line = br.readLine();
+ while (line != null) {
+ if (!line.endsWith(CLASS_REAR)) {
+ line = br.readLine();
+ continue;
+ }
+ String className = line.replace(CLASS_REAR, "");
+ doRegisterFunction(packageName + className);
+ line = br.readLine();
+ }
+ } catch (IOException e) {
+ LOG.error("注册类错误" + e.getMessage(), e);
+ }
+
+ }
+
+ private String createPackageName(String dirName) {
+ if (dirName.startsWith("/")) {
+ String packageName = dirName.substring(1).replace("/", ".") + ".";
+ return packageName;
+ } else {
+ return this.getClass().getPackage().getName() + "." + dirName + ".";
+ }
+ }
+
+ protected void doRegisterFunction(String className) {
+ doRegisterFunction(className, this.getClass().getClassLoader());
+ }
+
+ protected void doRegisterFunction(String className, ClassLoader classLoader) {
+ Class clazz = null;
+ try {
+ clazz = Class.forName(className, true, classLoader);
+ doProcessor(clazz);
+ } catch (Exception e) {
+ e.printStackTrace();
+ LOG.error("初始化类错误" + e.getMessage(), e);
+ }
+ }
+
+ protected abstract void doProcessor(Class clazz);
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/AbstractChannel.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/AbstractChannel.java
new file mode 100644
index 0000000..d002fa1
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/AbstractChannel.java
@@ -0,0 +1,220 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel;
+
+import org.apache.rocketmq.streams.common.channel.sink.ISink;
+import org.apache.rocketmq.streams.common.channel.source.AbstractSource;
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+import org.apache.rocketmq.streams.common.channel.split.ISplit;
+import org.apache.rocketmq.streams.common.checkpoint.CheckPointMessage;
+import org.apache.rocketmq.streams.common.configurable.BasedConfigurable;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.context.MessageOffset;
+import org.apache.rocketmq.streams.common.interfaces.IStreamOperator;
+import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder;
+import org.apache.rocketmq.streams.common.utils.Base64Utils;
+import org.apache.rocketmq.streams.common.utils.InstantiationUtil;
+
+import java.util.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.alibaba.fastjson.JSONObject;
+
+/**
+ * channel的抽象,实现了消息的封装,发送等核心逻辑
+ */
+public abstract class AbstractChannel extends BasedConfigurable implements IChannel<IChannel> {
+ protected transient ISink sink;
+ protected transient ISource source;
+
+ protected abstract ISink createSink();
+
+ protected abstract ISource createSource();
+
+ protected transient AtomicBoolean hasCreated = new AtomicBoolean(false);
+
+ @Override
+ protected boolean initConfigurable() {
+ create();
+ return true;
+ }
+
+ protected void create() {
+ if (hasCreated.compareAndSet(false, true)) {
+ sink = createSink();
+ source = createSource();
+ sink.init();
+ source.init();
+ }
+
+ }
+
+ @Override
+ public void getJsonObject(JSONObject jsonObject) {
+
+ super.getJsonObject(jsonObject);
+ String sinkValue = jsonObject.getString("sink");
+ String sourceValue = jsonObject.getString("source");
+ create();
+ if (sourceValue != null) {
+ source = InstantiationUtil.deserializeObject(Base64Utils.decode(sourceValue));
+ source.init();
+ }
+ if (sinkValue != null) {
+ sink = InstantiationUtil.deserializeObject(Base64Utils.decode(sinkValue));
+ sink.init();
+ }
+
+ }
+
+ @Override
+ public boolean flush(String... splitIds) {
+ return sink.flush(splitIds);
+ }
+
+ @Override
+ protected void setJsonObject(JSONObject jsonObject) {
+ super.setJsonObject(jsonObject);
+ jsonObject.put("sink", Base64Utils.encode(InstantiationUtil.serializeObject(sink)));
+ jsonObject.put("source", Base64Utils.encode(InstantiationUtil.serializeObject(source)));
+ }
+
+ @Override
+ public void removeSplit(Set<String> splitIds) {
+ source.removeSplit(splitIds);
+ }
+
+ @Override
+ public void addNewSplit(Set<String> splitIds) {
+ source.addNewSplit(splitIds);
+ }
+
+ @Override
+ public Map<String, MessageOffset> getFinishedQueueIdAndOffsets(CheckPointMessage checkPointMessage) {
+ return sink.getFinishedQueueIdAndOffsets(checkPointMessage);
+ }
+
+ @Override
+ public boolean flush(Set<String> splitIds) {
+ return sink.flush(splitIds);
+ }
+
+ @Override
+ public boolean flushMessage(List<IMessage> messages) {
+ return sink.flushMessage(messages);
+ }
+
+ @Override
+ public IChannel createStageChain(PipelineBuilder pipelineBuilder) {
+ return this;
+ }
+
+ @Override
+ public void addConfigurables(PipelineBuilder pipelineBuilder) {
+ pipelineBuilder.addConfigurables(this);
+ }
+
+ @Override
+ public ISink getSink() {
+ return sink;
+ }
+
+ @Override
+ public ISource getSource() {
+ return source;
+ }
+
+ @Override
+ public boolean batchAdd(IMessage fieldName2Value, ISplit split) {
+ return sink.batchAdd(fieldName2Value, split);
+ }
+
+ @Override
+ public boolean batchAdd(IMessage fieldName2Value) {
+ return sink.batchAdd(fieldName2Value);
+ }
+
+ @Override
+ public boolean batchSave(List<IMessage> messages) {
+ return sink.batchSave(messages);
+ }
+
+ @Override
+ public boolean flush() {
+ return sink.flush();
+ }
+
+ @Override
+ public void openAutoFlush() {
+ sink.openAutoFlush();
+ }
+
+ @Override
+ public void closeAutoFlush() {
+ sink.closeAutoFlush();
+ }
+
+ @Override
+ public void setBatchSize(int batchSize) {
+ sink.setBatchSize(batchSize);
+ }
+
+ @Override
+ public int getBatchSize() {
+ return sink.getBatchSize();
+ }
+
+ @Override
+ public boolean start(IStreamOperator receiver) {
+ return source.start(receiver);
+ }
+
+ @Override
+ public String getGroupName() {
+ return source.getGroupName();
+ }
+
+ @Override
+ public void setGroupName(String groupName) {
+ source.setGroupName(groupName);
+ }
+
+ @Override
+ public int getMaxThread() {
+ return source.getMaxThread();
+ }
+
+ @Override
+ public void setMaxThread(int maxThread) {
+ source.setMaxThread(maxThread);
+ }
+
+ @Override
+ public void setMaxFetchLogGroupSize(int size) {
+ source.setMaxFetchLogGroupSize(size);
+ }
+
+ @Override
+ public long getCheckpointTime() {
+ return source.getCheckpointTime();
+ }
+
+ public void setJsonData(Boolean isJsonData) {
+ create();
+ ((AbstractSource)source).setJsonData(isJsonData);
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/IChannel.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/IChannel.java
new file mode 100644
index 0000000..f0384a3
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/IChannel.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel;
+
+import org.apache.rocketmq.streams.common.channel.sink.ISink;
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+
+import java.io.Serializable;
+
+/**
+ * source和sink的结合,dipper框架中,不单独区分source和sink,统一用channel 接口表示
+ */
+public interface IChannel<T extends IChannel>
+ extends ISink<T>, ISource<T>, Serializable {
+ String TYPE = "channel";
+
+ ISink getSink();
+
+ ISource getSource();
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/AbstractSupportShuffleChannelBuilder.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/AbstractSupportShuffleChannelBuilder.java
new file mode 100644
index 0000000..9153292
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/AbstractSupportShuffleChannelBuilder.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.builder;
+
+import com.alibaba.fastjson.JSONObject;
+
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+import org.apache.rocketmq.streams.common.utils.ConfigurableUtil;
+
+public abstract class AbstractSupportShuffleChannelBuilder implements IChannelBuilder, IShuffleChannelBuilder {
+
+ @Override
+ public ISource copy(ISource pipelineSource) {
+ JSONObject jsonObject = JSONObject.parseObject(pipelineSource.toJson());
+ return (ISource)ConfigurableUtil.create(pipelineSource.getNameSpace(), pipelineSource.getConfigureName(), jsonObject, pipelineSource.getClass().getName());
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/IChannelBuilder.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/IChannelBuilder.java
new file mode 100644
index 0000000..0c26811
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/IChannelBuilder.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.builder;
+
+import com.alibaba.fastjson.JSONObject;
+import org.apache.rocketmq.streams.common.channel.sink.ISink;
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+import org.apache.rocketmq.streams.common.metadata.MetaData;
+import org.apache.rocketmq.streams.common.utils.StringUtil;
+
+import java.util.Properties;
+
+/**
+ * 创建channel,如果需要扩展channel,需要实现这个接口,跟定属性文件,能够创建channel对象 如果想和sql对接,实现这个接口,properties中的kv是sql中with部分的内容
+ */
+public interface IChannelBuilder {
+
+ /**
+ * 主要完成sql中的with 属性和source/sink字段名的映射
+ *
+ * @param formatProperties
+ * @param inputProperties
+ * @param formatName
+ * @param inputName
+ */
+ static void formatPropertiesName(JSONObject formatProperties, Properties inputProperties, String formatName, String inputName) {
+ String inputValue = inputProperties.getProperty(inputName);
+ if (StringUtil.isNotEmpty(inputValue)) {
+ formatProperties.put(formatName, inputValue);
+ }
+ }
+
+ /**
+ * 创建channel
+ *
+ * @param properties
+ * @return
+ */
+ ISource createSource(String namespace, String name, Properties properties, MetaData metaData);
+
+ /**
+ * 返回channel类型,和blink语句中的type值一致
+ *
+ * @return
+ */
+ String getType();
+
+ /**
+ * 创建channel
+ *
+ * @param properties
+ * @return
+ */
+ ISink createSink(String namespace, String name, Properties properties, MetaData metaData);
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/IShuffleChannelBuilder.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/IShuffleChannelBuilder.java
new file mode 100644
index 0000000..a6e64f4
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/IShuffleChannelBuilder.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.builder;
+
+import org.apache.rocketmq.streams.common.channel.sink.ISink;
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+
+public interface IShuffleChannelBuilder {
+
+ /**
+ * 根据数据源source 创建一个source shuffle出来
+ *
+ * @param pipelineSource
+ * @return
+ */
+ ISource copy(ISource pipelineSource);
+
+ /**
+ * 根据数据源source 创建一个source shuffle出来
+ *
+ * @param pipelineSource
+ * @return
+ */
+ ISink createBySource(ISource pipelineSource);
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/OutputPrintChannel.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/OutputPrintChannel.java
new file mode 100644
index 0000000..775581c
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/OutputPrintChannel.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.impl;
+
+import org.apache.commons.lang3.time.StopWatch;
+import org.apache.rocketmq.streams.common.channel.sink.AbstractSink;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.utils.PrintUtil;
+
+import java.util.List;
+
+/**
+ * 测试使用,输出就是把消息打印出来
+ */
+public class OutputPrintChannel extends AbstractSink {
+
+ private static int counter = 1;
+ private transient boolean start = false;
+ private static long startTime = System.currentTimeMillis();
+ private static long begin = startTime;
+ private static int step = 40000;
+
+ @Override
+ protected boolean batchInsert(List<IMessage> messages) {
+ StringBuilder stringBuilder = new StringBuilder();
+ for (IMessage msg : messages) {
+ stringBuilder.append(msg.getMessageValue().toString() + PrintUtil.LINE);
+ }
+ System.out.println(stringBuilder.toString());
+ return false;
+ }
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/PrintChannelBuilder.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/PrintChannelBuilder.java
new file mode 100644
index 0000000..d094315
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/PrintChannelBuilder.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.impl;
+
+import java.util.Properties;
+
+import com.google.auto.service.AutoService;
+import org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder;
+import org.apache.rocketmq.streams.common.channel.sink.ISink;
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+import org.apache.rocketmq.streams.common.metadata.MetaData;
+import org.apache.rocketmq.streams.common.model.ServiceName;
+
+@AutoService(IChannelBuilder.class)
+@ServiceName(value = PrintChannelBuilder.TYPE, aliasName = "OutputPrintChannel")
+public class PrintChannelBuilder implements IChannelBuilder {
+ public static final String TYPE = "print";
+
+ @Override
+ public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) {
+ throw new RuntimeException("can not support this method");
+ }
+
+ @Override
+ public String getType() {
+ return TYPE;
+ }
+
+ @Override
+ public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) {
+ return new OutputPrintChannel();
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileChannelBuilder.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileChannelBuilder.java
new file mode 100644
index 0000000..559a9ec
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileChannelBuilder.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.impl.file;
+
+import java.util.Properties;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import com.alibaba.fastjson.JSONObject;
+
+import com.google.auto.service.AutoService;
+import org.apache.rocketmq.streams.common.channel.builder.AbstractSupportShuffleChannelBuilder;
+import org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder;
+import org.apache.rocketmq.streams.common.channel.impl.memory.MemoryChannel;
+import org.apache.rocketmq.streams.common.channel.impl.memory.MemorySink;
+import org.apache.rocketmq.streams.common.channel.impl.memory.MemorySource;
+import org.apache.rocketmq.streams.common.channel.sink.ISink;
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+import org.apache.rocketmq.streams.common.metadata.MetaData;
+import org.apache.rocketmq.streams.common.model.ServiceName;
+import org.apache.rocketmq.streams.common.utils.ConfigurableUtil;
+
+@AutoService(IChannelBuilder.class)
+@ServiceName(value = FileChannelBuilder.TYPE, aliasName = "FileSource")
+public class FileChannelBuilder extends AbstractSupportShuffleChannelBuilder {
+
+ public static final String TYPE = "file";
+
+ /**
+ * @param namespace
+ * @param name
+ * @param properties
+ * @return
+ */
+ @Override
+ public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) {
+
+ FileSource fileSource = (FileSource)ConfigurableUtil.create(FileSource.class.getName(), namespace, name, createFormatProperty(properties), null);
+ return fileSource;
+ }
+
+ @Override
+ public String getType() {
+ return TYPE;
+ }
+
+ @Override
+ public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) {
+ FileSink fileSink = (FileSink)ConfigurableUtil.create(FileSink.class.getName(), namespace, name, createFormatProperty(properties), null);
+ return fileSink;
+ }
+
+ /**
+ * 创建标准的属性文件
+ *
+ * @param properties
+ * @return
+ */
+ protected JSONObject createFormatProperty(Properties properties) {
+ JSONObject formatProperties = new JSONObject();
+ for (Object object : properties.keySet()) {
+ String key = (String)object;
+ if ("type".equals(key)) {
+ continue;
+ }
+ formatProperties.put(key, properties.getProperty(key));
+ }
+ IChannelBuilder.formatPropertiesName(formatProperties, properties, "filePath", "filePath");
+ IChannelBuilder.formatPropertiesName(formatProperties, properties, "filePath", "fileName");
+ IChannelBuilder.formatPropertiesName(formatProperties, properties, "maxThread", "thread.max.count");
+ return formatProperties;
+ }
+
+ @Override
+ public ISource copy(ISource pipelineSource) {
+ return new MemoryChannel();
+ }
+
+ @Override
+ public ISink createBySource(ISource pipelineSource) {
+ return new MemoryChannel();
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileSink.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileSink.java
new file mode 100644
index 0000000..1fbc3cc
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileSink.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.impl.file;
+
+import org.apache.rocketmq.streams.common.channel.sink.AbstractSupportShuffleSink;
+import org.apache.rocketmq.streams.common.channel.sink.ISink;
+import org.apache.rocketmq.streams.common.channel.split.ISplit;
+import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.utils.FileUtil;
+import org.apache.rocketmq.streams.common.utils.PrintUtil;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * 文件的输入输出,source是把指定的文件数据加载到内存,for循环输出到后续节点 sink,把内容写入文件,可以配置写入模式,是追加还是覆盖
+ */
+public class FileSink extends AbstractSupportShuffleSink {
+
+ /**
+ * 文件全路径。如果目录是环境变量,可以写成evnDir/文件名
+ */
+ @ENVDependence
+ private String filePath;
+
+ /**
+ * 文件写入时,是否追加
+ */
+ private boolean needAppend = true;
+
+ protected transient BufferedWriter writer;
+ /**
+ * FileChannel 中 writer 完成 初始化 标识
+ */
+ private volatile transient boolean writerInitFlag = false;
+
+ public FileSink(String filePath) {
+ this(filePath, false);
+
+ }
+
+ public FileSink(String filePath, boolean needAppend) {
+ this();
+ this.filePath = filePath;
+ this.needAppend = needAppend;
+
+ }
+
+ @Override
+ public String getShuffleTopicFieldName() {
+ return "filePath";
+ }
+
+ @Override
+ protected void createTopicIfNotExist(int splitNum) {
+
+ }
+
+ @Override
+ public List<ISplit> getSplitList() {
+ File file = new File(filePath);
+ List<ISplit> splits = new ArrayList<>();
+ splits.add(new FileSplit(file));
+ return splits;
+ }
+
+ @Override
+ public int getSplitNum() {
+ return 1;
+ }
+
+ @Override
+ protected boolean batchInsert(List<IMessage> messages) {
+ // 初始化 write 防止 文件不存在导致异常
+ initWrite();
+ if (messages != null) {
+ try {
+ for (IMessage message : messages) {
+ writer.write(message.getMessageValue().toString() + PrintUtil.LINE);
+ }
+ writer.flush();
+ } catch (IOException e) {
+ throw new RuntimeException("write line error " + filePath, e);
+ }
+ }
+ return true;
+ }
+
+ @Override
+ public void destroy() {
+
+ try {
+ if (writer != null) {
+ writer.flush();
+ writer.close();
+ }
+
+ } catch (IOException e) {
+ throw new RuntimeException("close error " + filePath, e);
+ }
+ }
+
+ public FileSink() {
+ setType(ISink.TYPE);
+ }
+
+ private static final String PREFIX = "dipper.upgrade.channel.file.envkey";
+
+ /**
+ * 初始化 witer 防止文件不存在异常
+ */
+ private void initWrite() {
+ if (!writerInitFlag) {
+ synchronized (this) {
+ if (!writerInitFlag) {
+ try {
+ writer = new BufferedWriter(new FileWriter(filePath, needAppend));
+ writerInitFlag = true;
+ } catch (Exception e) {
+ throw new RuntimeException("create write error " + filePath, e);
+ }
+ }
+ }
+ }
+ }
+
+ public String getFilePath() {
+ return filePath;
+ }
+
+ public void setFilePath(String filePath) {
+ this.filePath = filePath;
+ }
+
+ public boolean isNeedAppend() {
+ return needAppend;
+ }
+
+ public void setNeedAppend(boolean needAppend) {
+ this.needAppend = needAppend;
+ }
+
+}
+
+
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileSource.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileSource.java
new file mode 100644
index 0000000..dbf29e9
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileSource.java
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.impl.file;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.alibaba.fastjson.JSONObject;
+
+import org.apache.rocketmq.streams.common.channel.source.AbstractBatchSource;
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence;
+import org.apache.rocketmq.streams.common.utils.DateUtil;
+
+/**
+ * 文件的输入输出,source是把指定的文件数据加载到内存,for循环输出到后续节点 sink,把内容写入文件,可以配置写入模式,是追加还是覆盖
+ */
+public class FileSource extends AbstractBatchSource {
+
+ /**
+ * 文件全路径。如果目录是环境变量,可以写成evnDir/文件名
+ */
+ @ENVDependence
+ private String filePath;
+
+ protected transient BufferedReader reader;
+ protected transient ExecutorService executorService;
+
+ public FileSource(String filePath) {
+ this();
+ this.filePath = filePath;
+ }
+
+ @Override
+ protected boolean initConfigurable() {
+ super.initConfigurable();
+ File file = new File(filePath);
+ if (file.exists() && file.isDirectory()) {
+ executorService = new ThreadPoolExecutor(maxThread, maxThread,
+ 0L, TimeUnit.MILLISECONDS,
+ new LinkedBlockingQueue<Runnable>(1000));
+ }
+ return true;
+ }
+
+ @Override
+ protected boolean startSource() {
+
+ LinkedBlockingQueue<FileIterator> queue = createIteratorList();
+ AtomicInteger count = new AtomicInteger(0);
+ long startTime = System.currentTimeMillis();
+ CountDownLatch countDownLatch = new CountDownLatch(queue.size());
+ try {
+ FileIterator fileIterator = queue.poll();
+ while (fileIterator != null) {
+ ReadTask readTask = new ReadTask(fileIterator, count, countDownLatch);
+ if (executorService != null) {
+ executorService.execute(readTask);
+ } else {
+ readTask.run();
+ }
+ fileIterator = queue.poll();
+ }
+ countDownLatch.await();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+
+ System.out.println("process data cost :" + (System.currentTimeMillis() - startTime) + ", the count is " + count.get() + " now " + DateUtil.getCurrentTimeString());
+
+ return true;
+ }
+
+ /**
+ * 如果是目录,每个文件一个iterator,如果是文件只生成一个iterator
+ */
+ protected LinkedBlockingQueue<FileIterator> createIteratorList() {
+ LinkedBlockingQueue<FileIterator> iterators = new LinkedBlockingQueue<>(1000);
+ File file = new File(filePath);
+ if (file.exists() == false) {
+ return null;
+ }
+ try {
+ if (file.isDirectory() == false) {
+ iterators.put(new FileIterator(file));
+ return iterators;
+ }
+ File[] files = file.listFiles();
+ for (File subFile : files) {
+ iterators.add(new FileIterator(subFile));
+ }
+ return iterators;
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+
+ }
+
+ /**
+ * 一个文件生成一个Iterator,每次加载一行数据
+ */
+ public static class FileIterator implements Iterator<String> {
+ protected File file;
+ private String line;
+ protected int index=0;
+ protected BufferedReader reader = null;
+
+ public FileIterator(File file) throws FileNotFoundException {
+ this.file = file;
+ reader = new BufferedReader(new FileReader(file));
+ }
+
+ @Override
+ public boolean hasNext() {
+ try {
+ line = reader.readLine();
+ index++;
+ if (line != null) {
+ return true;
+ }
+ } catch (IOException e) {
+ throw new RuntimeException("read error ", e);
+ }
+
+ return false;
+ }
+
+ public int getIndex() {
+ return index;
+ }
+
+ public void close() {
+ try {
+ reader.close();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public String next() {
+ return line;
+ }
+ }
+
+ /**
+ * 如果是个目录,1个文件一个线程
+ */
+ protected class ReadTask implements Runnable {
+ protected FileIterator fileIterator;
+ protected AtomicInteger count;
+ private CountDownLatch countDownLatch;
+
+ public ReadTask(FileIterator fileIterator, AtomicInteger count, CountDownLatch countDownLatch) {
+ this.fileIterator = fileIterator;
+ this.count = count;
+ this.countDownLatch = countDownLatch;
+ }
+
+ @Override
+ public void run() {
+ if (fileIterator != null) {
+ int offset = 1;
+ while (fileIterator.hasNext()) {
+ String line = fileIterator.next();
+ doReceiveMessage(line, false, fileIterator.file.getName(), offset + "");
+ offset++;
+ count.incrementAndGet();
+
+ }
+ sendCheckpoint(fileIterator.file.getName());
+ fileIterator.close();
+ countDownLatch.countDown();
+ ;
+ }
+
+ }
+ }
+
+ @Override
+ public void destroy() {
+
+ try {
+ if (reader != null) {
+ reader.close();
+ }
+
+ } catch (IOException e) {
+ String realFilePath = filePath;
+ throw new RuntimeException("close error " + realFilePath, e);
+ }
+ }
+
+ public FileSource() {
+ setType(ISource.TYPE);
+ }
+
+ public String getFilePath() {
+ return filePath;
+ }
+
+ public void setFilePath(String filePath) {
+ this.filePath = filePath;
+ }
+
+}
+
+
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileSplit.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileSplit.java
new file mode 100644
index 0000000..134c1e6
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileSplit.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.impl.file;
+
+import java.io.File;
+
+import com.alibaba.fastjson.JSONObject;
+
+import org.apache.rocketmq.streams.common.channel.split.ISplit;
+import org.apache.rocketmq.streams.common.configurable.BasedConfigurable;
+
+public class FileSplit extends BasedConfigurable implements ISplit<FileSplit, File> {
+ protected transient File file;
+ private String filePath;
+ protected int splitId;
+
+ @Override
+ public String getQueueId() {
+ return filePath + "_" + splitId;
+ }
+
+ @Override
+ public String getPlusQueueId() {
+ return filePath + "_" + (splitId + 1);
+ }
+
+ @Override
+ public File getQueue() {
+ return file;
+ }
+
+ @Override
+ public int compareTo(FileSplit o) {
+ return splitId - o.splitId;
+ }
+
+ @Override
+ protected void getJsonObject(JSONObject jsonObject) {
+ super.getJsonObject(jsonObject);
+ File file = new File(filePath);
+ splitId = Integer.valueOf(file.getName().replace("split_", ""));
+ this.file = file;
+ }
+
+ public FileSplit(File file) {
+ this.filePath = file.getAbsolutePath();
+ splitId = Integer.valueOf(file.getName().replace("split_", ""));
+ this.file = file;
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemoryCache.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemoryCache.java
new file mode 100644
index 0000000..050e0e6
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemoryCache.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.impl.memory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import com.alibaba.fastjson.JSONObject;
+
+import org.apache.rocketmq.streams.common.configurable.BasedConfigurable;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.utils.Base64Utils;
+import org.apache.rocketmq.streams.common.utils.InstantiationUtil;
+
+public class MemoryCache extends BasedConfigurable {
+ public static String TYPE = "memoryCache";
+ protected List<String> cache = new ArrayList<>();
+ protected transient ConcurrentLinkedQueue<String> queue;
+
+ public <T> MemoryCache(T[] array) {
+ this();
+ for (T t : array) {
+ cache.add(Base64Utils.encode(InstantiationUtil.serializeObject(t)));
+ }
+ }
+
+ public MemoryCache(List<?> list) {
+ this();
+ for (Object t : list) {
+ cache.add(Base64Utils.encode(InstantiationUtil.serializeObject(t)));
+ }
+ }
+
+ public MemoryCache() {
+ setType(TYPE);
+ }
+
+ @Override
+ protected boolean initConfigurable() {
+ queue = new ConcurrentLinkedQueue();
+ for (String t : cache) {
+ byte[] bytes = Base64Utils.decode(t);
+ queue.offer(InstantiationUtil.deserializeObject(bytes));
+ }
+ return super.initConfigurable();
+ }
+
+ public List<String> getCache() {
+ return cache;
+ }
+
+ public void setCache(List<String> cache) {
+ this.cache = cache;
+ }
+
+ public ConcurrentLinkedQueue getQueue() {
+ return queue;
+ }
+
+ public void addMessage(String msg) {
+ queue.offer(msg);
+ }
+
+ public void addMessage(JSONObject msg) {
+ addMessage(msg.toJSONString());
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemoryChannel.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemoryChannel.java
new file mode 100644
index 0000000..ad6752a
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemoryChannel.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.impl.memory;
+
+import org.apache.rocketmq.streams.common.channel.AbstractChannel;
+import org.apache.rocketmq.streams.common.channel.sink.AbstractSink;
+import org.apache.rocketmq.streams.common.channel.sink.ISink;
+import org.apache.rocketmq.streams.common.channel.source.AbstractUnreliableSource;
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.context.Message;
+
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * 消息产生的source数据,就是通过sink写入的消息
+ */
+public class MemoryChannel extends AbstractChannel {
+ /**
+ * 是否启动qps的统计
+ */
+ protected transient volatile boolean startQPSCount = false;
+ /**
+ * 总处理数据数
+ */
+ protected transient AtomicLong COUNT = new AtomicLong(0);
+ /**
+ * 最早的处理时间
+ */
+ protected transient long firstReceiveTime = System.currentTimeMillis();
+
+ public void setStartQPSCount(boolean startQPSCount) {
+ this.startQPSCount = startQPSCount;
+ }
+
+ @Override
+ protected ISink createSink() {
+ return new AbstractSink() {
+ @Override
+ protected boolean batchInsert(List<IMessage> messages) {
+ if (startQPSCount) {
+ long count = COUNT.addAndGet(messages.size());
+ long second = ((System.currentTimeMillis() - firstReceiveTime) / 1000);
+ double qps = count / second;
+ System.out.println("qps is " + qps + "。the count is " + count + ".the process time is " + second);
+ }
+ for (IMessage msg : messages) {
+ ((AbstractUnreliableSource)source).doUnreliableReceiveMessage(msg.getMessageValue());
+ }
+ return true;
+ }
+ };
+ }
+
+ @Override
+ protected ISource createSource() {
+ return new AbstractUnreliableSource() {
+ @Override
+ protected boolean startSource() {
+ return super.startSource();
+ }
+ };
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemoryChannelBuilder.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemoryChannelBuilder.java
new file mode 100644
index 0000000..224b62b
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemoryChannelBuilder.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.impl.memory;
+
+import java.util.Properties;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import com.alibaba.fastjson.JSONObject;
+
+import com.google.auto.service.AutoService;
+import org.apache.rocketmq.streams.common.channel.builder.AbstractSupportShuffleChannelBuilder;
+import org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder;
+import org.apache.rocketmq.streams.common.channel.impl.file.FileSink;
+import org.apache.rocketmq.streams.common.channel.impl.file.FileSource;
+import org.apache.rocketmq.streams.common.channel.sink.ISink;
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+import org.apache.rocketmq.streams.common.metadata.MetaData;
+import org.apache.rocketmq.streams.common.model.ServiceName;
+import org.apache.rocketmq.streams.common.utils.ConfigurableUtil;
+
+@AutoService(IChannelBuilder.class)
+@ServiceName(value = MemoryChannelBuilder.TYPE, aliasName = "MemorySource")
+public class MemoryChannelBuilder extends AbstractSupportShuffleChannelBuilder {
+
+ public static final String TYPE = "memory";
+
+ /**
+ * @param namespace
+ * @param name
+ * @param properties
+ * @return
+ */
+ @Override
+ public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) {
+ return new MemorySource();
+ }
+
+ @Override
+ public String getType() {
+ return TYPE;
+ }
+
+ @Override
+ public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) {
+ return new MemorySink();
+ }
+
+ @Override
+ public ISource copy(ISource pipelineSource) {
+ return new MemoryChannel();
+ }
+
+ @Override
+ public ISink createBySource(ISource pipelineSource) {
+ return new MemoryChannel();
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemorySink.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemorySink.java
new file mode 100644
index 0000000..c9cdf11
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemorySink.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.impl.memory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.rocketmq.streams.common.channel.sink.AbstractSupportShuffleSink;
+import org.apache.rocketmq.streams.common.channel.split.ISplit;
+import org.apache.rocketmq.streams.common.configurable.IAfterConfiguableRefreshListerner;
+import org.apache.rocketmq.streams.common.configurable.IConfigurableService;
+import org.apache.rocketmq.streams.common.context.IMessage;
+
+public class MemorySink extends AbstractSupportShuffleSink implements IAfterConfiguableRefreshListerner {
+ /**
+ * 是否启动qps的统计
+ */
+ protected transient volatile boolean startQPSCount = false;
+ /**
+ * 总处理数据数
+ */
+ protected transient AtomicLong COUNT = new AtomicLong(0);
+ /**
+ * 最早的处理时间
+ */
+ protected transient long firstReceiveTime = System.currentTimeMillis();
+ protected String cacheName;
+ protected transient MemoryCache memoryCache;
+
+ public MemorySink() {}
+
+ @Override
+ protected boolean batchInsert(List<IMessage> messages) {
+ if (startQPSCount) {
+ long count = COUNT.addAndGet(messages.size());
+ long second = ((System.currentTimeMillis() - firstReceiveTime) / 1000);
+ double qps = count / second;
+ System.out.println("qps is " + qps + "。the count is " + count + ".the process time is " + second);
+ }
+ try {
+ for (IMessage msg : messages) {
+ memoryCache.queue.offer(msg.getMessageValue().toString());
+ }
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+
+ return true;
+ }
+
+ @Override
+ public String getShuffleTopicFieldName() {
+ return null;
+ }
+
+ @Override
+ protected void createTopicIfNotExist(int splitNum) {
+
+ }
+
+ @Override
+ public List<ISplit> getSplitList() {
+ List<ISplit> splits = new ArrayList<>();
+ splits.add(new MemorySplit());
+ return splits;
+ }
+
+ @Override
+ public int getSplitNum() {
+ return 1;
+ }
+
+ @Override
+ public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) {
+ memoryCache = configurableService.queryConfigurable(MemoryCache.TYPE, cacheName);
+ }
+
+ public String getCacheName() {
+ return cacheName;
+ }
+
+ public void setCacheName(String cacheName) {
+ this.cacheName = cacheName;
+ }
+
+ public void setMemoryCache(MemoryCache memoryCache) {
+ this.memoryCache = memoryCache;
+ setCacheName(memoryCache.getConfigureName());
+
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemorySource.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemorySource.java
new file mode 100644
index 0000000..806c36d
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemorySource.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.impl.memory;
+
+import org.apache.rocketmq.streams.common.channel.source.AbstractUnreliableSource;
+import org.apache.rocketmq.streams.common.configurable.IAfterConfiguableRefreshListerner;
+import org.apache.rocketmq.streams.common.configurable.IConfigurableService;
+
+public class MemorySource extends AbstractUnreliableSource implements IAfterConfiguableRefreshListerner {
+
+ protected String cacheName;
+ protected transient MemoryCache memoryCache;
+
+ public MemorySource() {
+
+ }
+
+ @Override
+ protected boolean initConfigurable() {
+ this.enableAsyncReceive = false;
+ return super.initConfigurable();
+ }
+
+ @Override
+ protected boolean startSource() {
+ boolean success = super.startSource();
+ Thread thread = new Thread(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ while (true) {
+ Object message = memoryCache.queue.poll();
+ while (message != null) {
+ doUnreliableReceiveMessage(message);
+ message = memoryCache.queue.poll();
+ }
+ sendCheckpoint(getQueueId());
+ Thread.sleep(1000);
+ }
+
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+
+ }
+ });
+ thread.start();
+ return true;
+ }
+
+ @Override
+ public String getQueueId() {
+ return "1";
+ }
+
+ public String getCacheName() {
+ return cacheName;
+ }
+
+ public void setCacheName(String cacheName) {
+ this.cacheName = cacheName;
+ }
+
+ @Override
+ public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) {
+ memoryCache = configurableService.queryConfigurable(MemoryCache.TYPE, cacheName);
+ }
+
+ public void setMemoryCache(MemoryCache memoryCache) {
+ this.memoryCache = memoryCache;
+ setCacheName(memoryCache.getConfigureName());
+
+ }
+
+ public MemoryCache getMemoryCache() {
+ return memoryCache;
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemorySplit.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemorySplit.java
new file mode 100644
index 0000000..0525df8
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemorySplit.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.impl.memory;
+
+import com.alibaba.fastjson.JSONObject;
+import org.apache.rocketmq.streams.common.channel.split.ISplit;
+import org.apache.rocketmq.streams.common.configurable.BasedConfigurable;
+
+public class MemorySplit extends BasedConfigurable implements ISplit<MemorySplit, MemorySplit> {
+ @Override
+ public String getQueueId() {
+ return 1 + "";
+ }
+
+ @Override
+ public String getPlusQueueId() {
+ return 2 + "";
+ }
+
+ @Override
+ public MemorySplit getQueue() {
+ return this;
+ }
+
+ @Override
+ public int compareTo(MemorySplit o) {
+ return 0;
+ }
+
+ @Override
+ protected void getJsonObject(JSONObject jsonObject) {
+ super.getJsonObject(jsonObject);
+
+ }
+
+ public MemorySplit() {
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractSink.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractSink.java
new file mode 100644
index 0000000..2f24491
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractSink.java
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.sink;
+
+import com.alibaba.fastjson.JSONObject;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageCache;
+import org.apache.rocketmq.streams.common.channel.sinkcache.impl.MultiSplitMessageCache;
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+import org.apache.rocketmq.streams.common.channel.split.ISplit;
+import org.apache.rocketmq.streams.common.checkpoint.CheckPointManager;
+import org.apache.rocketmq.streams.common.checkpoint.CheckPointManager.SourceState;
+import org.apache.rocketmq.streams.common.checkpoint.CheckPointMessage;
+import org.apache.rocketmq.streams.common.configurable.BasedConfigurable;
+import org.apache.rocketmq.streams.common.configurable.IConfigurableIdentification;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.context.MessageOffset;
+import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder;
+import org.apache.rocketmq.streams.common.utils.StringUtil;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * 输出的接口抽象,针对json消息的场景
+ */
+public abstract class AbstractSink extends BasedConfigurable implements ISink<AbstractSink> {
+ private static final Log LOG = LogFactory.getLog(AbstractSink.class);
+ public static String TARGET_QUEUE = "target_queue";//指定发送queue
+
+ public static final int DEFAULT_BATCH_SIZE = 3000;
+
+ protected transient IMessageCache<IMessage> messageCache;
+
+ protected volatile int batchSize = DEFAULT_BATCH_SIZE;
+ protected transient volatile Map<String, SourceState> sourceName2State = new HashMap<>();//保存完成刷新的queueid和offset
+
+ public AbstractSink() {
+ setType(TYPE);
+
+ }
+
+ @Override
+ protected boolean initConfigurable() {
+ messageCache = new MultiSplitMessageCache(this);
+ messageCache.openAutoFlush();
+ return super.initConfigurable();
+ }
+
+ @Override
+ public boolean batchAdd(IMessage fieldName2Value, ISplit split) {
+ fieldName2Value.getMessageBody().put(TARGET_QUEUE, split);
+ return batchAdd(fieldName2Value);
+ }
+
+ public ISplit getSPlit(IMessage message) {
+ return (ISplit)message.getMessageBody().get(TARGET_QUEUE);
+ }
+
+ @Override
+ public boolean batchAdd(IMessage fieldName2Value) {
+ messageCache.addCache(fieldName2Value);
+
+ return true;
+ }
+
+ @Override
+ public void openAutoFlush() {
+ messageCache.openAutoFlush();
+ }
+
+ @Override
+ public boolean batchSave(List<IMessage> messages) {
+ if (messages == null || messages.size() == 0) {
+ //LOG.warn("has empty data to insert");
+ return true;
+ }
+ int batchSize = this.batchSize;
+ if (batchSize == -1) {
+ batchSize = DEFAULT_BATCH_SIZE;
+ }
+ int length = messages.size();
+ if (length <= batchSize) {
+ batchInsert(messages);
+ return true;
+ }
+ int count = length / batchSize;
+ if (length % batchSize > 0) {
+ count++;
+ }
+ int startIndex = 0;
+ int endIndex = batchSize;
+ if (endIndex > length) {
+ endIndex = length;
+ }
+ for (int i = 0; i < count; i++) {
+ List<IMessage> batchItem = messages.subList(startIndex, endIndex);
+ batchInsert(batchItem);
+ startIndex = endIndex;
+ endIndex += batchSize;
+ if (endIndex > length) {
+ endIndex = length;
+ }
+ }
+ return true;
+ }
+
+ @Override
+ public boolean flush(Set<String> splitIds) {
+ int size = messageCache.flush(splitIds);
+ return size > 0;
+ }
+
+ @Override
+ public boolean flush(String... splitIds) {
+ if(splitIds==null){
+ return true;
+ }
+ Set<String> splitIdSet =new HashSet<>();
+ for(String splitId:splitIds){
+ splitIdSet.add(splitId);
+ }
+ return flush(splitIdSet);
+ }
+
+ protected abstract boolean batchInsert(List<IMessage> messages);
+
+ @Override
+ public void closeAutoFlush() {
+ messageCache.closeAutoFlush();
+ }
+
+ @Override
+ public boolean flushMessage(List<IMessage> messages) {
+
+ boolean success = batchSave(messages);
+ for (IMessage message : messages) {
+ String queueId = message.getHeader().getQueueId();
+ MessageOffset messageOffset = message.getHeader().getMessageOffset();
+ ISource source = message.getHeader().getSource();
+ String pipelineName = message.getHeader().getPiplineName();
+ String sourceName = CheckPointManager.createSourceName(source, pipelineName);
+ SourceState sourceState = this.sourceName2State.get(sourceName);
+ if (sourceState == null) {
+ sourceState = new SourceState();
+ sourceState.setSourceName(sourceName);
+ }
+ sourceState.getQueueId2Offsets().put(queueId, messageOffset);
+ }
+ return success;
+ }
+
+ @Override
+ public boolean flush() {
+ String name = getConfigureName();
+ if (StringUtil.isEmpty(name)) {
+ name = getClass().getName();
+ }
+ int size = messageCache.flush();
+ if (size > 0) {
+ System.out.println(name + " finish flush data " + size);
+ }
+
+ return true;
+ }
+
+ /**
+ * 把message对象转化成jsonobject
+ *
+ * @param messageList
+ * @return
+ */
+ protected List<JSONObject> convertJsonObjectFromMessage(List<IMessage> messageList) {
+ List<JSONObject> messages = new ArrayList<>();
+ for (IMessage message : messageList) {
+ messages.add(message.getMessageBody());
+ }
+ return messages;
+ }
+
+ @Override
+ public AbstractSink createStageChain(PipelineBuilder pipelineBuilder) {
+ return this;
+ }
+
+ @Override
+ public void addConfigurables(PipelineBuilder pipelineBuilder) {
+ pipelineBuilder.addConfigurables(this);
+ }
+
+ @Override
+ public int getBatchSize() {
+ return batchSize;
+ }
+
+ @Override
+ public void setBatchSize(int batchSize) {
+ this.batchSize = batchSize;
+ }
+
+ public IMessageCache<IMessage> getMessageCache() {
+ return messageCache;
+ }
+
+ @Override
+ public Map<String, MessageOffset> getFinishedQueueIdAndOffsets(CheckPointMessage checkPointMessage) {
+ String piplineName = null;
+ if (IConfigurableIdentification.class.isInstance(checkPointMessage.getStreamOperator())) {
+ IConfigurableIdentification configurable = (IConfigurableIdentification)checkPointMessage.getStreamOperator();
+ piplineName = configurable.getConfigureName();
+ }
+ SourceState sourceState = this.sourceName2State.get(
+ CheckPointManager.createSourceName(checkPointMessage.getSource(), piplineName));
+ if (sourceState != null) {
+ return sourceState.getQueueId2Offsets();
+ }
+ return new HashMap<>();
+ }
+
+ public void setMessageCache(
+ IMessageCache<IMessage> messageCache) {
+ this.messageCache = messageCache;
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractSupportShuffleSink.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractSupportShuffleSink.java
new file mode 100644
index 0000000..efcce01
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractSupportShuffleSink.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.sink;
+
+import java.util.List;
+
+import org.apache.rocketmq.streams.common.channel.IChannel;
+import org.apache.rocketmq.streams.common.channel.split.ISplit;
+import org.apache.rocketmq.streams.common.context.IMessage;
+
+public abstract class AbstractSupportShuffleSink extends AbstractSink {
+
+ protected transient int splitNum = 10;//分片个数
+
+ //sls对应的project和logstore初始化是否完成标志
+ protected volatile transient boolean hasCreated = false;
+
+ /**
+ * 获取sink的主题,在sls中是logStore,metaq是topic
+ *
+ * @return
+ */
+ public abstract String getShuffleTopicFieldName();
+
+ /**
+ * 创建一个消息队列主题,需要判断是否已经存在,如果已经存在,不需要重复创建
+ */
+ protected abstract void createTopicIfNotExist(int splitNum);
+
+ /**
+ * 获取所有的分片
+ *
+ * @return
+ */
+ public abstract List<ISplit> getSplitList();
+
+ @Override
+ protected boolean initConfigurable() {
+ boolean success = super.initConfigurable();
+ hasCreated = false;
+ checkAndCreateTopic();
+ return success;
+ }
+
+ /**
+ * 创建主题,只创建一次
+ */
+ protected void checkAndCreateTopic() {
+ if (!hasCreated) {
+ synchronized (this) {
+ if (!hasCreated) {
+ createTopicIfNotExist(splitNum);
+ hasCreated = true;
+ }
+
+ }
+ }
+
+ }
+
+ public void setSplitNum(int splitNum) {
+ this.splitNum = splitNum;
+ }
+
+ public abstract int getSplitNum();
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/ISink.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/ISink.java
new file mode 100644
index 0000000..1261320
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/ISink.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.sink;
+
+import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageFlushCallBack;
+import org.apache.rocketmq.streams.common.channel.split.ISplit;
+import org.apache.rocketmq.streams.common.checkpoint.CheckPointMessage;
+import org.apache.rocketmq.streams.common.configurable.IConfigurable;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.context.MessageOffset;
+import org.apache.rocketmq.streams.common.topology.builder.IStageBuilder;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+
+public interface ISink<T extends ISink> extends IConfigurable, IStageBuilder<T>, IMessageFlushCallBack<IMessage> {
+
+ String TYPE = "sink";
+
+ /**
+ * 根据channel推断 meta,或者不需要meta,如消息对垒
+ *
+ * @param fieldName2Value
+ * @return
+ */
+ boolean batchAdd(IMessage fieldName2Value, ISplit split);
+
+ /**
+ * 根据channel推断 meta,或者不需要meta,如消息对垒
+ *
+ * @param fieldName2Value
+ * @return
+ */
+ boolean batchAdd(IMessage fieldName2Value);
+
+ /**
+ * 直接存储存储,不过缓存
+ *
+ * @param messages
+ * @return
+ */
+ boolean batchSave(List<IMessage> messages);
+
+ /**
+ * 刷新某个分片
+ *
+ * @return
+ */
+ boolean flush(Set<String> splitId);
+
+
+ /**
+ * 刷新某个分片
+ *
+ * @return
+ */
+ boolean flush(String... splitIds);
+ /**
+ * 如果支持批量保存,此方法完成数据的全部写入
+ *
+ * @return
+ */
+ boolean flush();
+
+ /**
+ * 调用这个方法后,不必调用flush,由框架定时或定批完成刷新
+ */
+ void openAutoFlush();
+
+ void closeAutoFlush();
+
+ /**
+ * 设置缓存大小,超过条数,强制刷新
+ *
+ * @param batchSize
+ */
+ void setBatchSize(int batchSize);
+
+ /**
+ * 获取批次大小
+ *
+ * @return
+ */
+ int getBatchSize();
+
+ Map<String, MessageOffset> getFinishedQueueIdAndOffsets(CheckPointMessage checkPointMessage);
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/DataSourceAutoFlushTask.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/DataSourceAutoFlushTask.java
new file mode 100644
index 0000000..78b7547
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/DataSourceAutoFlushTask.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.sinkcache;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.rocketmq.streams.common.channel.sink.AbstractSink;
+import org.apache.rocketmq.streams.common.channel.sinkcache.impl.MessageCache;
+
+/**
+ * 自动刷新缓存的任务,开始openAutoFlush后,可以由独立线程完成数据的flush,不必显式调用
+ */
+public class DataSourceAutoFlushTask implements Runnable {
+
+ private static final Log LOG = LogFactory.getLog(DataSourceAutoFlushTask.class);
+
+ private volatile boolean isAutoFlush = false;
+ private volatile IMessageCache messageCache;
+ protected transient Long lastUpdateTime;
+
+ public DataSourceAutoFlushTask(boolean isAutoFlush,
+ IMessageCache messageCache) {
+ this.isAutoFlush = isAutoFlush;
+ this.messageCache = messageCache;
+ }
+
+ @Override
+ public void run() {
+ while (isAutoFlush) {
+ try {
+ if (messageCache.getMessageCount() < 300 && (lastUpdateTime != null && (System.currentTimeMillis() - lastUpdateTime) < 300)) {
+ Thread.sleep(100);
+ continue;
+ }
+
+ messageCache.flush();
+ lastUpdateTime = System.currentTimeMillis();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ }
+
+ public boolean isAutoFlush() {
+ return isAutoFlush;
+ }
+
+ public void setAutoFlush(boolean autoFlush) {
+ isAutoFlush = autoFlush;
+ }
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/IMessageCache.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/IMessageCache.java
new file mode 100644
index 0000000..d462336
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/IMessageCache.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.sinkcache;
+
+import java.util.Set;
+
+import org.apache.rocketmq.streams.common.metadata.MetaData;
+
+/**
+ * 消息缓存组件,支持消息缓存,刷新逻辑
+ *
+ * @param <R>
+ */
+public interface IMessageCache<R> {
+
+ /**
+ * 把消息插入缓存中
+ *
+ * @param msg 待缓存的消息
+ * @return
+ */
+ int addCache(R msg);
+
+ /**
+ * 把队列排空,并写入到存储中
+ *
+ * @return
+ */
+ int flush();
+
+ /**
+ * 刷新某个分片
+ *
+ * @return
+ */
+ int flush(Set<String> splitId);
+
+ /**
+ * 返回消息个数
+ *
+ * @return
+ */
+ Integer getMessageCount();
+
+ /**
+ * 调用这个方法后,不必调用flush,由框架定时或定批完成刷新
+ */
+ void openAutoFlush();
+
+ void closeAutoFlush();
+
+ /**
+ * 设置缓存大小,超过条数,强制刷新
+ *
+ * @param batchSize
+ */
+ void setBatchSize(int batchSize);
+
+ /**
+ * 获取批次大小
+ *
+ * @return
+ */
+ int getBatchSize();
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/IMessageFlushCallBack.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/IMessageFlushCallBack.java
new file mode 100644
index 0000000..325b4e8
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/IMessageFlushCallBack.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.sinkcache;
+
+import java.util.List;
+
+import org.apache.rocketmq.streams.common.context.IMessage;
+
+public interface IMessageFlushCallBack<R> {
+ /**
+ * 直接存储存储,不过缓存
+ *
+ * @param messages
+ * @return
+ */
+ boolean flushMessage(List<R> messages);
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/AbstractMutilSplitMessageCache.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/AbstractMutilSplitMessageCache.java
new file mode 100644
index 0000000..c2da923
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/AbstractMutilSplitMessageCache.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.sinkcache.impl;
+
+import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageCache;
+import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageFlushCallBack;
+import org.apache.rocketmq.streams.common.context.IMessage;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+public abstract class AbstractMutilSplitMessageCache<R> extends MessageCache<R> {
+ protected ConcurrentHashMap<String, MessageCache<IMessage>> queueMessageCaches = new ConcurrentHashMap();
+
+ public AbstractMutilSplitMessageCache(
+ IMessageFlushCallBack<R> flushCallBack) {
+ super(null);
+ this.flushCallBack = new MessageFlushCallBack(flushCallBack);
+ }
+
+ @Override
+ public int addCache(R msg) {
+ String queueId = createSplitId(msg);
+ MessageCache messageCache = new MessageCache(flushCallBack);
+ MessageCache existMessageCache = queueMessageCaches.putIfAbsent(queueId, messageCache);
+ if (existMessageCache != null) {
+ messageCache = existMessageCache;
+ } else {
+ messageCache.setBatchSize(batchSize);
+ messageCache.openAutoFlush();
+ }
+ messageCache.addCache(msg);
+ int size = messageCount.incrementAndGet();
+ if (batchSize > 0 && size >= batchSize) {
+ flush(queueId);
+ size = messageCount.get();
+ }
+ return size;
+ }
+
+ protected abstract String createSplitId(R msg);
+
+ @Override
+ public int flush() {
+ int size = 0;
+ for (IMessageCache cache : this.queueMessageCaches.values()) {
+ size += cache.flush();
+ }
+ return size;
+ }
+
+ @Override
+ public int flush(Set<String> splitIds) {
+ int size=0;
+ if(queueMessageCaches==null||queueMessageCaches.size()==0){
+ return 0;
+ }
+ for(String splitId:splitIds){
+
+ IMessageCache cache= queueMessageCaches.get(splitId);
+ if(cache!=null){
+ size+=cache.flush();
+ }
+
+ }
+ return size;
+ }
+
+ protected int flush(String splitId) {
+ Set<String> splitIds = new HashSet<>();
+ splitIds.add(splitId);
+ return flush(splitIds);
+ }
+
+ @Override
+ public Integer getMessageCount() {
+ return messageCount.get();
+ }
+
+ @Override
+ public void openAutoFlush() {
+ if (this.queueMessageCaches == null) {
+ return;
+ }
+ for (IMessageCache cache : this.queueMessageCaches.values()) {
+ cache.openAutoFlush();
+ }
+ }
+
+ @Override
+ public void closeAutoFlush() {
+ for (IMessageCache cache : this.queueMessageCaches.values()) {
+ cache.closeAutoFlush();
+ }
+ }
+
+ @Override
+ public void setBatchSize(int batchSize) {
+ this.batchSize = batchSize;
+ }
+
+ @Override
+ public int getBatchSize() {
+ return batchSize;
+ }
+
+ protected class MessageFlushCallBack implements IMessageFlushCallBack<R> {
+ protected IMessageFlushCallBack<R> callBack;
+
+ public MessageFlushCallBack(IMessageFlushCallBack<R> callBack) {
+ this.callBack = callBack;
+ }
+
+ @Override
+ public boolean flushMessage(List<R> messages) {
+ boolean success = callBack.flushMessage(messages);
+ messageCount.addAndGet(-messages.size());
+ return success;
+ }
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/MessageCache.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/MessageCache.java
new file mode 100644
index 0000000..00c0a58
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/MessageCache.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.sinkcache.impl;
+
+import org.apache.rocketmq.streams.common.channel.sinkcache.DataSourceAutoFlushTask;
+import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageCache;
+import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageFlushCallBack;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * 消息缓存的实现,通过消息队列做本地缓存。目前多是用了这个实现
+ */
+public class MessageCache<R> implements IMessageCache<R> {
+
+ protected IMessageFlushCallBack<R> flushCallBack;
+
+ protected volatile AtomicInteger messageCount = new AtomicInteger(0);//缓存中的数据条数
+ protected int batchSize = 1000;//最大缓存条数,超过后需要,刷新出去,做内存保护
+ protected transient DataSourceAutoFlushTask autoFlushTask;//自动任务刷新,可以均衡实时性和吞吐率
+ protected volatile transient ConcurrentLinkedQueue<R> dataQueue = new ConcurrentLinkedQueue<>();//缓存数据的消息队列
+
+ protected AtomicBoolean openAutoFlushLock = new AtomicBoolean(false);
+
+ public MessageCache(IMessageFlushCallBack<R> flushCallBack) {
+ this.flushCallBack = flushCallBack;
+ }
+
+ /**
+ * 把待插入的数据写入队列 如果缓存超过batchsize,需要强制刷新
+ *
+ * @param msg
+ * @return
+ */
+ @Override
+ public int addCache(R msg) {
+ offerQueue(msg);
+ int size = messageCount.incrementAndGet();
+ if (batchSize > 0 && size >= batchSize) {
+ flush();
+ }
+ return size;
+ }
+
+ @Override
+ public void openAutoFlush() {
+ if (openAutoFlushLock.compareAndSet(false, true)) {//可重入锁
+ autoFlushTask = new DataSourceAutoFlushTask(true, this);
+ Thread thread = new Thread(autoFlushTask);
+ thread.start();
+ }
+ }
+
+ @Override
+ public void closeAutoFlush() {
+ if (autoFlushTask != null) {
+ autoFlushTask.setAutoFlush(false);
+ openAutoFlushLock.set(false);
+ }
+ }
+
+ protected synchronized void offerQueue(R msg) {
+ dataQueue.offer(msg);
+ }
+
+ protected List<R> getMessagesFromQueue() {
+ List<R> messages = new ArrayList<>();
+ ConcurrentLinkedQueue<R> tmp = dataQueue;
+ dataQueue = new ConcurrentLinkedQueue<>();
+ while (!tmp.isEmpty()) {
+ R msg = tmp.poll();
+ messages.add(msg);
+ }
+ return messages;
+ }
+
+ @Override
+ public Integer getMessageCount() {
+ return messageCount.get();
+ }
+
+ /**
+ * 把队列排空,并写入到存储中
+ *
+ * @return
+ */
+ @Override
+ public int flush() {
+ if (getMessageCount() == 0) {
+ return 0;
+ }
+ synchronized (this) {
+ if (getMessageCount() == 0) {
+ return 0;
+ }
+ List<R> messages = null;
+ messageCount = new AtomicInteger(0);
+ messages = getMessagesFromQueue();
+ flushCallBack.flushMessage(messages);
+ return messages.size();
+ }
+
+ }
+
+ @Override
+ public int flush(Set<String> splitIds) {
+ return flush();
+ }
+
+ @Override
+ public int getBatchSize() {
+ return batchSize;
+ }
+
+ @Override
+ public void setBatchSize(int batchSize) {
+ this.batchSize = batchSize;
+ }
+
+ public IMessageFlushCallBack<R> getFlushCallBack() {
+ return flushCallBack;
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/MultiSplitMessageCache.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/MultiSplitMessageCache.java
new file mode 100644
index 0000000..622b07b
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/MultiSplitMessageCache.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.sinkcache.impl;
+
+import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageFlushCallBack;
+import org.apache.rocketmq.streams.common.context.IMessage;
+
+public class MultiSplitMessageCache extends AbstractMutilSplitMessageCache<IMessage> {
+
+ public MultiSplitMessageCache(
+ IMessageFlushCallBack<IMessage> flushCallBack) {
+ super(flushCallBack);
+ }
+
+ @Override
+ protected String createSplitId(IMessage msg) {
+ return msg.getHeader().getQueueId();
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/MutilSplitMessageCache.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/MutilSplitMessageCache.java
new file mode 100644
index 0000000..ba8ea90
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/MutilSplitMessageCache.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.streams.common.channel.sinkcache.impl;
+
+import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageFlushCallBack;
+import org.apache.rocketmq.streams.common.context.IMessage;
+
+public class MutilSplitMessageCache extends AbstractMutilSplitMessageCache<IMessage>{
+
+ public MutilSplitMessageCache(IMessageFlushCallBack<IMessage> flushCallBack) {
+ super(flushCallBack);
+ }
+
+ @Override
+ protected String createSplitId(IMessage msg) {
+ return msg.getHeader().getQueueId();
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractBatchSource.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractBatchSource.java
new file mode 100644
index 0000000..e6a16e5
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractBatchSource.java
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.source;
+
+import java.util.List;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.alibaba.fastjson.JSONObject;
+
+import org.apache.rocketmq.streams.common.context.AbstractContext;
+import org.apache.rocketmq.streams.common.context.BatchMessageOffset;
+import org.apache.rocketmq.streams.common.context.Message;
+import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder;
+import org.apache.rocketmq.streams.common.utils.RuntimeUtil;
+
+/**
+ * 用批处理实现数据流 比如通过sql,定时获取数据,这类非消息队列数据源,没有offset和queueId,系统会模拟实现 也会增加offset的存储,实现断点续传
+ */
+public abstract class AbstractBatchSource extends AbstractSource {
+
+ /**
+ * 一个批次能处理的最大数据量
+ */
+ private static final Long MAX_BATCH_SIZE = 10000000000L;
+
+ /**
+ * 因为是批次数据,所以只有一个queue
+ */
+ private static final String BATCH_MESSAGE_QUEUE_ID = "1";
+
+ /**
+ * 定时发checkpoint
+ */
+ protected transient ScheduledExecutorService scheduled;
+
+ /**
+ * 模拟offset生成,递增产生
+ */
+ protected transient AtomicLong offsetGenerator;
+
+ /**
+ * 最后一次提交的时间,用来判断是否需要checkpoint
+ */
+ protected transient long lastCommitTime;
+
+ private transient BatchMessageOffset progress;//如果需要保存offset,通过这个对象保存
+
+ public AbstractBatchSource() {
+ setBatchMessage(true);
+ }
+
+ @Override
+ protected boolean initConfigurable() {
+ scheduled = new ScheduledThreadPoolExecutor(2);
+ offsetGenerator = new AtomicLong(System.currentTimeMillis());
+ long lastCommitTime = System.currentTimeMillis();
+ return super.initConfigurable();
+ }
+
+ @Override
+ protected boolean startSource() {
+ String queueId = getQueueId();
+ scheduled.scheduleWithFixedDelay(new Runnable() {
+ @Override
+ public void run() {
+ if (System.currentTimeMillis() - lastCommitTime > getCheckpointTime()) {
+ lastCommitTime = System.currentTimeMillis();
+ sendCheckpoint(queueId);
+ }
+
+ }
+ }, 0, getCheckpointTime(), TimeUnit.SECONDS);
+ return true;
+ }
+
+ public AbstractContext doReceiveMessage(JSONObject message) {
+ return doReceiveMessage(message, false);
+ }
+
+ public AbstractContext doReceiveMessage(String message, boolean needFlush) {
+ String queueId = getQueueId();
+ String offset = this.offsetGenerator.incrementAndGet() + "";
+ return doReceiveMessage(message, needFlush, queueId, offset);
+ }
+
+ public AbstractContext doReceiveMessage(JSONObject message, boolean needFlush) {
+ String queueId = getQueueId();
+ String offset = this.offsetGenerator.incrementAndGet() + "";
+ return doReceiveMessage(message, needFlush, queueId, offset);
+ }
+
+ @Override
+ public boolean supportNewSplitFind() {
+ return false;
+ }
+
+ /**
+ * 设置初始化参数和进度
+ *
+ * @param msg
+ */
+ public void setProgress(JSONObject msg) {
+ BatchMessageOffset offset = new BatchMessageOffset();
+ if (msg != null) {
+ offset.setCurrentMessage(msg.toJSONString());
+ }
+ offset.setOwnerType(this.getType());
+ this.progress = offset;
+ }
+
+ @Override
+ public void addConfigurables(PipelineBuilder pipelineBuilder) {
+ super.addConfigurables(pipelineBuilder);
+ if (progress != null) {
+ pipelineBuilder.addConfigurables(progress);
+ }
+ }
+
+ /**
+ * 提供单条消息的处理逻辑,默认不会加入checkpoint
+ *
+ * @param message
+ * @return
+ */
+ @Override
+ public AbstractContext doReceiveMessage(JSONObject message, boolean needSetCheckPoint, String queueId,
+ String offset) {
+ Message msg = createMessage(message, queueId, offset, needSetCheckPoint);
+ return executeMessage(msg);
+ }
+
+ @Override
+ protected boolean isNotDataSplit(String queueId) {
+ return false;
+ }
+
+ /**
+ * 对于批量接入的消息,可以在消息中加入checkpoint,在这批消息执行完成后,flush所有的输出节点,确保消息至少被消费一次
+ *
+ * @param messages 这批消息会作为一个批次
+ * @param needSetCheckPoint 是否在最后一条消息加入checkpoint标志
+ * @return
+ */
+ public AbstractContext doReceiveMessage(List<JSONObject> messages, boolean needSetCheckPoint) {
+ if (messages == null || messages.size() == 0) {
+ return null;
+ }
+
+ AbstractContext context = null;
+ int i = 0;
+ for (JSONObject jsonObject : messages) {
+
+ if (i == messages.size() - 1) {
+ doReceiveMessage(jsonObject, needSetCheckPoint);
+ } else {
+ doReceiveMessage(jsonObject, false);
+ }
+ i++;
+ }
+ return context;
+ }
+
+ @Override
+ public boolean supportRemoveSplitFind() {
+ return false;
+ }
+
+ @Override
+ public boolean supportOffsetRest() {
+ return false;
+ }
+
+ @Override
+ public boolean isBatchMessage() {
+ return true;
+ }
+
+ public String getQueueId() {
+ return RuntimeUtil.getDipperInstanceId();
+ }
+
+ public Long createOffset() {
+ return offsetGenerator.incrementAndGet();
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractSource.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractSource.java
new file mode 100644
index 0000000..722175f
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractSource.java
@@ -0,0 +1,536 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.source;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.alibaba.fastjson.JSONArray;
+import com.alibaba.fastjson.JSONObject;
+
+import org.apache.rocketmq.streams.common.channel.source.systemmsg.NewSplitMessage;
+import org.apache.rocketmq.streams.common.channel.source.systemmsg.RemoveSplitMessage;
+import org.apache.rocketmq.streams.common.checkpoint.CheckPointManager;
+import org.apache.rocketmq.streams.common.checkpoint.CheckPointMessage;
+import org.apache.rocketmq.streams.common.configurable.BasedConfigurable;
+import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence;
+import org.apache.rocketmq.streams.common.context.AbstractContext;
+import org.apache.rocketmq.streams.common.context.Context;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.context.Message;
+import org.apache.rocketmq.streams.common.context.MessageHeader;
+import org.apache.rocketmq.streams.common.context.UserDefinedMessage;
+import org.apache.rocketmq.streams.common.interfaces.IStreamOperator;
+import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder;
+import org.apache.rocketmq.streams.common.utils.StringUtil;
+
+/**
+ * channel的抽象,实现了消息的封装,发送等核心逻辑
+ */
+public abstract class AbstractSource extends BasedConfigurable implements ISource<AbstractSource> {
+
+ public static String CHARSET = "UTF-8";
+
+ protected Boolean isJsonData = true;//输入的消息是否为json
+ protected Boolean msgIsJsonArray = false;//输入的消息是否为json array
+ @ENVDependence
+ protected String groupName;//group name
+ protected int maxThread = Runtime.getRuntime().availableProcessors();
+
+ @ENVDependence
+ protected String topic = "";
+
+ /**
+ * 多长时间做一次checkpoint
+ */
+ protected long checkpointTime = 1000 * 60 * 2;
+ /**
+ * 是否是批量消息,批量消息会一批做一次checkpoint,比如通过数据库加载的批消息
+ */
+ protected boolean isBatchMessage = false;
+ /**
+ * 每次拉取的最大条数,多用于消息队列
+ */
+ protected int maxFetchLogGroupSize = 100;
+
+ /**
+ * 数据源投递消息的算子,此算子用来接收source的数据,做处理
+ */
+ protected volatile transient IStreamOperator receiver;
+ /**
+ * 开启mock模式,则会收集mock数据,如果数据源没数据,则会发送mock数据
+ */
+ protected transient volatile Boolean openMock = false;
+
+ protected transient AtomicBoolean hasStart = new AtomicBoolean(false);
+
+ /**
+ * 做checkpoint的管理
+ */
+ protected transient CheckPointManager checkPointManager = new CheckPointManager();
+
+ @Override
+ protected boolean initConfigurable() {
+ hasStart = new AtomicBoolean(false);
+ openMock = false;
+ return super.initConfigurable();
+ }
+
+ @Override
+ public boolean start(IStreamOperator receiver) {
+ this.receiver = receiver;
+ boolean isStartSucess = true;
+ if (hasStart.compareAndSet(false, true)) {
+ isStartSucess = startSource();
+ }
+
+ return isStartSucess;
+ }
+
+ /**
+ * 启动 source
+ *
+ * @return
+ */
+ protected abstract boolean startSource();
+
+ public AbstractSource() {
+ setType(ISource.TYPE);
+ }
+
+ /**
+ * 提供单条消息的处理逻辑,默认不会加入checkpoint
+ *
+ * @param message
+ * @return
+ */
+ public AbstractContext doReceiveMessage(JSONObject message, boolean needSetCheckPoint, String queueId,
+ String offset) {
+ Message msg = createMessage(message, queueId, offset, needSetCheckPoint);
+ AbstractContext context = executeMessage(msg);
+ return context;
+ }
+
+ /**
+ * 处理消息,并且判断是否需要进行加入check point表识别
+ *
+ * @param message
+ * @param needSetCheckPoint
+ * @return
+ */
+ public AbstractContext doReceiveMessage(String message, boolean needSetCheckPoint, String queueId, String offset) {
+ if (this.msgIsJsonArray) {
+ JSONArray jsonArray = JSONObject.parseArray(message);
+ if (jsonArray == null || jsonArray.size() == 0) {
+ return null;
+ }
+ AbstractContext context = null;
+ for (int i = 0; i < jsonArray.size(); i++) {
+ JSONObject msgBody = jsonArray.getJSONObject(i);
+ boolean checkpoint = false;
+ if (needSetCheckPoint && i == jsonArray.size() - 1) {
+ checkpoint = true;
+ }
+ context = doReceiveMessage(msgBody, checkpoint, queueId, createBatchOffset(offset, i));
+ if (!context.isContinue()) {
+ continue;
+ }
+ }
+ return context;
+ } else {
+ JSONObject jsonObject = create(message);
+ return doReceiveMessage(jsonObject, needSetCheckPoint, queueId, offset);
+ }
+ }
+
+ /**
+ * 发送一个系统消息,执行组件不可见,告诉所有组件刷新存储
+ *
+ * @param queueId
+ */
+ public void sendCheckpoint(String queueId) {
+ Set<String> queues = new HashSet<>();
+ queues.add(queueId);
+ sendCheckpoint(queues);
+ }
+
+ /**
+ * 发送系统消息,执行组件不可见,告诉所有组件刷新存储
+ *
+ * @param queueIds
+ */
+ public void sendCheckpoint(Set<String> queueIds) {
+ JSONObject msg = new JSONObject();
+ Message message = createMessage(msg, null, null, true);
+ message.getMessageBody().put("_queues", queueIds);
+ message.getHeader().setCheckpointQueueIds(queueIds);
+ message.getHeader().setNeedFlush(true);
+ message.getHeader().setSystemMessage(true);
+ if (supportOffsetRest()) {
+ message.getHeader().setNeedFlush(false);
+ }
+
+ CheckPointMessage checkPointMessage = new CheckPointMessage();
+ checkPointMessage.setStreamOperator(this.receiver);
+ checkPointMessage.setSource(this);
+ message.setSystemMessage(checkPointMessage);
+ executeMessage(message);
+ if (checkPointMessage.isValidate() && supportOffsetRest()) {
+ saveCheckpoint(checkPointMessage);
+ }
+ }
+
+ protected void saveCheckpoint(CheckPointMessage checkPointMessage) {
+ this.checkPointManager.addCheckPointMessage(checkPointMessage);
+ }
+
+ public JSONObject createJson(Object message) {
+ JSONObject jsonObject = null;
+ if (!isJsonData) {
+ jsonObject = new UserDefinedMessage(message);
+ jsonObject.put(IMessage.DATA_KEY, message);
+ jsonObject.put(IMessage.IS_NOT_JSON_MESSAGE, true);
+ } else {
+ jsonObject = Message.parseObject(message.toString());
+ }
+ return jsonObject;
+
+ }
+
+ public JSONObject create(String message) {
+ return createJson(message);
+ }
+
+ /**
+ * 交给receiver执行后续逻辑
+ *
+ * @param channelMessage
+ * @return
+ */
+ public AbstractContext executeMessage(Message channelMessage) {
+ AbstractContext context = new Context(channelMessage);
+ if (isSplitInRemoving(channelMessage)) {
+ return context;
+ }
+ if (!channelMessage.getHeader().isSystemMessage()) {
+ messageQueueChangedCheck(channelMessage.getHeader());
+ }
+
+ boolean needFlush = channelMessage.getHeader().isSystemMessage() == false && channelMessage.getHeader().isNeedFlush();
+
+ if (receiver != null) {
+ receiver.doMessage(channelMessage, context);
+ }
+ if (needFlush) {
+ sendCheckpoint(channelMessage.getHeader().getQueueId());
+ }
+ executeMessageAfterReceiver(channelMessage, context);
+ return context;
+ }
+
+ protected boolean isSplitInRemoving(Message channelMessage) {
+ return this.checkPointManager.isRemovingSplit(channelMessage.getHeader().getQueueId());
+ }
+
+ /**
+ * source 能否自动返现新增的分片,如果不支持,系统将会模拟实现
+ *
+ * @return
+ */
+ public abstract boolean supportNewSplitFind();
+
+ /**
+ * 能否发现分片移走了,如果不支持,系统会模拟实现
+ *
+ * @return
+ */
+ public abstract boolean supportRemoveSplitFind();
+
+ /**
+ * 是否运行中,在分片发现时,自动设置分片的offset。必须支持supportNewSplitFind
+ *
+ * @return
+ */
+ public abstract boolean supportOffsetRest();
+
+ /**
+ * 系统模拟新分片发现,把消息中的分片保存下来,如果第一次收到,认为是新分片
+ *
+ * @param header
+ */
+ protected void messageQueueChangedCheck(MessageHeader header) {
+ if (supportNewSplitFind() && supportRemoveSplitFind()) {
+ return;
+ }
+ Set<String> queueIds = new HashSet<>();
+ String msgQueueId = header.getQueueId();
+ if (StringUtil.isNotEmpty(msgQueueId)) {
+ queueIds.add(msgQueueId);
+ }
+ Set<String> checkpointQueueIds = header.getCheckpointQueueIds();
+ if (checkpointQueueIds != null) {
+ queueIds.addAll(checkpointQueueIds);
+ }
+ Set<String> newQueueIds = new HashSet<>();
+ Set<String> removeQueueIds = new HashSet<>();
+ for (String queueId : queueIds) {
+ if (isNotDataSplit(queueId)) {
+ continue;
+ }
+ if (StringUtil.isNotEmpty(queueId)) {
+ if (!this.checkPointManager.contains(queueId)) {
+ synchronized (this) {
+ if (!this.checkPointManager.contains(queueId)) {
+ this.checkPointManager.addSplit(queueId);
+ newQueueIds.add(queueId);
+ }
+ }
+ } else {
+
+ this.checkPointManager.updateLastUpdate(queueId);
+ //if(this.checkPointManager.isRemovedSplit(queueId)){
+ // this.checkPointManager.removeSplit(queueId);
+ // removeQueueIds.add(queueId);
+ //}else {
+ //
+ //}
+ }
+ }
+ }
+ //if(!supportRemoveSplitFind()){
+ // removeSplit(removeQueueIds);
+ //}
+ if (!supportNewSplitFind()) {
+ addNewSplit(newQueueIds);
+ }
+
+ }
+
+ protected abstract boolean isNotDataSplit(String queueId);
+
+ @Override
+ public void removeSplit(Set<String> splitIds) {
+ if (splitIds == null || splitIds.size() == 0) {
+ return;
+ }
+ this.checkPointManager.addRemovingSplit(splitIds);
+ sendRemoveSplitSystemMessage(splitIds);
+ //先保存所有的分片
+ sendCheckpoint(splitIds);
+ this.checkPointManager.flush();
+ synchronized (this) {
+ for (String splitId : splitIds) {
+ this.checkPointManager.removeSplit(splitId);
+ }
+
+ }
+ }
+
+ @Override
+ public void addNewSplit(Set<String> splitIds) {
+ if (splitIds == null || splitIds.size() == 0) {
+ return;
+ }
+ this.checkPointManager.deleteRemovingSplit(splitIds);
+
+ JSONObject msg = new JSONObject();
+ Message message = createMessage(msg, null, null, false);
+ message.getMessageBody().put("_queues", splitIds);
+ //message.getHeader().setCheckpointQueueIds(queueIds);
+
+ message.getHeader().setNeedFlush(false);
+ message.getHeader().setSystemMessage(true);
+ NewSplitMessage systemMessage = new NewSplitMessage(splitIds, this.checkPointManager.getCurrentSplits());
+ systemMessage.setStreamOperator(this.receiver);
+ systemMessage.setSource(this);
+ message.setSystemMessage(systemMessage);
+ executeMessage(message);
+ }
+
+ /**
+ * 发送系统消息,执行组件不可见,告诉所有组件刷新存储
+ *
+ * @param queueIds
+ */
+ public void sendRemoveSplitSystemMessage(Set<String> queueIds) {
+ JSONObject msg = new JSONObject();
+ Message message = createMessage(msg, null, null, true);
+ message.getMessageBody().put("_queues", queueIds);
+ //message.getHeader().setCheckpointQueueIds(queueIds);
+ message.getHeader().setNeedFlush(true);
+ message.getHeader().setSystemMessage(true);
+ Set<String> currentSplitIds = new HashSet<>();
+ currentSplitIds.addAll(this.checkPointManager.getCurrentSplits());
+ for (String queueId : queueIds) {
+ currentSplitIds.remove(queueId);
+ }
+ RemoveSplitMessage systemMessage = new RemoveSplitMessage(queueIds, currentSplitIds);
+ systemMessage.setStreamOperator(this.receiver);
+ systemMessage.setSource(this);
+ message.setSystemMessage(systemMessage);
+ executeMessage(message);
+ }
+
+ /**
+ * 如果存在offset,做更新,这里的offset是批流的offset,有系统创建和保存,多用于数据库查询结果场景
+ *
+ * @param channelMessage
+ * @param context
+ */
+ protected void executeMessageAfterReceiver(Message channelMessage, AbstractContext context) {
+ //如果有进度,则保存进度
+ if (channelMessage.getHeader() != null && channelMessage.getHeader().getProgress() != null) {
+ JSONObject msg = channelMessage.getHeader().getProgress().getCurrentMsg();
+ Iterator<Entry<String, Object>> it = msg.entrySet().iterator();
+ JSONObject newMsg = new JSONObject();
+ newMsg.putAll(msg);
+ while (it.hasNext()) {
+ Entry<String, Object> entry = it.next();
+ String key = entry.getKey();
+ if (channelMessage.getMessageBody().containsKey(key)) {
+ newMsg.put(key, channelMessage.getMessageBody().get(key));
+ }
+ }
+ channelMessage.getHeader().getProgress().setCurrentMessage(newMsg.toJSONString());
+ channelMessage.getHeader().getProgress().update();
+ }
+
+ }
+
+ /**
+ * 把json 转换成一个message对象
+ *
+ * @param msg
+ * @return
+ */
+ public Message createMessage(JSONObject msg, String queueId, String offset, boolean checkpoint) {
+ Message channelMessage = new Message(msg);
+ channelMessage.getHeader().setSource(this);
+ channelMessage.getHeader().setOffset(offset);
+ channelMessage.getHeader().setQueueId(queueId);
+ channelMessage.getHeader().setNeedFlush(checkpoint);
+ channelMessage.setJsonMessage(isJsonData);
+ return channelMessage;
+ }
+
+ /**
+ * 每批次通过加小序号来区分offset的大小
+ *
+ * @param offset
+ * @param i
+ * @return
+ */
+ private String createBatchOffset(String offset, int i) {
+ String index = "" + i;
+ for (int j = index.length(); j < 5; j++) {
+ index = "0" + index;
+ }
+ return offset + index;
+ }
+
+ @Override
+ public void setMaxFetchLogGroupSize(int size) {
+ this.maxFetchLogGroupSize = size;
+ }
+
+ @Override
+ public AbstractSource createStageChain(PipelineBuilder pipelineBuilder) {
+ return this;
+ }
+
+ @Override
+ public void addConfigurables(PipelineBuilder pipelineBuilder) {
+ pipelineBuilder.addConfigurables(this);
+ }
+
+ @Override
+ public String getGroupName() {
+ return groupName;
+ }
+
+ @Override
+ public void setGroupName(String groupName) {
+ this.groupName = groupName;
+ }
+
+ @Override
+ public int getMaxThread() {
+ return maxThread;
+ }
+
+ @Override
+ public void setMaxThread(int maxThread) {
+ this.maxThread = maxThread;
+ }
+
+ public IStreamOperator getReceiver() {
+ return receiver;
+ }
+
+ public void setReceiver(IStreamOperator receiver) {
+ this.receiver = receiver;
+ }
+
+ public Boolean getJsonData() {
+ return isJsonData;
+ }
+
+ public void setJsonData(Boolean jsonData) {
+ isJsonData = jsonData;
+ }
+
+ public Boolean getMsgIsJsonArray() {
+ return msgIsJsonArray;
+ }
+
+ public void setMsgIsJsonArray(Boolean msgIsJsonArray) {
+ this.msgIsJsonArray = msgIsJsonArray;
+ }
+
+ public void setBatchMessage(boolean batchMessage) {
+ isBatchMessage = batchMessage;
+ }
+
+ public int getMaxFetchLogGroupSize() {
+ return maxFetchLogGroupSize;
+ }
+
+ public String getTopic() {
+ return topic;
+ }
+
+ public void setTopic(String topic) {
+ this.topic = topic;
+ }
+
+ public void setCheckpointTime(long checkpointTime) {
+ this.checkpointTime = checkpointTime;
+ }
+
+ @Override
+ public long getCheckpointTime() {
+ return checkpointTime;
+ }
+
+ public boolean isBatchMessage() {
+ return isBatchMessage;
+ }
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractSupportOffsetResetSource.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractSupportOffsetResetSource.java
new file mode 100644
index 0000000..78b6597
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractSupportOffsetResetSource.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.source;
+
+public abstract class AbstractSupportOffsetResetSource extends AbstractSource {
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractUnreliableSource.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractUnreliableSource.java
new file mode 100644
index 0000000..410c96d
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractUnreliableSource.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.source;
+
+import com.alibaba.fastjson.JSONObject;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.YieldingWaitStrategy;
+import com.lmax.disruptor.dsl.Disruptor;
+import com.lmax.disruptor.dsl.ProducerType;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.rocketmq.streams.common.context.AbstractContext;
+import org.apache.rocketmq.streams.common.context.Message;
+import org.apache.rocketmq.streams.common.disruptor.BufferFullFunction;
+import org.apache.rocketmq.streams.common.disruptor.DisruptorEvent;
+import org.apache.rocketmq.streams.common.disruptor.DisruptorEventFactory;
+import org.apache.rocketmq.streams.common.disruptor.DisruptorProducer;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 不可靠的消息源,如http,syslog,可以继承这个类。做了系统保护,如果消息发送太快,可能会出现丢失。
+ */
+public abstract class AbstractUnreliableSource extends AbstractBatchSource {
+ private static final Log LOG = LogFactory.getLog(AbstractUnreliableSource.class);
+
+ protected Boolean enableAsyncReceive = true;
+ protected boolean isSingleType = false;//是否只有单个生产者,如果是,则为true
+
+ private transient ExecutorService cachedThreadPool = null;
+ private transient int bufferSize = 1024;
+ private transient Disruptor<DisruptorEvent> disruptor;
+ private transient DisruptorProducer<Message> disruptorProducer;
+ private transient BufferFullFunction bufferFullFunction;
+ protected transient boolean discard = false;//如果过快,直接丢弃。只有enableAsyncReceive生效时使用
+ private transient EventHandler<DisruptorEvent> eventEventHandler;
+
+ @Override
+ protected boolean initConfigurable() {
+ bufferSize = 1024;
+ boolean discard = false;//如果过快,直接丢弃。只有enableAsyncReceive生效时使用
+ return super.initConfigurable();
+ }
+
+ public AbstractUnreliableSource() {
+ super();
+ if (!enableAsyncReceive) {
+ return;
+ }
+ cachedThreadPool = new ThreadPoolExecutor(maxThread, maxThread,
+ 0L, TimeUnit.MILLISECONDS,
+ new LinkedBlockingQueue<Runnable>());
+ ProducerType producerType = ProducerType.MULTI;
+ if (isSingleType) {
+ producerType = ProducerType.SINGLE;
+ }
+ disruptor = new Disruptor<>(new DisruptorEventFactory(), bufferSize, cachedThreadPool, producerType,
+ new YieldingWaitStrategy());
+ eventEventHandler = new MessageEventHandler();
+ disruptor.handleEventsWith(eventEventHandler);
+ disruptor.start();
+ disruptorProducer = new DisruptorProducer<>(disruptor);
+ //bufferFullFunction = data -> batchAdd((JSONObject)data);
+ bufferFullFunction = new BufferFullFunction() {
+ @Override
+ public void process(Object data) {
+ LOG.warn("discard data ");
+ }
+ };
+ }
+
+ @Override
+ public AbstractContext executeMessage(Message channelMessage) {
+ if (enableAsyncReceive) {
+ disruptorProducer.publish(channelMessage, bufferFullFunction, discard);
+ return null;
+ } else {
+ return executeMessageBySupper(channelMessage);
+ }
+
+ }
+
+ @Override
+ public boolean supportRemoveSplitFind() {
+ return false;
+ }
+
+ @Override
+ public boolean supportOffsetRest() {
+ return false;
+ }
+
+ @Override
+ protected boolean isNotDataSplit(String queueId) {
+ return false;
+ }
+
+ /**
+ * 收到消息发送出去,因为是不可靠队列,如果对象不是json,则用UserDefinedMessage处理
+ *
+ * @param t
+ * @param <T>
+ * @return
+ */
+ public <T> AbstractContext doUnreliableReceiveMessage(T t) {
+ return super.doReceiveMessage(createJson(t));
+ }
+
+ @Override
+ public AbstractContext doReceiveMessage(JSONObject message) {
+ return doUnreliableReceiveMessage(message);
+ }
+
+ @Override
+ protected void executeMessageAfterReceiver(Message channelMessage, AbstractContext context) {
+ }
+
+ public Boolean getEnableAsyncReceive() {
+ return enableAsyncReceive;
+ }
+
+ public void setEnableAsyncReceive(Boolean enableAsyncReceive) {
+ this.enableAsyncReceive = enableAsyncReceive;
+ }
+
+ public AbstractContext executeMessageBySupper(Message msg) {
+ return super.executeMessage(msg);
+
+ }
+
+ protected class MessageEventHandler implements EventHandler<DisruptorEvent> {
+ @Override
+ public void onEvent(DisruptorEvent event, long sequence, boolean endOfBatch) throws Exception {
+ // LOG.info("get event " + event);
+ Message msg = (Message)event.getData();
+ executeMessageBySupper(msg);
+ }
+ }
+
+ public boolean isSingleType() {
+ return isSingleType;
+ }
+
+ public void setSingleType(boolean singleType) {
+ isSingleType = singleType;
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/ISource.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/ISource.java
new file mode 100644
index 0000000..5b79b10
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/ISource.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.source;
+
+import java.util.Set;
+
+import org.apache.rocketmq.streams.common.channel.split.ISplit;
+import org.apache.rocketmq.streams.common.configurable.IConfigurable;
+import org.apache.rocketmq.streams.common.interfaces.IStreamOperator;
+import org.apache.rocketmq.streams.common.topology.builder.IStageBuilder;
+
+public interface ISource<T extends ISource> extends IConfigurable, IStageBuilder<T> {
+ String TYPE = "source";
+
+ /**
+ * 开始接收数据,把收到的数据交给receiver处理
+ *
+ * @param receiver 处理流数据
+ * @return 是否正常启动
+ */
+ boolean start(IStreamOperator receiver);
+
+ /**
+ * 同一个group name共同消费一份数据,主要针对消息队列,如果实现的子类用到这个字段,需要保持语义
+ *
+ * @return 组名
+ */
+ String getGroupName();
+
+ /**
+ * 设置组名
+ *
+ * @param groupName 组名
+ */
+ void setGroupName(String groupName);
+
+ /**
+ * 需要的最大处理线程
+ *
+ * @return 最大处理线程
+ */
+ int getMaxThread();
+
+ /**
+ * 设置线程
+ *
+ * @param maxThread 线程数
+ */
+ void setMaxThread(int maxThread);
+
+ /**
+ * 每次最大抓取个数,只针对消息队列适用,这个参数主要可以控制内存占用
+ *
+ * @param size 每次最大抓取个数
+ */
+ void setMaxFetchLogGroupSize(int size);
+
+ /**
+ * 消息超过多长时间,会被checkpoint一次,对于批量消息无效
+ *
+ * @return checkpoint时间
+ */
+ long getCheckpointTime();
+
+ /**
+ * 当分片被移走前需要做的回调
+ *
+ * @param splitIds 要移走的分片
+ */
+ void removeSplit(Set<String> splitIds);
+
+ /**
+ * 当新增分片时,需要做的回调
+ */
+ void addNewSplit(Set<String> splitIds);
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/ISplitChangedListener.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/ISplitChangedListener.java
new file mode 100644
index 0000000..fc959ab
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/ISplitChangedListener.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.source;
+
+import org.apache.rocketmq.streams.common.channel.source.systemmsg.NewSplitMessage;
+import org.apache.rocketmq.streams.common.channel.source.systemmsg.RemoveSplitMessage;
+import org.apache.rocketmq.streams.common.checkpoint.CheckPointMessage;
+import org.apache.rocketmq.streams.common.context.AbstractContext;
+import org.apache.rocketmq.streams.common.context.IMessage;
+
+public interface ISplitChangedListener {
+ /**
+ * 当channel保存commit offset时,发送系统消息,收到系统消息的stage需要完成state保存。
+ *
+ * @param message
+ * @param context
+ */
+ void addNewSplit(IMessage message, AbstractContext context, NewSplitMessage newSplitMessage);
+
+ /**
+ * 当channel保存commit offset时,发送系统消息,收到系统消息的stage需要完成state保存。
+ *
+ * @param message
+ * @param context
+ */
+ void removeSplit(IMessage message, AbstractContext context, RemoveSplitMessage removeSplitMessage);
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/systemmsg/NewSplitMessage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/systemmsg/NewSplitMessage.java
new file mode 100644
index 0000000..a3c6bc5
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/systemmsg/NewSplitMessage.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.source.systemmsg;
+
+import java.util.Set;
+
+public class NewSplitMessage extends SplitChangedMessage {
+
+ public NewSplitMessage(Set<String> splitIds, Set<String> currentSplitIds) {
+ super(splitIds, currentSplitIds, false);
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/systemmsg/RemoveSplitMessage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/systemmsg/RemoveSplitMessage.java
new file mode 100644
index 0000000..59540bd
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/systemmsg/RemoveSplitMessage.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.source.systemmsg;
+
+import java.util.Set;
+
+public class RemoveSplitMessage extends SplitChangedMessage {
+
+ public RemoveSplitMessage(Set<String> splitIds, Set<String> currentSplitIds) {
+ super(splitIds, currentSplitIds, true);
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/systemmsg/SplitChangedMessage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/systemmsg/SplitChangedMessage.java
new file mode 100644
index 0000000..909ebd7
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/systemmsg/SplitChangedMessage.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.source.systemmsg;
+
+import java.util.Set;
+
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+import org.apache.rocketmq.streams.common.interfaces.IStreamOperator;
+import org.apache.rocketmq.streams.common.interfaces.ISystemMessage;
+
+public class SplitChangedMessage implements ISystemMessage {
+ protected Set<String> splitIds;
+ protected boolean needFlush;//需要同步刷新
+ protected ISource source;//数据源对象
+ protected IStreamOperator streamOperator;//当前的pipline
+ protected Set<String> currentSplitIds = null;
+
+ public SplitChangedMessage(Set<String> splitIds, Set<String> currentSplitIds, boolean needFlush) {
+ this.splitIds = splitIds;
+ this.needFlush = needFlush;
+ this.currentSplitIds = currentSplitIds;
+ }
+
+ public Set<String> getSplitIds() {
+ return splitIds;
+ }
+
+ public void setSplitIds(Set<String> splitIds) {
+ this.splitIds = splitIds;
+ }
+
+ public boolean isNeedFlush() {
+ return needFlush;
+ }
+
+ public void setNeedFlush(boolean needFlush) {
+ this.needFlush = needFlush;
+ }
+
+ public ISource getSource() {
+ return source;
+ }
+
+ public void setSource(ISource source) {
+ this.source = source;
+ }
+
+ public IStreamOperator getStreamOperator() {
+ return streamOperator;
+ }
+
+ public void setStreamOperator(IStreamOperator streamOperator) {
+ this.streamOperator = streamOperator;
+ }
+
+ public Set<String> getCurrentSplitIds() {
+ return currentSplitIds;
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/split/ISplit.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/split/ISplit.java
new file mode 100644
index 0000000..679f0cd
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/split/ISplit.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.channel.split;
+
+import org.apache.rocketmq.streams.common.datatype.IJsonable;
+
+import java.io.Serializable;
+
+/**
+ * 对消息队列分片的抽象。代表一个分片
+ *
+ * @param <T>
+ * @param <Q>
+ */
+public interface ISplit<T, Q> extends Comparable<T>, Serializable, IJsonable {
+
+ String getQueueId();
+
+ /**
+ * 比当前queueId大的queueId的值,最好只大一点点,主要应用在rocksdb 做范围查询,因为结束部分是开区间,只要大一点点就行
+ *
+ * @return
+ */
+ String getPlusQueueId();
+
+ /**
+ * 获取具体的队列 获取具体的队列
+ *
+ * @return
+ */
+ Q getQueue();
+
+ // public T getQueue();
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/CheckPoint.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/CheckPoint.java
new file mode 100644
index 0000000..d8d1305
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/CheckPoint.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.checkpoint;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+import org.apache.rocketmq.streams.common.model.Entity;
+
+/**
+ * model for checkpoint,need save in store
+ */
+public class CheckPoint extends Entity {
+ protected String sourceNamespace;
+ protected String sourceName;
+ protected String queueId;
+ protected String offset;
+
+ public static String loadOffset(ISource source, String splitId) {
+ return null;
+ //Set<String> splits=new HashSet<>();
+ //splits.add(splitId);
+ //Map<String,String> queueId2Offset=loadOffset(source,splits);
+ //if(queueId2Offset==null||queueId2Offset.containsKey(splitId)==false){
+ // return null;
+ //}
+ //return queueId2Offset.get(splitId);
+ }
+
+ public String getQueueId() {
+ return queueId;
+ }
+
+ public void setQueueId(String queueId) {
+ this.queueId = queueId;
+ }
+
+ public String getOffset() {
+ return offset;
+ }
+
+ public void setOffset(String offset) {
+ this.offset = offset;
+ }
+
+ public String getSourceNamespace() {
+ return sourceNamespace;
+ }
+
+ public void setSourceNamespace(String sourceNamespace) {
+ this.sourceNamespace = sourceNamespace;
+ }
+
+ public String getSourceName() {
+ return sourceName;
+ }
+
+ public void setSourceName(String sourceName) {
+ this.sourceName = sourceName;
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/CheckPointManager.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/CheckPointManager.java
new file mode 100644
index 0000000..e1eae3d
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/CheckPointManager.java
@@ -0,0 +1,279 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.checkpoint;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageCache;
+import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageFlushCallBack;
+import org.apache.rocketmq.streams.common.channel.sinkcache.impl.MessageCache;
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+import org.apache.rocketmq.streams.common.configurable.IConfigurableIdentification;
+import org.apache.rocketmq.streams.common.context.MessageOffset;
+import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
+import org.apache.rocketmq.streams.common.utils.StringUtil;
+
+public class CheckPointManager {
+ protected IMessageCache<CheckPointMessage> messageCache;
+ protected transient Map<String, Long> currentSplitAndLastUpdateTime = new HashMap<>();//保存这个实例处理的分片数
+
+ protected transient Map<String, Long> removingSplits = new HashMap<>();//正在删除的分片
+
+ public CheckPointManager() {
+ messageCache = new MessageCache<>(new IMessageFlushCallBack<CheckPointMessage>() {
+ @Override
+ public boolean flushMessage(List<CheckPointMessage> messages) {
+ //合并最近的checkpoint,只存储一次,为了
+ Map<String, SourceState> sourceStateMap = mergeSourceState(messages);
+ saveCheckPoint(sourceStateMap);
+ return true;
+ }
+ });
+ messageCache.openAutoFlush();
+ }
+
+ public void flush() {
+ messageCache.flush();
+ }
+
+ public synchronized void addSplit(String splitId) {
+ this.currentSplitAndLastUpdateTime.put(splitId, System.currentTimeMillis());
+ }
+
+ public synchronized void removeSplit(String splitId) {
+ this.currentSplitAndLastUpdateTime.remove(splitId);
+ }
+
+ public boolean contains(String splitId) {
+ return this.currentSplitAndLastUpdateTime.containsKey(splitId);
+ }
+
+ /**
+ * 可能有多次的offset合并在一起,对offset合并 合并包含两个逻辑:1.同1个CheckPointMessage中,选择最小的作为本次的offset
+ *
+ * @param messages
+ */
+ protected Map<String, SourceState> mergeSourceState(List<CheckPointMessage> messages) {
+ Map<String, SourceState> sourceStateMap = new HashMap<>();
+ for (CheckPointMessage checkPointMessage : messages) {
+ SourceState sourceState = createSourceState(checkPointMessage);
+ if (sourceState == null) {
+ continue;
+ }
+ String sourceName = sourceState.getSourceName();
+ SourceState existSourceState = sourceStateMap.get(sourceName);
+ if (existSourceState != null) {
+ SourceState lastSourceState = merge(sourceState, existSourceState);
+ sourceStateMap.put(sourceName, lastSourceState);
+ }
+ }
+ return sourceStateMap;
+ }
+
+ /**
+ * 一个pipeline流程中,找最小的offset提交保存
+ *
+ * @param checkPointMessage
+ * @return
+ */
+ protected SourceState createSourceState(CheckPointMessage checkPointMessage) {
+ SourceState sourceState = new SourceState();
+ String pipelineName = null;
+ if (checkPointMessage.getStreamOperator() instanceof IConfigurableIdentification) {
+ IConfigurableIdentification configurable = (IConfigurableIdentification)checkPointMessage.getCheckPointStates();
+ pipelineName = configurable.getConfigureName();
+ }
+ Map<String, MessageOffset> queueId2Offsets = new HashMap<>();
+ sourceState.setSourceName(createSourceName(checkPointMessage.getSource(), pipelineName));
+ sourceState.setQueueId2Offsets(queueId2Offsets);
+
+ for (CheckPointState checkPointState : checkPointMessage.getCheckPointStates()) {
+ if (checkPointState.isReplyAnyOny()) {
+ continue;
+ }
+ if (checkPointState.isReplyRefuse()) {
+ return null;
+ }
+ for (Entry<String, MessageOffset> entry : checkPointState.getQueueIdAndOffset().entrySet()) {
+ String queueId = entry.getKey();
+ MessageOffset offset = entry.getValue();
+ MessageOffset existOffset = queueId2Offsets.get(queueId);
+ if (existOffset == null) {
+ queueId2Offsets.put(queueId, offset);
+ } else {
+ boolean isGreateThan = existOffset.greateThan(offset.getOffsetStr());
+ if (isGreateThan) {
+ queueId2Offsets.put(queueId, offset);
+ } else {
+ queueId2Offsets.put(queueId, existOffset);
+ }
+ }
+ }
+ }
+ return sourceState;
+ }
+
+ /**
+ * 先查询现在数据源的分片,如果已经不处理的分片,不做保存 否则把结果保存到db中
+ *
+ * @param sourceStateMap
+ */
+ protected void saveCheckPoint(Map<String, SourceState> sourceStateMap) {
+ List<CheckPoint> checkPoints = new ArrayList<>();
+ for (SourceState sourceState : sourceStateMap.values()) {
+ CheckPoint checkPoint = new CheckPoint();
+ // checkPoint.setOffset();
+ }
+ }
+
+ /**
+ * 如果多次的checkpoint在一起,先合并再保存
+ *
+ * @param sourceState
+ * @param existSourceState
+ * @return
+ */
+ protected SourceState merge(SourceState sourceState, SourceState existSourceState) {
+ Iterator<Entry<String, MessageOffset>> it = sourceState.getQueueId2Offsets().entrySet()
+ .iterator();
+ while (it.hasNext()) {
+ Entry<String, MessageOffset> entry = it.next();
+ String queueId = entry.getKey();
+ MessageOffset offset = entry.getValue();
+ MessageOffset existOffset = existSourceState.getQueueId2Offsets().get(queueId);
+ if (existOffset == null) {
+ existSourceState.getQueueId2Offsets().put(queueId, offset);
+ } else {
+ boolean isGreaterThan = offset.greateThan(existOffset.getOffsetStr());
+ if (isGreaterThan) {
+ existSourceState.getQueueId2Offsets().put(queueId, offset);
+ }
+ }
+ }
+ return existSourceState;
+ }
+
+ public void addCheckPointMessage(CheckPointMessage message) {
+ this.messageCache.addCache(message);
+ }
+
+ //public boolean isRemovedSplit(String queueId) {
+ // Long lastUpdateTime=this.currentSplitAndLastUpdateTime.get(queueId);
+ // if(lastUpdateTime==null){
+ // return false;
+ // }
+ // if(System.currentTimeMillis()-lastUpdateTime>10000*1000){
+ // return true;
+ // }
+ // return false;
+ //}
+
+ public void updateLastUpdate(String queueId) {
+ addSplit(queueId);
+ }
+
+ public Set<String> getCurrentSplits() {
+ return this.currentSplitAndLastUpdateTime.keySet();
+ }
+
+ public static class SourceState {
+ protected String sourceName;
+ protected Map<String, MessageOffset> queueId2Offsets = new HashMap<>();
+
+ public String getSourceName() {
+ return sourceName;
+ }
+
+ public void setSourceName(String sourceName) {
+ this.sourceName = sourceName;
+ }
+
+ public Map<String, MessageOffset> getQueueId2Offsets() {
+ return queueId2Offsets;
+ }
+
+ public void setQueueId2Offsets(
+ Map<String, MessageOffset> queueId2Offsets) {
+ this.queueId2Offsets = queueId2Offsets;
+ }
+ }
+
+ /**
+ * 根据source进行划分,主要是针对双流join的场景
+ *
+ * @param source
+ * @return
+ */
+ public static String createSourceName(ISource source, String piplineName) {
+ String namespace = null;
+ String name = null;
+ if (source != null) {
+ namespace = source.getNameSpace();
+ name = source.getConfigureName();
+ }
+
+ if (StringUtil.isEmpty(namespace)) {
+ namespace = "default_namespace";
+ }
+ if (StringUtil.isEmpty(name)) {
+ name = "default_name";
+ }
+ if (StringUtil.isEmpty(piplineName)) {
+ piplineName = "default_piplineName";
+ }
+ return MapKeyUtil.createKey(namespace, piplineName, name);
+ }
+
+ public Map<String, Long> getCurrentSplitAndLastUpdateTime() {
+ return currentSplitAndLastUpdateTime;
+ }
+
+ public synchronized void addRemovingSplit(Set<String> removingSplits) {
+ long removingTime = System.currentTimeMillis();
+ for (String split : removingSplits) {
+ this.removingSplits.put(split, removingTime);
+ }
+ }
+
+ public synchronized void deleteRemovingSplit(Set<String> removingSplits) {
+ for (String split : removingSplits) {
+ this.removingSplits.remove(split);
+ }
+
+ }
+
+ public synchronized boolean isRemovingSplit(String splitId) {
+ Long removingTime = this.removingSplits.get(splitId);
+ if (removingTime == null) {
+ return false;
+ }
+ //超过10秒才允许当作新分片进来
+ if (System.currentTimeMillis() - removingTime > 10 * 1000) {
+ this.removingSplits.remove(splitId);
+ return false;
+ }
+ return true;
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/CheckPointMessage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/CheckPointMessage.java
new file mode 100644
index 0000000..975ccb8
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/CheckPointMessage.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.checkpoint;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import javafx.stage.Stage;
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+import org.apache.rocketmq.streams.common.interfaces.IStreamOperator;
+import org.apache.rocketmq.streams.common.interfaces.ISystemMessage;
+import org.apache.rocketmq.streams.common.topology.ChainPipeline;
+import org.apache.rocketmq.streams.common.topology.ChainStage;
+
+public class CheckPointMessage implements ISystemMessage {
+ protected ISource source;//数据源对象
+ protected IStreamOperator streamOperator;//当前的pipline
+ protected List<CheckPointState> checkPointStates = new ArrayList<>();
+ protected boolean isValidate = true;
+
+ public ISource getSource() {
+ return source;
+ }
+
+ public void setSource(ISource source) {
+ this.source = source;
+ }
+
+ public IStreamOperator getStreamOperator() {
+ return streamOperator;
+ }
+
+ public void setStreamOperator(IStreamOperator streamOperator) {
+ this.streamOperator = streamOperator;
+ }
+
+ public List<CheckPointState> getCheckPointStates() {
+ return checkPointStates;
+ }
+
+ public void setCheckPointStates(
+ List<CheckPointState> checkPointStates) {
+ this.checkPointStates = checkPointStates;
+ }
+
+ public void reply(CheckPointState checkPointState) {
+ checkPointStates.add(checkPointState);
+ }
+
+ public void replyAnyone() {
+
+ }
+
+ public void replyRefuse() {
+ isValidate = false;
+ }
+
+ public boolean isValidate() {
+ return isValidate;
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/CheckPointState.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/CheckPointState.java
new file mode 100644
index 0000000..73508b0
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/CheckPointState.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.checkpoint;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.rocketmq.streams.common.context.MessageOffset;
+import org.apache.rocketmq.streams.common.topology.ChainPipeline;
+import org.apache.rocketmq.streams.common.topology.ChainStage;
+
+public class CheckPointState {
+ protected Map<String, MessageOffset> queueIdAndOffset = new HashMap<>();//存储已经处理完成的队列id和最大的offset
+ /**
+ * 0:基于queueIdAndOffset作为已经完成的状态;-1,本次状态不反馈,请忽略,此时不做offset保存;1.我已经反馈,但我不存储状态,以其他stage状态为主
+ */
+ protected int resultType = 0;
+
+ public Map<String, MessageOffset> getQueueIdAndOffset() {
+ return queueIdAndOffset;
+ }
+
+ public void setQueueIdAndOffset(
+ Map<String, MessageOffset> queueIdAndOffset) {
+ this.queueIdAndOffset = queueIdAndOffset;
+ }
+
+ public boolean isReplyRefuse() {
+ return this.resultType == -1;
+ }
+
+ public boolean isReplyAnyOny() {
+ return this.resultType == 1;
+ }
+
+ public void replyRefuse() {
+ this.resultType = -1;
+ }
+
+ public void replyAnyOne() {
+ this.resultType = 1;
+ }
+
+ public int getResultType() {
+ return resultType;
+ }
+
+ public void setResultType(int resultType) {
+ this.resultType = resultType;
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/ICheckPoint.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/ICheckPoint.java
new file mode 100644
index 0000000..50d9c7a
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/ICheckPoint.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.checkpoint;
+
+import org.apache.rocketmq.streams.common.context.AbstractContext;
+import org.apache.rocketmq.streams.common.context.IMessage;
+
+public interface ICheckPoint {
+
+ /**
+ * 当channel保存commit offset时,发送系统消息,收到系统消息的stage需要完成state保存。
+ *
+ * @param message
+ * @param context
+ */
+ void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage);
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/classloader/ByteClassLoader.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/classloader/ByteClassLoader.java
new file mode 100644
index 0000000..92f3f36
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/classloader/ByteClassLoader.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.classloader;
+
+public class ByteClassLoader extends ClassLoader {
+ public ByteClassLoader(ClassLoader classLoader) {
+ super(classLoader);
+ }
+
+ public Class<?> defineClass(String name, byte[] b) {
+ // ClassLoader是个抽象类,而ClassLoader.defineClass 方法是protected的
+ // 所以我们需要定义一个子类将这个方法暴露出来
+ return super.defineClass(name, b, 0, b.length);
+ }
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/classloader/FileClassLoader.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/classloader/FileClassLoader.java
new file mode 100644
index 0000000..0bf6c10
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/classloader/FileClassLoader.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.classloader;
+
+import java.io.*;
+
+/**
+ * 加载指定目录的class
+ */
+public class FileClassLoader extends ClassLoader {
+
+ protected String dir;
+
+ public FileClassLoader(String dir, ClassLoader parent) {
+ super(parent);
+ this.dir = dir;
+ }
+
+ //用于寻找类文件
+ @Override
+ public Class findClass(String name) {
+ byte[] b = loadClassData(name);
+ return defineClass(name, b, 0, b.length);
+ }
+
+ //用于加载类文件
+ private byte[] loadClassData(String fileName) {
+ String classDir = dir;
+ if (!dir.endsWith(File.separator)) {
+ classDir += File.separator;
+ }
+ if (fileName.indexOf(".") != -1) {
+ fileName = fileName.replace(".", File.separator);
+ }
+ String name = classDir + fileName + ".class";
+
+ //使用输入流读取类文件
+ InputStream in = null;
+ //使用byteArrayOutputStream保存类文件。然后转化为byte数组
+ ByteArrayOutputStream out = null;
+ try {
+ in = new FileInputStream(new File(name));
+ out = new ByteArrayOutputStream();
+ int i = 0;
+ while ((i = in.read()) != -1) {
+ out.write(i);
+ }
+
+ } catch (Exception e) {
+ } finally {
+ try {
+ out.close();
+ in.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+
+ }
+
+ return out.toByteArray();
+
+ }
+
+ public static void main(String[] args) throws ClassNotFoundException {
+ FileClassLoader fileClassLoader = new FileClassLoader("/tmp", FileClassLoader.class.getClassLoader());
+ Class clazz = fileClassLoader.loadClass("com.aliyun.yundun.dipper.channel.self.UpgradeFunction");
+ System.out.println(clazz.getName());
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/classloader/IsolationClassLoader.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/classloader/IsolationClassLoader.java
new file mode 100644
index 0000000..cd86623
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/classloader/IsolationClassLoader.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.classloader;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLClassLoader;
+
+/**
+ * 加载指定目录的class
+ */
+public class IsolationClassLoader extends URLClassLoader {
+
+ public IsolationClassLoader(String libDir, ClassLoader parent) {
+ super(new URL[] {}, parent);
+ tryLoadJarInDir(libDir);
+ }
+
+ public IsolationClassLoader(String libDir) {
+ super(new URL[] {}, null);
+ tryLoadJarInDir(libDir);
+ }
+
+ @Override
+ public Class<?> loadClass(String name) throws ClassNotFoundException {
+ return super.loadClass(name);
+ }
+
+ @Override
+ protected Class<?> findClass(String name) throws ClassNotFoundException {
+ try {
+ return super.findClass(name);
+ } catch (ClassNotFoundException e) {
+ return IsolationClassLoader.class.getClassLoader().loadClass(name);
+ }
+ }
+
+ private void tryLoadJarInDir(String dirPath) {
+ File dir = new File(dirPath);
+ // 自动加载目录下的jar包
+ if (dir.exists() && dir.isDirectory()) {
+ for (File file : dir.listFiles()) {
+ if (file.isFile() && file.getName().endsWith(".jar") && file.getName().startsWith("operator")) {
+ this.addURL(file);
+ continue;
+ }
+ }
+ } else if (dir.exists() && dir.getName().endsWith(".jar") && dir.getName().startsWith("operator")) {
+ this.addURL(dir);
+ }
+ }
+
+ private void addURL(File file) {
+ try {
+ super.addURL(new URL("file", null, file.getCanonicalPath()));
+ } catch (MalformedURLException e) {
+ e.printStackTrace();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/compiler/CustomJavaCompiler.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/compiler/CustomJavaCompiler.java
new file mode 100644
index 0000000..d8b277e
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/compiler/CustomJavaCompiler.java
@@ -0,0 +1,248 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.compiler;
+
+import org.apache.rocketmq.streams.common.utils.FileUtil;
+
+import javax.tools.*;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * 编译java源码
+ */
+public class CustomJavaCompiler {
+ //类全名
+ private String fullClassName;
+ private String sourceCode;
+ //存放编译之后的字节码(key:类全名,value:编译之后输出的字节码)
+ private Map<String, ByteJavaFileObject> javaFileObjectMap = new ConcurrentHashMap<>();
+ //获取java的编译器
+ private JavaCompiler compiler = ToolProvider.getSystemJavaCompiler();
+ //存放编译过程中输出的信息
+ private DiagnosticCollector<JavaFileObject> diagnosticsCollector = new DiagnosticCollector<>();
+ //执行结果(控制台输出的内容)
+ private String runResult;
+ //编译耗时(单位ms)
+ private long compilerTakeTime;
+ //运行耗时(单位ms)
+ private long runTakeTime;
+
+ /**
+ * @param sourceFile java 文件
+ */
+ public CustomJavaCompiler(File sourceFile) {
+ this(FileUtil.loadFileContent(sourceFile.getAbsolutePath()));
+ }
+
+ /**
+ * @param sourceCode java 源码的字符串形式
+ */
+ public CustomJavaCompiler(String sourceCode) {
+ this.sourceCode = sourceCode;
+ this.fullClassName = getFullClassName(sourceCode);
+ }
+
+ public <T> T compileAndNewInstance() {
+ try {
+ Class aClass = compileClass();
+ return (T)aClass.newInstance();
+ } catch (Exception e) {
+ throw new RuntimeException("compile class error, the class name is " + fullClassName, e);
+ }
+ }
+
+ public Class compileClass() {
+ compiler();
+ StringClassLoader scl = new StringClassLoader();
+ Class<?> aClass = null;
+ try {
+ aClass = scl.findClass(fullClassName);
+ if (aClass == null) {
+ return null;
+ }
+ return aClass;
+ } catch (Exception e) {
+ throw new RuntimeException("compile class error, the class name is " + fullClassName, e);
+ }
+ }
+
+ /**
+ * 编译字符串源代码,编译失败在 diagnosticsCollector 中获取提示信息
+ *
+ * @return true:编译成功 false:编译失败
+ */
+ protected boolean compiler() {
+ long startTime = System.currentTimeMillis();
+ //标准的内容管理器,更换成自己的实现,覆盖部分方法
+ StandardJavaFileManager standardFileManager = compiler.getStandardFileManager(diagnosticsCollector, null, null);
+ JavaFileManager javaFileManager = new StringJavaFileManage(standardFileManager);
+ //构造源代码对象
+ JavaFileObject javaFileObject = new StringJavaFileObject(fullClassName, sourceCode);
+ //获取一个编译任务
+ JavaCompiler.CompilationTask task = compiler.getTask(null, javaFileManager, diagnosticsCollector, null, null, Arrays
+ .asList(javaFileObject));
+ //设置编译耗时
+ compilerTakeTime = System.currentTimeMillis() - startTime;
+ return task.call();
+ }
+
+ /**
+ * @return 编译信息(错误 警告)
+ */
+ public String getCompilerMessage() {
+ StringBuilder sb = new StringBuilder();
+ List<Diagnostic<? extends JavaFileObject>> diagnostics = diagnosticsCollector.getDiagnostics();
+ for (Diagnostic diagnostic : diagnostics) {
+ sb.append(diagnostic.toString()).append("\r\n");
+ }
+ return sb.toString();
+ }
+
+ /**
+ * @return 控制台打印的信息
+ */
+ public String getRunResult() {
+ return runResult;
+ }
+
+ public long getCompilerTakeTime() {
+ return compilerTakeTime;
+ }
+
+ public long getRunTakeTime() {
+ return runTakeTime;
+ }
+
+ /**
+ * 获取类的全名称
+ *
+ * @param sourceCode 源码
+ * @return 类的全名称
+ */
+ public static String getFullClassName(String sourceCode) {
+ String className = "";
+ Pattern pattern = Pattern.compile("package\\s+\\S+\\s*;");
+ Matcher matcher = pattern.matcher(sourceCode);
+ if (matcher.find()) {
+ className = matcher.group().replaceFirst("package", "").replace(";", "").trim() + ".";
+ }
+
+ pattern = Pattern.compile("class\\s+\\S+.*\\{");
+ matcher = pattern.matcher(sourceCode);
+ if (matcher.find()) {
+ className += matcher.group().replaceFirst("class", "").replace("{", "").trim();
+ }
+ int index = className.indexOf(" ");
+ if (index != -1) {
+ //如果类有实现接口或继承等,只取类名部分
+ className = className.substring(0, index);
+ }
+ return className;
+ }
+
+ /**
+ * 自定义一个字符串的源码对象
+ */
+ private class StringJavaFileObject extends SimpleJavaFileObject {
+ //等待编译的源码字段
+ private String contents;
+
+ //java源代码 => StringJavaFileObject对象 的时候使用
+ public StringJavaFileObject(String className, String contents) {
+ super(URI.create("string:///" + className.replaceAll("\\.", "/") + Kind.SOURCE.extension), Kind.SOURCE);
+ this.contents = contents;
+ }
+
+ //字符串源码会调用该方法
+ @Override
+ public CharSequence getCharContent(boolean ignoreEncodingErrors) throws IOException {
+ return contents;
+ }
+
+ }
+
+ /**
+ * 自定义一个编译之后的字节码对象
+ */
+ private class ByteJavaFileObject extends SimpleJavaFileObject {
+ //存放编译后的字节码
+ private ByteArrayOutputStream outPutStream;
+
+ public ByteJavaFileObject(String className, Kind kind) {
+ super(URI.create("string:///" + className.replaceAll("\\.", "/") + Kind.SOURCE.extension), kind);
+ }
+
+ //StringJavaFileManage 编译之后的字节码输出会调用该方法(把字节码输出到outputStream)
+ @Override
+ public OutputStream openOutputStream() {
+ outPutStream = new ByteArrayOutputStream();
+ return outPutStream;
+ }
+
+ //在类加载器加载的时候需要用到
+ public byte[] getCompiledBytes() {
+ return outPutStream.toByteArray();
+ }
+ }
+
+ /**
+ * 自定义一个JavaFileManage来控制编译之后字节码的输出位置
+ */
+ private class StringJavaFileManage extends ForwardingJavaFileManager {
+ StringJavaFileManage(JavaFileManager fileManager) {
+ super(fileManager);
+ }
+
+ //获取输出的文件对象,它表示给定位置处指定类型的指定类。
+ @Override
+ public JavaFileObject getJavaFileForOutput(Location location, String className, JavaFileObject.Kind kind, FileObject sibling) throws IOException {
+ ByteJavaFileObject javaFileObject = new ByteJavaFileObject(className, kind);
+ javaFileObjectMap.put(className, javaFileObject);
+ return javaFileObject;
+ }
+ }
+
+ /**
+ * 自定义类加载器, 用来加载动态的字节码
+ */
+ private class StringClassLoader extends ClassLoader {
+ @Override
+ protected Class<?> findClass(String name) throws ClassNotFoundException {
+ ByteJavaFileObject fileObject = javaFileObjectMap.get(name);
+ if (fileObject != null) {
+ byte[] bytes = fileObject.getCompiledBytes();
+ return defineClass(name, bytes, 0, bytes.length);
+ }
+ try {
+ return ClassLoader.getSystemClassLoader().loadClass(name);
+ } catch (Exception e) {
+ return super.findClass(name);
+ }
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/AbstractComponent.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/AbstractComponent.java
new file mode 100644
index 0000000..e0aa35d
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/AbstractComponent.java
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.component;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.rocketmq.streams.common.configure.ConfigureFileKey;
+import org.apache.rocketmq.streams.common.utils.ENVUtile;
+import org.apache.rocketmq.streams.common.utils.PropertiesUtils;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public abstract class AbstractComponent<T> implements IComponent<T>, ConfigureFileKey {
+
+ private static final Log LOG = LogFactory.getLog(AbstractComponent.class);
+
+ public final static String ENV_JDBC_URL = "rocketmq_streams_sync_jdbc_url";
+
+ public final static String ENV_JDBC_USERNAME = "rocketmq_streams_sync_jdbc_username";
+
+ public final static String ENV_JDBC_PASSWORD = "rocketmq_streams_sync_jdbc_password";
+
+ public final static String ENV_JDBC_DRIVER = "rocketmq_streams_sync_jdbc_driver";
+
+ public final static String HTTP_AK = "rocketmq.streams.channel.ak";
+
+ public final static String HTTP_SK = "rocketmq.streams.channel.sk";
+
+ public final static String DEFAULT_JDBC_DRIVER = "com.mysql.jdbc.Driver";
+
+ public final static String DEFAULT_JDBC_TABLE_NAME = "rocketmq_streams_checkpoint_table";
+
+ /**
+ * xml的位置,如果没有即默认位置
+ */
+ protected PropertyConfigureDescriptorManager configureDiscriptorManager = new PropertyConfigureDescriptorManager();
+
+ private Properties properties;
+
+ @Override
+ public boolean init() {
+ Properties properties = createDefaultProperty();
+ return initProperty(properties);
+ }
+
+ protected Properties createDefaultProperty() {
+ //createENVProperties();
+ Properties properties = null;
+ properties = ComponentCreator.getProperties();
+ if (properties == null) {
+ properties = getDefaultProperties();
+ }
+ if (properties == null) {
+ properties = new Properties();
+ }
+ addSystemProperties(properties);
+ return properties;
+ }
+
+ public void initConfigurableServiceDescriptor() {
+ addConfigureDescriptor(new ConfigureDescriptor("jdbc", JDBC_URL, null, true, ENV_JDBC_URL));
+ addConfigureDescriptor(new ConfigureDescriptor("jdbc", JDBC_USERNAME, null, true, ENV_JDBC_USERNAME));
+ addConfigureDescriptor(new ConfigureDescriptor("jdbc", JDBC_PASSWORD, null, true, ENV_JDBC_PASSWORD));
+ addConfigureDescriptor(
+ new ConfigureDescriptor("jdbc", JDBC_DRIVER, DEFAULT_JDBC_DRIVER, false, ENV_JDBC_DRIVER));
+ addConfigureDescriptor(new ConfigureDescriptor("http", HTTP_AK, true));
+ addConfigureDescriptor(new ConfigureDescriptor("http", HTTP_SK, true));
+ }
+
+ protected void addConfigureDescriptor(ConfigureDescriptor configureDiscriptor) {
+ configureDiscriptorManager.addConfigureDescriptor(configureDiscriptor);
+ }
+
+ /**
+ * 如果系统属性中有对应的属性配置,覆盖掉环境变量和文件的属性,主要用于测试,不推荐用在生产环境
+ *
+ * @param properties
+ */
+ protected void addSystemProperties(Properties properties) {
+ for (List<ConfigureDescriptor> configureDescriptors : configureDiscriptorManager.getGroupByConfigures().values()) {
+ for (ConfigureDescriptor configureDiscriptor : configureDescriptors) {
+ String key = configureDiscriptor.getPropertyKey();
+ String value = ENVUtile.getSystemParameter(key);
+ if (value != null) {
+ properties.put(key, value);
+ }
+ }
+ }
+ }
+
+ /**
+ * 根据配置文件的配置情况,进行环境变量检查,如果所有的必须参数都在环境变量中,则创建属性文件。支持多组属性,如configure,既支持http配置,也支持db配置
+ *
+ * @return
+ */
+ protected Properties createENVProperties() {
+ if (configureDiscriptorManager.getGroupByConfigures() == null) {
+ return null;
+ }
+ Iterator<List<ConfigureDescriptor>> it = configureDiscriptorManager.getGroupByConfigures().values().iterator();
+ Properties properties = new Properties();
+ boolean hasProperties = false;
+ while (it.hasNext()) {
+ List<ConfigureDescriptor> configureDiscriptors = it.next();
+ Properties p = configureDiscriptorManager.createENVProperties(configureDiscriptors);
+ if (p != null) {
+ properties.putAll(p);
+ hasProperties = true;
+ }
+ }
+ if (hasProperties) {
+ return properties;
+ }
+ return null;
+ }
+
+ @Override
+ public boolean initByClassPath(String propertiesPath) {
+ Properties properties = PropertiesUtils.getResourceProperties(propertiesPath);
+ return initProperty(properties);
+ }
+
+ @Override
+ public boolean initByFilePath(String filePath) {
+ Properties properties = PropertiesUtils.loadPropertyByFilePath(filePath);
+ return initProperty(properties);
+ }
+
+ @Override
+ public boolean initByPropertiesStr(String... kvs) {
+ Properties properties = createDefaultProperty();
+ if (kvs != null && kvs.length > 0) {
+ for (String ky : kvs) {
+ PropertiesUtils.putProperty(ky, ":", properties);
+ }
+ }
+ return initProperty(properties);
+ }
+
+ /**
+ * 根据属性文件init component
+ *
+ * @param properties
+ * @return
+ */
+ protected boolean initProperty(Properties properties) {
+ boolean success = initProperties(properties);
+ this.properties = properties;
+ return success;
+ }
+
+ protected Properties getDefaultProperties() {
+ try {
+ return PropertiesUtils.getResourceProperties(getComponentPropertyPath());
+ } catch (Exception e) {
+ LOG.error("load jar file error", e);
+ return null;
+ }
+
+ }
+
+ private String getComponentPropertyPath() {
+ return PropertiesUtils.getComponentPropertyPath(this.getClass());
+ }
+
+ protected AtomicBoolean isStart = new AtomicBoolean(false);
+
+ @Override
+ public boolean start(String name) {
+ if (isStart.compareAndSet(false, true)) {
+ startComponent(name);
+ }
+ return true;
+ }
+
+ protected abstract boolean startComponent(String name);
+
+ protected void finishStart() {
+ isStart.set(true);
+ }
+
+ public Properties getProperties() {
+ return properties;
+ }
+
+ protected abstract boolean initProperties(Properties properties);
+
+ public boolean directInitProperties(Properties properties) {
+ return initProperties(properties);
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/ComponentCreator.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/ComponentCreator.java
new file mode 100644
index 0000000..de1f5df
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/ComponentCreator.java
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.component;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.rocketmq.streams.common.configurable.AbstractConfigurable;
+import org.apache.rocketmq.streams.common.configurable.IConfigurableService;
+import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
+import org.apache.rocketmq.streams.common.utils.PropertiesUtils;
+import org.apache.rocketmq.streams.common.utils.SQLUtil;
+
+import java.net.URL;
+import java.util.*;
+import java.util.Map.Entry;
+
+/**
+ * 创建组件,如果参数未发生变化(如果未传入,则是配置文件的参数),返回同一个组件对象,如果发生变化,返回不同的组件对象
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class ComponentCreator {
+
+ private static final Log LOG = LogFactory.getLog(ComponentCreator.class);
+
+ /**
+ * 代理dbchannel的class,需要继承JDBCDataSource抽象类。如果配置这个参数,则会给dbchannel增加一层代理,所有需要db访问的,都是通过open api发送sql给代理
+ */
+ public static final String DB_PROXY_CLASS_NAME = "db_proxy_class_name";
+
+ /**
+ * 创建channel的服务
+ */
+ public static final String DIPPER_INSTANCE_CHANNEL_CREATOR_SERVICE_NAME
+ = "dipper_instance_channel_creator_service_name";
+
+ /**
+ * blink jar包所在的路径
+ */
+ public static final String BLINK_UDF_JAR_PATH = "dipper.blink.udf.jar.path";
+ private static final Map<String, IComponent> key2Component = new HashMap<>();
+ private static Properties properties;
+ public static String propertiesPath;//属性文件位置,便于定期刷新
+
+ static {
+ Properties properties1 = PropertiesUtils.getResourceProperties("dipper.properties");
+ if (properties1 == null) {
+ ComponentCreator.createMemoryProperties(10000L);
+ } else {
+ ComponentCreator.setProperties(properties1);
+ }
+ }
+
+ public static String getDBProxyClassName() {
+ return properties.getProperty(DB_PROXY_CLASS_NAME);
+ }
+
+ public static void setProperties(String propertiesPath) {
+ ComponentCreator.propertiesPath = propertiesPath;
+ createProperties(propertiesPath);
+ }
+
+ public static void setProperties(Properties properties) {
+ ComponentCreator.properties = properties;
+ }
+
+ public static String[] createKV(Properties properties) {
+ List<String> keys = new ArrayList<>();
+ Iterator<Object> keyIterator = properties.keySet().iterator();
+ while (keyIterator.hasNext()) {
+ keys.add(keyIterator.next().toString());
+ }
+ Collections.sort(keys);
+ Iterator<String> it = keys.iterator();
+ String[] kvs = new String[properties.size()];
+ int i = 0;
+ while (it.hasNext()) {
+ String key = it.next();
+ String value = properties.getProperty(key);
+ kvs[i] = key + ":" + value;
+ i++;
+ }
+ return kvs;
+ }
+
+ public static void createProperties(Properties properties) {
+ ComponentCreator.properties = loadOtherProperty(properties);
+ }
+
+ public static void createDBProperties(String url, String userName, String password, String tableName, String driverClass) {
+ Properties properties = new Properties();
+ properties.put(AbstractComponent.JDBC_DRIVER, driverClass);
+ properties.put(AbstractComponent.JDBC_URL, url);
+ properties.put(AbstractComponent.JDBC_USERNAME, userName);
+ properties.put(AbstractComponent.JDBC_PASSWORD, password);
+ properties.put(AbstractComponent.JDBC_TABLE_NAME, tableName);
+ properties.put(AbstractComponent.CONNECT_TYPE, IConfigurableService.DEFAULT_SERVICE_NAME);
+ ComponentCreator.properties = loadOtherProperty(properties);
+ }
+
+ public static void createDBProperties(String url, String userName, String password, String tableName, Long pollingTime, String... kvs) {
+ Properties properties = new Properties();
+ properties.put(AbstractComponent.JDBC_DRIVER, AbstractComponent.DEFAULT_JDBC_DRIVER);
+ properties.put(AbstractComponent.JDBC_URL, url);
+ properties.put(AbstractComponent.JDBC_USERNAME, userName);
+ properties.put(AbstractComponent.JDBC_PASSWORD, password);
+ properties.put(AbstractComponent.JDBC_TABLE_NAME, tableName);
+ properties.put(AbstractComponent.POLLING_TIME, pollingTime + "");
+ properties.put(AbstractComponent.CONNECT_TYPE, IConfigurableService.DEFAULT_SERVICE_NAME);
+ ComponentCreator.properties = loadOtherProperty(properties, kvs);
+ }
+
+ public static Properties createFileProperties(String filePathName, Long pollingTime, String... kvs) {
+ Properties properties = new Properties();
+ properties.put(AbstractComponent.CONNECT_TYPE, IConfigurableService.FILE_SERVICE_NAME);
+ properties.put(IConfigurableService.FILE_PATH_NAME, filePathName);
+ properties.put(AbstractComponent.POLLING_TIME, pollingTime + "");
+ Properties properties1 = loadOtherProperty(properties, kvs);
+ ComponentCreator.properties = properties1;
+ return properties1;
+ }
+
+ public static void createMemoryProperties(Long pollingTime, String... kvs) {
+ Properties properties = new Properties();
+ properties.put(AbstractComponent.CONNECT_TYPE, IConfigurableService.MEMORY_SERVICE_NAME);
+ properties.put(AbstractComponent.POLLING_TIME, pollingTime + "");
+ ComponentCreator.properties = loadOtherProperty(properties, kvs);
+ }
+
+ public static void createProperties(String propertiesFilePath, String... kvs) {
+ Properties properties = PropertiesUtils.getResourceProperties(propertiesFilePath);
+ if (properties == null) {
+ properties = PropertiesUtils.loadPropertyByFilePath(propertiesFilePath);
+ }
+ ComponentCreator.properties = loadOtherProperty(properties, kvs);
+ }
+
+ private static Properties loadOtherProperty(Properties tmp, String... kvs) {
+ Properties properties = new Properties();
+ for (Entry<Object, Object> entry : tmp.entrySet()) {
+ String key = (String)entry.getKey();
+ String value = (String)entry.getValue();
+ String realValue = value;
+ if (value.contains("#{")) {
+ realValue = SQLUtil.parseIbatisSQL(tmp, value, true);
+ if (realValue != null && realValue.startsWith("'")) {
+ realValue = realValue.replace("'", "");
+ }
+ }
+
+ properties.put(key, realValue);
+ }
+ if (kvs == null || kvs.length == 0) {
+ return properties;
+ }
+ for (String kv : kvs) {
+ int startIndex = kv.indexOf(":");
+ String key = kv.substring(0, startIndex);
+ String value = kv.substring(startIndex + 1);
+ properties.put(key, value);
+
+ }
+ return properties;
+ }
+
+ public static <T extends IComponent> T getComponent(String namespace, Class componentType, String... kvs) {
+ return getComponent(namespace, componentType, true, kvs);
+ }
+
+ protected static <T extends IComponent> T getComponent(String namespace, Class componentType, boolean isStart,
+ String... kvs) {
+ Properties properties = loadOtherProperty(ComponentCreator.properties, kvs);
+ String[] kvArray = createKV(properties);
+ return (T)getComponentInner(namespace, componentType, isStart, kvArray);
+ }
+
+ public static <T extends IComponent> T getComponent(String namespace, Class componentType) {
+ return (T)getComponent(namespace, componentType, true, createKV(properties));
+ }
+
+ public static <T extends IComponent> T getComponentNotStart(String namespace, Class componentType) {
+ return (T)getComponentInner(namespace, componentType, false, createKV(properties));
+ }
+
+ public static <T extends IComponent> T getComponentNotStart(String namespace, Class componentType, String... kvs) {
+ return getComponent(namespace, componentType, false, kvs);
+ }
+
+ @Deprecated
+ public static <T extends IComponent> T getComponentUsingPropertiesFile(String namespace, Class componentType,
+ String propertiesPath) {
+ return (T)getComponentInner(namespace, componentType, true, propertiesPath);
+ }
+
+ private static IComponent getComponentInner(String namespace, Class<IComponent> componentType, boolean needStart, Object o) {
+ String key = createKey(componentType, namespace, o);
+ if (key2Component.containsKey(key) && key2Component.get(key) != null) {
+ return key2Component.get(key);
+ }
+ synchronized (ComponentCreator.class) {
+ if (key2Component.containsKey(key) && key2Component.get(key) != null) {
+ return key2Component.get(key);
+ }
+ try {
+ IComponent component = componentType.newInstance();
+ initComponent(component, o);
+ key2Component.put(key, component);
+ if (needStart) {
+ component.start(namespace);
+ }
+
+ return component;
+ } catch (Exception e) {
+ LOG.error("can not get component. namespace is " + namespace + ", type is " + componentType.getName(),
+ e);
+ throw new RuntimeException(e.getMessage(), e);
+ }
+ }
+
+ }
+
+ private static String createKey(Class<IComponent> componentType, String namespace, Object o) {
+ String key = MapKeyUtil.createKey(componentType.getName(), namespace);
+ if (o == null) {
+ return key;
+ } else if (o instanceof String) {
+ String propertiesPath = (String)o;
+ return MapKeyUtil.createKey(key, propertiesPath);
+ } else if (o.getClass().isArray()) {
+ String[] properties = (String[])o;
+ String pk = MapKeyUtil.createKeyBySign("_", properties);
+ return MapKeyUtil.createKey(key, pk);
+ }
+ return key;
+ }
+
+ /**
+ * 初始化组件,支持多种初始化方法
+ *
+ * @param component
+ * @param o
+ */
+ private static void initComponent(IComponent component, Object o) {
+ if (o == null) {
+ component.init();
+ } else if (o instanceof String) {
+ String propertiesPath = (String)o;
+ URL url = PropertiesUtils.class.getClassLoader().getResource(propertiesPath);
+ if (url != null) {
+ component.initByClassPath(propertiesPath);
+ } else {
+ component.initByFilePath(propertiesPath);
+ }
+ } else if (o.getClass().isArray()) {
+ component.initByPropertiesStr((String[])o);
+ }
+ return;
+ }
+
+
+ public static void createProperty(String outputFilePath) {
+ Properties properties = ComponentCreator.getProperties();
+ PropertiesUtils.flush(properties, outputFilePath);
+ }
+
+ public static Properties getProperties() {
+ return properties;
+ }
+
+ public static boolean getPropertyBooleanValue(String key) {
+ String value = properties.getProperty(key);
+ if (value == null) {
+ return false;
+ }
+ if ("true".equals(value)) {
+ return true;
+ }
+ return false;
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/ConfigureDescriptor.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/ConfigureDescriptor.java
new file mode 100644
index 0000000..ab5a336
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/ConfigureDescriptor.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.component;
+
+public class ConfigureDescriptor {
+
+ private String propertyKey;
+ private String defaultValue;
+ /**
+ * 这个值是否必须用户设置
+ */
+ private boolean requiredSet = false;
+ private String groupName;
+ /**
+ * 环境变量对应的可以
+ */
+ private String envPropertyKey;
+
+ public ConfigureDescriptor(String groupName, String propertyKey, String defaultValue, boolean requiredSet,
+ String envPropertyKey) {
+ this(groupName, propertyKey, defaultValue, requiredSet);
+ this.envPropertyKey = envPropertyKey;
+ }
+
+ public ConfigureDescriptor(String groupName, String propertyKey, String defaultValue, boolean requiredSet) {
+ this.groupName = groupName;
+ this.propertyKey = propertyKey;
+ this.defaultValue = defaultValue;
+ this.requiredSet = requiredSet;
+ this.envPropertyKey = propertyKey;
+ }
+
+ public ConfigureDescriptor(String groupName, String propertyKey, boolean requiredSet) {
+ this(groupName, propertyKey, null, requiredSet);
+ }
+
+ public ConfigureDescriptor(String propertyKey, boolean requiredSet) {
+ this(null, propertyKey, requiredSet);
+ }
+
+ public ConfigureDescriptor(String propertyKey, boolean requiredSet, String defaultValue) {
+ this(null, propertyKey, defaultValue, requiredSet);
+ }
+
+ public String getPropertyKey() {
+ return propertyKey;
+ }
+
+ public void setPropertyKey(String propertyKey) {
+ this.propertyKey = propertyKey;
+ }
+
+ public String getDefaultValue() {
+ return defaultValue;
+ }
+
+ public void setDefaultValue(String defaultValue) {
+ this.defaultValue = defaultValue;
+ }
+
+ public boolean isRequiredSet() {
+ return requiredSet;
+ }
+
+ public void setRequiredSet(boolean requiredSet) {
+ this.requiredSet = requiredSet;
+ }
+
+ public String getGroupName() {
+ return groupName;
+ }
+
+ public void setGroupName(String groupName) {
+ this.groupName = groupName;
+ }
+
+ public String getEnvPropertyKey() {
+ return envPropertyKey;
+ }
+
+ public void setEnvPropertyKey(String envPropertyKey) {
+ this.envPropertyKey = envPropertyKey;
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/IComponent.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/IComponent.java
new file mode 100644
index 0000000..1402f26
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/IComponent.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.component;
+
+/**
+ * 对组件的封装,隐藏组件的实现细节,组件的使用更简单
+ */
+public interface IComponent<T> {
+
+ /**
+ * 通过默认的配置加载组件,默认配置存放的路径为component/组件名称.default.properties
+ *
+ * @return
+ */
+ boolean init();
+
+ /**
+ * 启动组建,用到的服务需要进行初始化,默认逻辑是谁创建,谁初始化
+ *
+ * @return
+ */
+ boolean start(String namespace);
+
+ /**
+ * 关闭组建
+ *
+ * @return
+ */
+ boolean stop();
+
+ /**
+ * 获取组件对应的服务
+ *
+ * @return
+ */
+ T getService();
+
+ /**
+ * 覆盖init()属性中的内容,形式为key:value,可以多组
+ *
+ * @param kvs
+ * @return
+ */
+ @Deprecated
+ boolean initByPropertiesStr(String... kvs);
+
+ /**
+ * 加载文件初始化组件,如果是spring文件,同factory,如果是属性文件,同init,会对新创建的服务进行初始化。
+ *
+ * @param classPath
+ * @return
+ */
+ @Deprecated
+ boolean initByClassPath(String classPath);
+
+ @Deprecated
+ boolean initByFilePath(String filePath);
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/IgnoreNameSpace.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/IgnoreNameSpace.java
new file mode 100644
index 0000000..472aa0b
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/IgnoreNameSpace.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.component;
+
+/**
+ * 表明此组件,不需要namespace
+ */
+public interface IgnoreNameSpace {
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/PropertyConfigureDescriptorManager.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/PropertyConfigureDescriptorManager.java
new file mode 100644
index 0000000..873f25e
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/PropertyConfigureDescriptorManager.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.component;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.rocketmq.streams.common.utils.ENVUtile;
+import org.apache.rocketmq.streams.common.utils.StringUtil;
+
+import java.util.*;
+
+public class PropertyConfigureDescriptorManager {
+
+ private static final Log LOG = LogFactory.getLog(PropertyConfigureDescriptorManager.class);
+
+ protected transient Map<String, List<ConfigureDescriptor>> groupByConfigures = new HashMap<>();
+
+ /**
+ * 多组配置文件,比如http是一组,db是一组。需要子类在启动时在数据中写如配置描述
+ */
+ protected transient List<ConfigureDescriptor> defaultGroupConfigureDescriptors = new ArrayList<>();
+
+ /**
+ * 增加一个属性配置
+ *
+ * @param configureDescriptor
+ */
+ public void addConfigureDescriptor(ConfigureDescriptor configureDescriptor) {
+ if (configureDescriptor == null) {
+ return;
+ }
+ String groupName = configureDescriptor.getGroupName();
+ if (StringUtil.isEmpty(groupName)) {
+ defaultGroupConfigureDescriptors.add(configureDescriptor);
+ return;
+ }
+ List<ConfigureDescriptor> configureDescriptors = groupByConfigures.get(groupName);
+ if (configureDescriptors == null) {
+ synchronized (this) {
+ configureDescriptors = groupByConfigures.computeIfAbsent(groupName, k -> new ArrayList<>());
+ }
+ }
+ configureDescriptors.add(configureDescriptor);
+ }
+
+ /**
+ * 对于一组配置进行检查,如果所有的必须参数都在环境变量中,则创建属性文件
+ *
+ * @param configureDescriptorList
+ * @return
+ */
+ public Properties createENVProperties(List<ConfigureDescriptor> configureDescriptorList) {
+ if (configureDescriptorList == null || configureDescriptorList.size() == 0) {
+ return null;
+ }
+ Properties properties = new Properties();
+ for (ConfigureDescriptor configureDescriptor : configureDescriptorList) {
+ String key = configureDescriptor.getEnvPropertyKey();
+ String value = ENVUtile.getENVParameter(key);
+ if (configureDescriptor.isRequiredSet() && value == null) {
+ return null;
+ }
+ if (value != null) {
+ properties.put(configureDescriptor.getPropertyKey(), value);
+ } else {
+ value = configureDescriptor.getDefaultValue();
+ if (StringUtil.isNotEmpty(value)) {
+ properties.put(configureDescriptor.getPropertyKey(), configureDescriptor.getDefaultValue());
+ }
+ }
+ }
+ LOG.info("env properties:" + properties.entrySet());
+ return properties;
+ }
+
+ public Map<String, List<ConfigureDescriptor>> getGroupByConfigures() {
+ Map<String, List<ConfigureDescriptor>> map = new HashMap<>();
+ Iterator<Map.Entry<String, List<ConfigureDescriptor>>> it = groupByConfigures.entrySet().iterator();
+ while (it.hasNext()) {
+ Map.Entry<String, List<ConfigureDescriptor>> entry = it.next();
+ String groupName = entry.getKey();
+ List<ConfigureDescriptor> value = entry.getValue();
+ List<ConfigureDescriptor> newValue = new ArrayList<>();
+ newValue.addAll(value);
+ newValue.addAll(defaultGroupConfigureDescriptors);
+ map.put(groupName, newValue);
+ }
+ return map;
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/PropertyConfigureDiscriptorManager.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/PropertyConfigureDiscriptorManager.java
new file mode 100644
index 0000000..6858bd4
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/PropertyConfigureDiscriptorManager.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.streams.common.component;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.rocketmq.streams.common.utils.ENVUtile;
+import org.apache.rocketmq.streams.common.utils.StringUtil;
+
+import java.util.*;
+
+public class PropertyConfigureDiscriptorManager {
+
+ private static final Log LOG = LogFactory.getLog(PropertyConfigureDiscriptorManager.class);
+
+ protected transient Map<String, List<ConfigureDescriptor>> groupbyConfigures = new HashMap<>();
+
+ /**
+ * 多组配置文件,比如http是一组,db是一组。需要子类在启动时在数据中写如配置描述
+ */
+ protected transient List<ConfigureDescriptor> defaultGroupConfigureDiscriptors = new ArrayList<>();
+
+ /**
+ * 增加一个属性配置
+ *
+ * @param configureDiscriptor
+ */
+ public void addConfigureDiscriptor(ConfigureDescriptor configureDiscriptor) {
+ if (configureDiscriptor == null) {
+ return;
+ }
+ String groupName = configureDiscriptor.getGroupName();
+ if (StringUtil.isEmpty(groupName)) {
+ defaultGroupConfigureDiscriptors.add(configureDiscriptor);
+ return;
+ }
+ List<ConfigureDescriptor> configureDiscriptors = groupbyConfigures.get(groupName);
+ if (configureDiscriptors == null) {
+ synchronized (this) {
+ configureDiscriptors = groupbyConfigures.get(groupName);
+ if (configureDiscriptors == null) {
+ configureDiscriptors = new ArrayList<>();
+ groupbyConfigures.put(groupName, configureDiscriptors);
+ }
+ }
+ }
+ configureDiscriptors.add(configureDiscriptor);
+ }
+
+ /**
+ * 对于一组配置进行检查,如果所有的必须参数都在环境变量中,则创建属性文件
+ *
+ * @param configureDiscriptorList
+ * @return
+ */
+ public Properties createENVProperties(List<ConfigureDescriptor> configureDiscriptorList) {
+ if (configureDiscriptorList == null || configureDiscriptorList.size() == 0) {
+ return null;
+ }
+ Properties properties = new Properties();
+ for (ConfigureDescriptor configureDiscriptor : configureDiscriptorList) {
+ String key = configureDiscriptor.getEnvPropertyKey();
+ String value = ENVUtile.getENVParameter(key);
+ // LOG.info("@@@envkey:" + key + ",envValue:" + value);
+ if (configureDiscriptor.isRequiredSet() && value == null) {
+ return null;
+ }
+ if (value != null) {
+ properties.put(configureDiscriptor.getPropertyKey(), value);
+ } else {
+ value = configureDiscriptor.getDefaultValue();
+ if (StringUtil.isNotEmpty(value)) {
+ properties.put(configureDiscriptor.getPropertyKey(), configureDiscriptor.getDefaultValue());
+ }
+ }
+ }
+ LOG.info("env properties:" + properties.entrySet());
+ return properties;
+ }
+
+ public Map<String, List<ConfigureDescriptor>> getGroupbyConfigures() {
+ Map<String, List<ConfigureDescriptor>> map = new HashMap<>();
+ Iterator<Map.Entry<String, List<ConfigureDescriptor>>> it = groupbyConfigures.entrySet().iterator();
+ while (it.hasNext()) {
+ Map.Entry<String, List<ConfigureDescriptor>> entry = it.next();
+ String groupName = entry.getKey();
+ List<ConfigureDescriptor> value = entry.getValue();
+ List<ConfigureDescriptor> newValue = new ArrayList<>();
+ newValue.addAll(value);
+ newValue.addAll(defaultGroupConfigureDiscriptors);
+ map.put(groupName, newValue);
+ }
+ return map;
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/AbstractConfigurable.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/AbstractConfigurable.java
new file mode 100644
index 0000000..5800644
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/AbstractConfigurable.java
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.configurable;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.rocketmq.streams.common.configure.ConfigureFileKey;
+import org.apache.rocketmq.streams.common.configurable.annotation.Changeable;
+import org.apache.rocketmq.streams.common.model.Entity;
+import org.apache.rocketmq.streams.common.utils.AESUtil;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public abstract class AbstractConfigurable extends Entity implements IConfigurable {
+
+ private transient Log LOG = LogFactory.getLog(AbstractConfigurable.class);
+
+ /**
+ * 升级中心表
+ */
+ public static final String TABLE_NAME = "dipper_configure";
+
+ @Changeable
+ protected transient Map<String, Object> privateDatas = new HashMap<>();
+
+ protected transient IConfigurableService configurableService;
+
+ protected long updateFlag = 0;//通过它来触发更新,其他字段变更都不会触发更新
+
+ /**
+ * 是否完成初始化
+ */
+ private transient volatile boolean hasInit = false;
+
+ /**
+ * 是否初始化成功
+ */
+ protected transient boolean initSuccess = true;
+
+ /**
+ * 是否已经被销毁
+ */
+ protected transient boolean isDestroy = false;
+
+ /**
+ * 数据库的状态字段
+ */
+ private static final String STATUS = "status";
+
+ @Override
+ public boolean init() {
+ boolean initConfigurable = true;
+ if (!hasInit) {
+ try {
+ privateDatas = new HashMap<>();
+ hasInit = false;
+ initSuccess = true;
+ isDestroy = false;
+ initConfigurable = initConfigurable();
+ initSuccess = initConfigurable;
+ } catch (Exception e) {
+ initSuccess = false;
+ e.printStackTrace();
+ throw new RuntimeException("init configurable error " + this.toJson(), e);
+ }
+ hasInit = true;
+ }
+ return initConfigurable;
+ }
+
+ @Override
+ public void destroy() {
+ isDestroy = true;
+ }
+
+ /**
+ * 启用configurable 对象,可以被看到和应用
+ */
+ public void open() {
+ putPrivateData(STATUS, "1");
+ }
+
+ /**
+ * 关闭configuable 对象,对象失效
+ */
+ public void close() {
+ putPrivateData(STATUS, "0");
+ }
+
+ protected boolean initConfigurable() {
+ return true;
+ }
+
+ public String createSQL() {
+ return createSQL(this, TABLE_NAME);
+ }
+
+ public static String createSQL(IConfigurable configurable) {
+ return createSQL(configurable, "dipper_configure");
+ }
+
+ public static String createSQL(IConfigurable configurable, String tableName) {
+ String json = configurable.toJson();
+ Entity entity = null;
+ if (Entity.class.isInstance(configurable)) {
+ entity = (Entity)configurable;
+ } else {
+ entity = new Entity();
+ }
+ int status = 1;
+ if (configurable.getPrivateData("status") != null) {
+ status = Integer.valueOf(configurable.getPrivateData("status"));
+ }
+ String theSecretValue;
+ try {
+ theSecretValue = AESUtil.aesEncrypt(json, ConfigureFileKey.SECRECY);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ String sql = "insert into " + tableName
+ + "(gmt_create,gmt_modified,namespace,type,name,json_value,status)" + "values(" + "now(),now(),'" + configurable.getNameSpace() + "','"
+ + configurable.getType() + "','" + configurable.getConfigureName() + "','" + theSecretValue + "'," + status + ")"
+ + "ON DUPLICATE KEY UPDATE status=" + status + ", gmt_modified = now()" + ",json_value='" + theSecretValue + "'";
+ return sql;
+ }
+
+ public void update() {
+ if (configurableService != null) {
+ configurableService.update(this);
+ } else {
+ LOG.warn("can not support configurable update configurable service is null");
+ }
+ }
+
+ protected String getDipperConfigureTableName() {
+ return TABLE_NAME;
+ }
+
+ public <T extends IConfigurableService> T getConfigurableService() {
+ return (T)configurableService;
+ }
+
+ public void setConfigurableService(IConfigurableService configurableService) {
+ this.configurableService = configurableService;
+ }
+
+ @Override
+ public <T> void putPrivateData(String key, T value) {
+ this.privateDatas.put(key, value);
+ }
+
+ @Override
+ public <T> T getPrivateData(String key) {
+ return (T)this.privateDatas.get(key);
+ }
+
+ public <T> T removePrivateData(String key) {
+ return (T)this.privateDatas.remove(key);
+ }
+
+ @Override
+ public Map<String, Object> getPrivateData() {
+ return this.privateDatas;
+ }
+
+ public Map<String, Object> getPrivateDatas() {
+ return privateDatas;
+ }
+
+ public void setPrivateDatas(Map<String, Object> privateDatas) {
+ this.privateDatas = privateDatas;
+ }
+
+ public boolean isInitSuccess() {
+ return initSuccess;
+ }
+
+ public void setInitSuccess(boolean initSuccess) {
+ this.initSuccess = initSuccess;
+ }
+
+ public boolean isDestroy() {
+ return isDestroy;
+ }
+
+ public boolean isHasInit() {
+ return hasInit;
+ }
+
+ public long getUpdateFlag() {
+ return updateFlag;
+ }
+
+ public void setHasInit(boolean hasInit) {
+ this.hasInit = hasInit;
+ }
+
+ public void setUpdateFlag(long updateFlag) {
+ this.updateFlag = updateFlag;
+ }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/BasedConfigurable.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/BasedConfigurable.java
new file mode 100644
index 0000000..2865fa2
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/BasedConfigurable.java
@@ -0,0 +1,249 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.configurable;
+
+import com.alibaba.fastjson.JSON;
+import com.alibaba.fastjson.JSONObject;
+import org.apache.rocketmq.streams.common.component.ComponentCreator;
+import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence;
+import org.apache.rocketmq.streams.common.configurable.annotation.NoSerialized;
+import org.apache.rocketmq.streams.common.datatype.DataType;
+import org.apache.rocketmq.streams.common.utils.*;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+
+/**
+ * 这个类自动完成成员变量的序列化,反序列化,以及环境变量的替换 子类只要按pojo实现即可。 有几个要求: 1.需要序列化的类,必须实现getset方法,这块下个版本会优化,去掉这个限制 2.不需要序列化的字段必须加transient 关键字声明 3.成员变量是 DataType支持的类型
+ */
+public class BasedConfigurable extends AbstractConfigurable {
+
+ /**
+ * 扩展字段
+ */
+ public static final String EXTEND_FIELD_NAME = "extendField";
+
+ private String nameSpace;
+
+ private String configureName;
+
+ private String type;
+
+ protected String version = "1.0";
+
+ @Override
+ public String getNameSpace() {
+ return nameSpace;
+ }
+
+ @Override
+ public void setNameSpace(String nameSpace) {
+ this.nameSpace = nameSpace;
+ }
+
+ @Override
+ public String getConfigureName() {
+ return configureName;
+ }
+
+ @Override
+ public void setConfigureName(String configureName) {
+ this.configureName = configureName;
+ }
+
+ @Override
+ public String getType() {
+ return type;
+ }
+
+ @Override
+ public void setType(String type) {
+ this.type = type;
+ }
+
+ public JSONObject toJsonObject() {
+ JSONObject jsonObject = new JSONObject();
+ setJsonObject(jsonObject);
+ jsonObject.put(IConfigurableService.CLASS_NAME, this.getClass().getName());
+ return jsonObject;
+ }
+
+ @Override
+ public String toJson() {
+ JSONObject jsonObject = toJsonObject();
+ return jsonObject.toJSONString();
+ }
+
+ protected void setJsonObject(JSONObject jsonObject) {
+ Class<? extends BasedConfigurable> thisClass = this.getClass();
+ JSONObject tmp = jsonObject.getJSONObject(EXTEND_FIELD_NAME);
+ setJsonObject(thisClass, jsonObject);
+ }
+
+ protected void setJsonObject(Class clazz, JSONObject jsonObject) {
+ if (AbstractConfigurable.class.getName().equals(clazz.getName())) {
+ return;
+ }
+ Field[] fields = clazz.getDeclaredFields();
+ for (Field field : fields) {
+ if (field.isAnnotationPresent(NoSerialized.class)) {
+ continue;
+ }
+ if (Modifier.isStatic(field.getModifiers())) {
+ continue;
+ } else if (Modifier.isTransient(field.getModifiers())) {
+ continue;
+ } else if (Modifier.isNative(field.getModifiers())) {
+ continue;
+ } else if (field.getName().endsWith("this$0")) {
+ continue;
+ }
+
+ DataType dataType = DataTypeUtil.createFieldDataType(this, field.getName());
+ Object fieldValue = ReflectUtil.getBeanFieldValue(this, field.getName());
+ if (fieldValue != null) {
+ // 如果是空值则不再处理。入库也没意义
+ String fieldValueStr = dataType.toDataJson(fieldValue);
+ fieldValueStr = restoreFieldValue(field, fieldValueStr);
+ jsonObject.put(field.getName(), fieldValueStr);
+ }
+ }
+ Class parent = clazz.getSuperclass();
+ setJsonObject(parent, jsonObject);
+ }
+
+ public int getStatus() {
+ int status = 1;
+ if (this.getPrivateData("status") != null) {
+ status = Integer.valueOf(this.getPrivateData("status"));
+ }
+ return status;
+ }
+
+ protected void getJsonObject(JSONObject jsonObject) {
+ Class thisClass = this.getClass();
+ getJsonObject(thisClass, jsonObject);
+ }
+
+ protected void getJsonObject(Class clazz, JSONObject jsonObject) {
+ if (AbstractConfigurable.class.getName().equals(clazz.getName())) {
+ return;
+ }
+ if (Object.class.getName().equals(clazz.getName())) {
+ return;
+ }
+ Field[] fields = clazz.getDeclaredFields();
+ for (Field field : fields) {
+ if (field.isAnnotationPresent(NoSerialized.class)) {
+ continue;
+ }
+ if (Modifier.isStatic(field.getModifiers())) {
+ continue;
+ } else if (Modifier.isTransient(field.getModifiers())) {
+ continue;
+ } else if (Modifier.isNative(field.getModifiers())) {
+ continue;
+ }
+ DataType dataType = DataTypeUtil.createFieldDataType(this, field.getName());
+ String fieldJsonStr = jsonObject.getString(field.getName());
+ fieldJsonStr = getENVParamter(field, fieldJsonStr);
+ Object fieldValue = dataType.getData(fieldJsonStr);
+ if (fieldValue != null) {
+ ReflectUtil.setBeanFieldValue(this, field.getName(), fieldValue);
+ } else {
+ ReflectUtil.setFieldValue(this, field.getName(), null);
+ }
+ }
+ Class parent = clazz.getSuperclass();
+ getJsonObject(parent, jsonObject);
+ }
+
+ /**
+ * 支持存储env的key值,而具体的值存储在IENVParameter参数中
+ *
+ * @param fieldValue
+ * @return
+ */
+ protected String getENVParamter(Field field, String fieldValue) {
+ ENVDependence dependence = field.getAnnotation(ENVDependence.class);
+ if (dependence == null) {
+ return fieldValue;
+ }
+ String value = getENVVar(fieldValue);
+
+ if (StringUtil.isNotEmpty(value)) {
+ String key = MapKeyUtil.createKey(getNameSpace(), getType(), getConfigureName(), field.getName(), value);
+ this.putPrivateData(key, fieldValue);
+ return value;
+ }
+ return fieldValue;
+
+ }
+
+ protected String getENVVar(String fieldValue) {
+ if (StringUtil.isEmpty(fieldValue)) {
+ return null;
+ }
+ String value = ComponentCreator.getProperties().getProperty(fieldValue);
+ if (StringUtil.isNotEmpty(value)) {
+ return value;
+ }
+ return ENVUtile.getENVParameter(fieldValue);
+ }
+
+ private String restoreFieldValue(Field field, String fieldValueStr) {
+ return getOriFieldValue(field, fieldValueStr);
+ }
+
+ public String getOriFieldValue(Field field, String fieldValueStr) {
+ ENVDependence dependence = field.getAnnotation(ENVDependence.class);
+ if (dependence == null) {
+ return fieldValueStr;
+ }
+ String key =
+ MapKeyUtil.createKey(getNameSpace(), getType(), getConfigureName(), field.getName(), fieldValueStr);
+ String oriFieldValue = this.getPrivateData(key);
+ // if(needRemove){
+ // this.removePrivateData(key);
+ // }
+
+ if (StringUtil.isNotEmpty(oriFieldValue)) {
+ return oriFieldValue;
+ }
+ return fieldValueStr;
+ }
+
+ @Override
+ public void toObject(String jsonString) {
+ JSONObject jsonObject = JSON.parseObject(jsonString);
+ getJsonObject(jsonObject);
+ }
+
+ // @Override
+ // public String toString() {
+ // return toJson();
+ // }
+
+ public String getVersion() {
+ return version;
+ }
+
+ public void setVersion(String version) {
+ this.version = version;
+ }
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IAfterConfiguableRefreshListerner.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IAfterConfiguableRefreshListerner.java
new file mode 100644
index 0000000..7d78dff
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IAfterConfiguableRefreshListerner.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.configurable;
+
+public interface IAfterConfiguableRefreshListerner {
+
+ /**
+ * 当configurable数据全部加载完成时,调用实现这个接口的configurable对象
+ */
+ void doProcessAfterRefreshConfigurable(IConfigurableService configurableService);
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurable.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurable.java
new file mode 100644
index 0000000..79c5c05
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurable.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.configurable;
+
+import org.apache.rocketmq.streams.common.datatype.IJsonable;
+
+import java.io.Serializable;
+import java.util.Map;
+
+/**
+ * 对所有可配置的对象,做统一抽象,可配置的对象是:数据量不大,可以加载到内存,整个框架除了数据外,多数框架对象都是configuahle对象 统一抽象的好处是:可以统一的序列化,反序列化,存储,查询。 通过namespace隔离业务,通过type区分类型,namespace+type+name唯一标识一个对象。所有其他字段会被序列化到json_value中 继承BasedConfigurable 基类,会自动实现序列化和反序列化。也可以自己实现
+ */
+public interface IConfigurable extends IJsonable, IConfigurableIdentification, Serializable {
+
+ /**
+ * 把toJson的结果当作一个特殊属性
+ */
+ static final String JSON_PROPERTY = "configurable_json";
+
+ /**
+ * 把status当作configurable 的一个特殊属性
+ */
+ static final String STATUS_PROPERTY = "configurable_status";
+
+ /**
+ * 每个配置有一个独立的名字
+ *
+ * @param configureName
+ */
+ void setConfigureName(String configureName);
+
+ /**
+ * 每个配置有独立的命名空间
+ *
+ * @param nameSpace
+ */
+ void setNameSpace(String nameSpace);
+
+ /**
+ * 区分配置类型
+ *
+ * @param type
+ */
+ void setType(String type);
+
+ boolean init();
+
+ void destroy();
+
+ /**
+ * 设置私有数据
+ *
+ * @param key
+ * @param value
+ * @param <T>
+ */
+ <T> void putPrivateData(String key, T value);
+
+ /**
+ * 获取私有数据
+ *
+ * @param key
+ * @param <T>
+ * @return
+ */
+ <T> T getPrivateData(String key);
+
+ Map<String, Object> getPrivateData();
+
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurableIdentification.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurableIdentification.java
new file mode 100644
index 0000000..e82ac10
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurableIdentification.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.configurable;
+
+public interface IConfigurableIdentification {
+ /**
+ * 配置的名字,同一个配置空间名字必须唯一
+ *
+ * @return 配置项名称
+ */
+ String getConfigureName();
+
+ /**
+ * 用于区分不同的业务,用命名空间隔离业务
+ *
+ * @return 配置项命名空间
+ */
+ String getNameSpace();
+
+ /**
+ * 配置对应的type
+ *
+ * @return 配置项类型
+ */
+ String getType();
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurableListerner.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurableListerner.java
new file mode 100644
index 0000000..5911511
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurableListerner.java
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.configurable;
+
+public interface IConfigurableListerner {
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurableService.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurableService.java
new file mode 100644
index 0000000..13b7d12
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurableService.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.common.configurable;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * 对Configurable对象,做统一的管理,统一查询,插入和更新。 insert/update 把configuabel对象写入存储,支持文件存储(file),内存存储(memory)和db存储(DB)。可以在配置通过这个ConfigureFileKey.CONNECT_TYPE key 配置 query 是基于内存的查询,对象定时load到内存,可以在属性文件通过这个ConfigureFileKey.POLLING_TIME key配置加载周期,单位是秒 新对象加载后生效,已经存在的对象只有updateFlag发生变化才会被替换
+ */
+public interface IConfigurableService {
+
+ String CLASS_NAME = "className";
+
+ String CLASS_TYPE = "classType";
+
+ /**
+ * 顶层的命名空间
+ */
+ String PARENT_CHANNEL_NAME_SPACE = "rocketmq.streams.root.namespace";
+
+ /**
+ * 超顶层的命名空间
+ */
+ String ROOT_CHANNEL_NAME_SPACE = "rocketmq_streams_root_namespace";
+
+ String DEFAULT_SERVICE_NAME = "DB";
+
+ String MEMORY_SERVICE_NAME = "memory";
+
+ String FILE_SERVICE_NAME = "file";
+
+ String HTTP_SERVICE_NAME = "http";
+
+ String FILE_PATH_NAME = "filePathAndName";
+
+ /**
+ * 启动定时任务,定期从存储加载对象到内存
+ *
+ * @param namespace
+ */
+ void initConfigurables(String namespace);
+
+ /**
+ * 从存储加载对象到内存
+ *
+ * @param namespace
+ * @return
+ */
+ boolean refreshConfigurable(String namespace);
+
... 47711 lines suppressed ...