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 ...