From 9c2b4b15da741b907631866b2b94060880f6cf77 Mon Sep 17 00:00:00 2001 From: "junjie.cheng" Date: Sun, 21 Jan 2024 14:06:11 +0800 Subject: [PATCH] snapshot-2.0.1 add dispatcher module, support huawei and tencent cloud storage product and openapi --- README-chinese.md | 150 +++++ README.md | 149 ++++- SUMMARY.md | 7 - docs/README.md | 150 +++++ docs/SUMMARY.md | 8 + docs/quick_start/README.md | 46 ++ stream_sink.md => docs/stream_sink/README.md | 5 +- .../stream_source/README.md | 21 +- .../stream_transform/README.md | 0 pom.xml | 385 ++++++++---- quick_start.md | 92 ++- rocketmq-streams-channel-db/pom.xml | 9 +- .../rocketmq/streams/db/CycleSplit.java | 4 +- .../streams/db/DynamicMultipleDBSplit.java | 1 - .../rocketmq/streams/db/sink/DBSink.java | 49 +- .../streams/db/sink/DBSinkBuilder.java | 15 +- .../db/sink/DynamicMultipleDBSink.java | 19 +- .../streams/db/sink/EnhanceDBSink.java | 92 ++- .../streams/db/sink/SelfMultiTableSink.java | 14 +- .../db/sink/sqltemplate/ISqlTemplate.java | 9 +- .../MysqlInsertIgnoreIntoSqlTemplate.java | 4 +- ...InsertIntoWithDuplicateKeySqlTemplate.java | 17 +- .../sink/sqltemplate/SqlTemplateFactory.java | 8 +- .../db/source/AbstractDynamicSplitSource.java | 45 ++ .../CycleDynamicMultipleDBScanSource.java | 43 +- .../streams/db/source}/DBScanReader.java | 110 ++-- .../rocketmq/streams/db/source/DBSource.java | 108 ++++ .../source/DynamicMultipleDBScanSource.java | 44 +- .../db/sink/db/DBWriteOnlyChannelTest.java | 2 +- .../streams/db/sink/db/ISqlTemplateTest.java | 8 +- rocketmq-streams-channel-es/pom.xml | 8 +- .../streams/es/sink/ESChannelBuilder.java | 7 +- .../streams/es/sink/ESSinkOnlyChannel.java | 55 +- .../rocketmq/streams/es/sink/EsClient.java | 34 +- rocketmq-streams-channel-http/pom.xml | 2 +- .../streams/http/source/HttpSource.java | 7 +- .../http/source/server/HttpServer.java | 91 ++- .../http/source/server/HttpServerManager.java | 3 - rocketmq-streams-channel-huawei/pom.xml | 38 ++ .../streams/huawei/obs/ObsSource.java | 361 +++++++++++ .../huawei/obs/builder/ObsSourceBuilder.java | 89 +++ .../huawei/obs/iterator/ObsDataIterator.java | 237 +++++++ .../streams/huawei/obs/split/ObsSplit.java | 48 ++ .../rocketmq/streams/huawei/ObsTest.java | 137 ++++ .../rocketmq/streams/huawei/TimeTest.java | 154 +++++ rocketmq-streams-channel-kafka/pom.xml | 12 +- .../streams/kafka/KafkaChannelBuilder.java | 19 +- .../rocketmq/streams/kafka/KafkaSplit.java | 11 +- .../streams/kafka/sink/KafkaSink.java | 68 +- .../streams/kafka/source/KafkaSource.java | 201 ++++-- .../streams/kafka/KafkaChannelTest.java | 12 +- .../src/test/resources/log4j.xml | 0 .../pom.xml | 55 +- .../streams/metaq/MetaqChannelBuilder.java | 90 +++ .../rocketmq/streams/metaq/MetaqOffset.java | 109 ++++ .../streams/metaq/debug/DebugWriter.java | 114 ++++ .../metaq/queue/MetaqMessageQueue.java | 131 ++++ .../streams/metaq/sink/MetaqSink.java | 276 ++++++++ .../streams/metaq/source/MetaqSource.java | 328 ++++++++++ .../rocketmq/streams/AbstractChannelTest.java | 58 ++ rocketmq-streams-channel-mqtt/pom.xml | 15 +- .../streams/mqtt/MqttChannelBuilder.java | 10 +- .../streams/mqtt/factory/MessageProcess.java | 19 + .../mqtt/factory/MqttClientFactory.java | 211 +++++++ .../streams/mqtt/factory/MqttConnection.java | 122 ++++ .../rocketmq/streams/mqtt/sink/PahoSink.java | 98 ++- .../streams/mqtt/source/PahoSource.java | 219 +++---- rocketmq-streams-channel-openapi/.gitignore | 1 + rocketmq-streams-channel-openapi/pom.xml | 49 ++ .../openapi/OpenAPIChannelBuilder.java | 70 +++ .../rocketmq/streams/openapi/OpenAPIDim.java | 302 +++++++++ .../streams/openapi/OpenAPISQLParser.java | 64 ++ .../streams/openapi/OpenAPISource.java | 500 +++++++++++++++ .../executor/AbstractOpenAPIExecutor.java | 210 +++++++ .../openapi/executor/GenericAcsResponse.java | 55 ++ .../executor/GenericRpcAcsRequest.java | 53 ++ .../openapi/executor/HttpOpenAPIExecutor.java | 128 ++++ .../openapi/executor/OpenAPIExecutor.java | 97 +++ .../streams/openapi/splits/OpenAPISplit.java | 80 +++ .../streams/openapi/utils/ParamUtil.java | 153 +++++ .../streams/openapi/OpenAPIExecutorTest.java | 34 + .../streams/openapi/RuleEngineCenterTest.java | 67 ++ rocketmq-streams-channel-rocketmq/pom.xml | 35 +- .../streams/RocketMQChannelBuilder.java | 24 +- .../rocketmq/streams/RocketMQOffset.java | 10 +- .../{ => rocketmq}/debug/DebugWriter.java | 21 +- .../queue/RocketMQMessageQueue.java | 78 ++- .../{ => rocketmq}/sink/RocketMQSink.java | 118 ++-- .../{ => rocketmq}/source/RocketMQSource.java | 285 ++++----- .../rocketmq/streams/RocketMQChannelTest.java | 77 +-- rocketmq-streams-channel-sls/pom.xml | 51 ++ .../streams/sls/PullSlsChannelBuilder.java | 80 +++ .../streams/sls/SlsChannelBuilder.java | 37 +- .../streams/sls/SlsSearchBuilder.java | 84 +++ .../rocketmq/streams/sls/sink/SLSSink.java | 448 +++++++++++++ .../rocketmq/streams/sls/sink/SLSSplit.java | 122 ++++ .../streams/sls/source/LogHubProcessor.java | 181 ++++++ .../sls/source/LogHubProcessorFactory.java | 34 + .../streams/sls/source/SLSPullSource.java | 315 ++++++++++ .../streams/sls/source/SLSSearchSource.java | 189 ++++++ .../streams/sls/source/SLSSource.java | 400 ++++++++++++ .../sls/source/sts/StsClientWorker.java | 60 ++ .../streams/sls/source/utils/SLSUtil.java | 172 +++++ rocketmq-streams-channel-syslog/pom.xml | 10 +- .../streams/syslog/SyslogChannel.java | 34 +- .../streams/syslog/SyslogChannelManager.java | 16 +- .../rocketmq/streams/syslog/SyslogParser.java | 10 +- .../rocketmq/streams/syslog/SyslogServer.java | 79 +-- .../rocketmq/streams/syslog/SyslogClient.java | 9 +- rocketmq-streams-channel-tencent/pom.xml | 42 ++ .../streams/tencent/TencentOpenAPIClient.java | 55 ++ .../streams/tencent/TencentOpenAPISplit.java | 80 +++ .../common/TencentOpenAPIChannelBuilder.java | 81 +++ .../tencent/common/TencentOpenAPISource.java | 373 +++++++++++ .../waf/TencentWafAttackChannelBuilder.java | 76 +++ .../tencent/waf/TencentWafAttackSource.java | 377 +++++++++++ .../streams/tencent/waf/TestCommonAPI.java | 243 +++++++ .../streams/tencent/waf/TestWafAPI.java | 87 +++ rocketmq-streams-channel-view/.gitignore | 1 + rocketmq-streams-channel-view/pom.xml | 30 + .../streams/view/ViewChannelBuilder.java | 108 ++++ .../streams/db/sink/es/EsChannelTest.java | 43 ++ .../checkpoint/db/DBCheckPointStorage.java | 68 -- rocketmq-streams-clients/pom.xml | 63 +- .../client/AbstractExecutionEnvironment.java | 107 ++++ .../client/ScheduledStreamBuilder.java | 66 -- .../streams/client/ScheduledTask.java | 69 -- .../client/StreamExecutionEnvironment.java | 47 ++ .../client/source/DataStreamSource.java | 157 +++-- .../strategy/LogFingerprintStrategy.java | 15 +- .../client/strategy/ShuffleStrategy.java | 14 +- .../client/strategy/WindowStrategy.java | 28 +- .../streams/client/transform/DataStream.java | 560 +++++++++-------- .../streams/client/transform/JoinStream.java | 121 ++-- .../client/transform/OverWindowStream.java | 6 +- .../streams/client/transform/SplitStream.java | 12 +- .../client/transform/WindowStream.java | 68 +- .../streams/client/ApplicationTest.java | 57 -- .../rocketmq/streams/client/DBDriverTest.java | 74 --- .../rocketmq/streams/client/DBSinkTest.java | 17 +- .../streams/client/DataStreamTest.java | 13 +- .../rocketmq/streams/client/FileTest.java | 16 +- .../rocketmq/streams/client/FilterTest.java | 26 +- .../rocketmq/streams/client/JoinTest.java | 43 +- .../rocketmq/streams/client/LeaseTest.java | 47 +- .../rocketmq/streams/client/MQTTTest.java | 43 ++ .../streams/client/MqttSourceExample.java | 17 +- .../rocketmq/streams/client/ORMUtilTest.java | 53 +- .../streams/client/RocketMQGroupByTest.java | 3 +- .../rocketmq/streams/client/RocketMQTest.java | 29 +- .../rocketmq/streams/client/ScriptTest.java | 12 +- .../streams/client/SourceMergerTest.java | 71 +++ .../rocketmq/streams/client/SourceTest.java | 9 +- .../rocketmq/streams/client/SplitTest.java | 6 +- .../rocketmq/streams/client/UnionTest.java | 31 +- .../streams/client/UserDefinedSourceTest.java | 1 - .../rocketmq/streams/client/WindowTest.java | 261 ++++---- .../client/example/CountFromFileTest.java | 23 +- .../client/example/CountSupportEmitTest.java | 12 +- .../streams/client/example/DimTest.java | 6 +- .../streams/client/example/JoinTest.java | 12 +- .../streams/client/example/SplitTest.java | 6 +- .../streams/client/model/JobModel.java | 12 +- .../streams/client/model/RuleContainer.java | 51 ++ .../streams/client/model/TestRule.java | 26 + .../client/source/MqttStreamsTest.java | 4 +- .../client/source/UserDefinedSource.java | 10 +- .../windows/AbstractWindowFireModeTest.java | 99 +-- .../client/windows/AbstractWindowTest.java | 4 +- .../client/windows/MultiSplitTest.java | 8 +- .../client/windows/ShuffleOverWindowTest.java | 4 +- .../client/windows/SimpleWindowTest.java | 54 ++ .../client/windows/SingleSplitTest.java | 13 +- .../windows/WindowFireResultCompareTest.java | 149 +++-- .../client/windows/WindowMsgManager.java | 51 +- .../src/test/resources/log4j.xml | 0 rocketmq-streams-commons/pom.xml | 113 ++-- .../batchsystem/BatchFinishMessage.java | 30 + .../streams/common/cache/FileBasedTable.java | 54 +- .../common/cache/HybridByteBufferTable.java | 72 +-- .../common/cache/MappedByteBufferTable.java | 34 +- ...MappedByteBufferTableWithPrimaryIndex.java | 482 -------------- .../cache/compress/AbstractMemoryTable.java | 118 ++-- .../cache/compress/impl/IntValueKV.java | 185 ------ .../cache/compress/impl/StringValueKV.java | 136 ---- .../impl/SoftReferenceCache.java | 39 +- .../common/calssscaner/AbstractScan.java | 162 +++-- .../common/channel/AbstractChannel.java | 45 +- .../streams/common/channel/IChannel.java | 7 +- .../builder/AbstractChannelSQLParser.java | 4 +- ...ractChannelSQLSupportShuffleSQLParser.java | 5 +- .../AbstractSupportShuffleChannelBuilder.java | 2 +- .../common/channel/impl/CollectionSource.java | 36 +- .../channel/impl/file/FileChannelBuilder.java | 7 +- .../common/channel/impl/file/FileSink.java | 29 +- .../common/channel/impl/file/FileSource.java | 203 ++++-- .../common/channel/impl/file/FileSplit.java | 19 +- .../channel/impl/memory/MemoryChannel.java | 83 --- .../impl/memory/MemoryChannelBuilder.java | 4 +- .../channel/impl/memory/MemorySink.java | 26 +- .../channel/impl/memory/MemorySource.java | 67 +- .../channel/impl/memory/MemorySplit.java | 6 +- .../impl/memory/RowListChannelBuilder.java | 87 +++ .../channel/impl/memory/RowListSource.java | 51 ++ .../common/channel/impl/view/ViewSink.java | 25 +- .../common/channel/impl/view/ViewSource.java | 33 +- .../common/channel/sink/AbstractSink.java | 52 +- .../sink/AbstractSupportShuffleSink.java | 24 +- .../common/channel/sink/AbstractUDFSink.java | 8 +- .../streams/common/channel/sink/ISink.java | 44 +- .../sinkcache/DataSourceAutoFlushTask.java | 15 +- .../channel/sinkcache/IMessageCache.java | 12 +- .../impl/AbstractMultiSplitMessageCache.java | 24 +- .../channel/sinkcache/impl/MessageCache.java | 46 +- .../channel/source/AbstractPushSource.java | 27 + ...ce.java => AbstractSingleSplitSource.java} | 92 +-- .../common/channel/source/AbstractSource.java | 457 +++++++------- .../source/AbstractUnreliableSource.java | 62 +- .../common/channel/source/ISource.java | 38 +- .../channel/source/ISplitChangedListener.java | 22 +- .../common/channel/source/SplitProgress.java | 68 ++ .../systemmsg/ChangeTableNameMessage.java | 8 +- .../source/systemmsg/NewSplitMessage.java | 8 + .../source/systemmsg/RemoveSplitMessage.java | 8 + .../source/systemmsg/SplitChangedMessage.java | 5 +- .../systemmsg/WaterMarkNotifyMessage.java | 35 ++ .../common/channel/split/CommonSplit.java | 42 ++ .../common/channel/split/DynamicSplit.java | 35 ++ .../checkpoint/AbstractCheckPointStorage.java | 62 +- .../streams/common/checkpoint/CheckPoint.java | 14 +- .../common/checkpoint/CheckPointManager.java | 131 ++-- .../common/checkpoint/CheckPointMessage.java | 5 +- .../checkpoint/CheckPointStorageFactory.java | 6 +- .../common/checkpoint/ICheckPointStorage.java | 33 +- .../common/checkpoint/ISplitOffset.java | 41 ++ .../common/checkpoint/SourceSnapShot.java | 25 +- .../common/compiler/CustomJavaCompiler.java | 56 +- .../common/component/AbstractComponent.java | 82 +-- .../common/component/ComponentCreator.java | 157 ++--- .../common/component/ConfigureDescriptor.java | 2 +- .../PropertyConfigureDescriptorManager.java | 10 +- .../PropertyConfigureDiscriptorManager.java | 10 +- .../configurable/AbstractConfigurable.java | 187 ++---- .../configurable/BasedConfigurable.java | 212 ++----- .../streams/common/configurable/ENVField.java | 53 ++ .../common/configurable/IConfigurable.java | 47 +- .../IConfigurableIdentification.java | 2 +- .../configurable/IConfigurableQuery.java | 46 ++ .../configurable/IConfigurableService.java | 139 ---- .../annotation/ConfigurableReference.java | 27 + .../configuration/ConfigurationKey.java | 157 +++++ .../configuration/EmitInsertConfig.java | 47 ++ .../configuration/JobConfiguration.java | 109 ++++ .../common/configuration/SystemContext.java | 136 ++++ .../common/configure/ConfigureFileKey.java | 171 ----- .../common/configure/StreamsConfigure.java | 59 -- .../streams/common/constant/State.java | 24 + .../common/context/AbstractContext.java | 178 +----- .../common/context/BatchMessageOffset.java | 18 +- .../streams/common/context/IMessage.java | 1 - .../streams/common/context/Message.java | 47 +- .../streams/common/context/MessageHeader.java | 159 +++-- .../streams/common/context/MessageOffset.java | 133 ++-- .../common/context/UserDefinedMessage.java | 63 +- .../common/datatype/ArrayDataType.java | 30 +- .../streams/common/datatype/BaseDataType.java | 20 +- .../common/datatype/BooleanDataType.java | 14 +- .../streams/common/datatype/ByteDataType.java | 8 +- .../common/datatype/ConfigurableDataType.java | 16 +- .../streams/common/datatype/DateDataType.java | 72 +-- .../common/datatype/DateTimeDataType.java | 10 +- .../common/datatype/DoubleDataType.java | 16 +- .../common/datatype/FloatDataType.java | 48 +- .../datatype/GenericParamterDataType.java | 2 +- .../streams/common/datatype/HllDataType.java | 18 +- .../streams/common/datatype/IJsonable.java | 4 +- .../streams/common/datatype/IntDataType.java | 10 +- .../common/datatype/JavaBeanDataType.java | 186 +++--- .../common/datatype/JsonableDataType.java | 19 +- .../streams/common/datatype/ListDataType.java | 22 +- .../streams/common/datatype/LongDataType.java | 28 +- .../streams/common/datatype/MapDataType.java | 32 +- .../streams/common/datatype/SetDataType.java | 12 +- .../common/datatype/ShortDataType.java | 10 +- .../common/datatype/StringDataType.java | 8 +- .../streams/common/dboperator/IDBDriver.java | 20 +- .../common/disruptor/DisruptorProducer.java | 6 +- .../streams/common/enums/StageType.java | 52 ++ .../exception/CanNotSupportThisMethod.java | 33 + .../common/functions/FlatMapFunction.java | 2 +- .../streams/common/functions/Function.java | 3 +- .../common/functions/ReduceFunction.java | 2 +- .../interfaces/IBaseStreamOperator.java | 4 +- .../IConfigurableLifeStart.java} | 6 +- .../IConfigurablePropertySetter.java | 23 + .../streams/common/interfaces/IDim.java | 43 ++ .../common/interfaces/IFilterService.java | 10 +- .../common/interfaces/INeedAalignWaiting.java | 26 + .../common/interfaces/IPropertyEnable.java | 6 +- .../streams/common/interfaces/ISerialize.java | 1 - .../streams/common/interfaces/IStage.java | 25 + .../common/interfaces/IStreamOperator.java | 2 +- .../common/interfaces/ISystemMessage.java | 8 + .../interfaces/ISystemMessageProcessor.java | 58 +- .../ITaskStart.java} | 6 +- .../streams/common/interfaces/IUDF.java | 8 +- .../common/logger/LoggerOutputChannel.java | 2 +- .../logger/MyDailyRollingFileAppender.java | 110 ++-- .../common/metadata/AbstractMetaData.java | 78 +-- .../common/metadata/LogicMetaData.java | 12 +- .../streams/common/metadata/MetaData.java | 93 ++- .../common/metadata/MetaDataField.java | 73 ++- .../common/metadata/MetaDataUtils.java | 4 +- .../streams/common/model/JobConfigure.java | 62 ++ .../streams/common/model/JobStage.java | 109 ++++ .../streams/common/model/NameCreator.java | 25 +- .../common/model/NameCreatorContext.java | 44 +- .../rocketmq/streams/common/model/Pair.java | 122 ++++ .../model/SQLCompileContextForSource.java | 23 +- .../streams/common/model/ServiceName.java | 2 - .../streams/common/model/StageRelation.java | 112 ++++ .../common/monitor/ConsoleMonitorManager.java | 91 ++- .../common/monitor/DataSyncConstants.java | 30 - .../streams/common/monitor/HttpUtil.java | 78 ++- .../streams/common/monitor/IMonitor.java | 2 +- .../MonitorDataSyncServiceFactory.java | 13 +- .../common/monitor/MonitorFactory.java | 48 +- .../monitor/group/MonitorCommander.java | 17 +- .../common/monitor/impl/DipperMonitor.java | 215 +++---- .../common/monitor/impl/MonitorItem.java | 7 +- .../monitor/impl/NothingMonitorItem.java | 4 +- .../common/monitor/model/JobStage.java | 94 +-- .../common/monitor/model/TraceIdsDO.java | 32 +- .../common/monitor/model/TraceMonitorDO.java | 50 +- .../service/impl/HttpMonitorDataSyncImpl.java | 8 +- .../impl/RocketMQMonitorDataSyncImpl.java | 46 +- .../optimization/FilterResultCache.java | 21 +- .../common/optimization/HomologousVar.java | 3 +- .../optimization/IHomologousCalculate.java | 24 + .../optimization/IHomologousOptimization.java | 14 +- .../common/optimization/LikeRegex.java | 102 +-- ...sageGlobleTrace.java => MessageTrace.java} | 36 +- .../optimization/OptimizationRegex.java | 9 +- .../common/optimization/RegexEngine.java | 17 +- .../common/optimization/TaskOptimization.java | 71 --- .../fingerprint/FingerprintCache.java | 163 ++--- .../fingerprint/FingerprintMetric.java | 53 +- .../fingerprint/PreFingerprint.java | 38 +- .../common/schedule/ScheduleManager.java | 74 --- .../streams/common/schedule/ScheduleTask.java | 101 --- .../common/threadpool/ScheduleFactory.java | 43 ++ .../common/threadpool/ThreadPoolFactory.java | 58 +- .../AbstractMutilPipelineChainPipline.java | 225 ------- .../common/topology/ChainPipeline.java | 594 ------------------ .../streams/common/topology/IJobGraph.java | 13 + .../topology/{model => }/ILifeCycle.java | 2 +- .../topology/{stages/udf => }/IReducer.java | 2 +- .../{shuffle => }/IShuffleKeyGenerator.java | 2 +- .../topology/{builder => }/IStageBuilder.java | 4 +- .../common/topology/ISynchronousWindow.java | 25 + .../common/topology/{model => }/IWindow.java | 62 +- .../{task => assigner}/TaskAssigner.java | 13 +- .../topology/builder/PipelineBuilder.java | 255 ++++---- .../common/topology/graph/JobGraph.java | 95 +++ .../common/topology/metric/NotFireReason.java | 136 ++-- .../common/topology/metric/SourceMetric.java | 181 ++++++ .../common/topology/metric/StageGroup.java | 156 ++--- .../common/topology/metric/StageMetric.java | 64 +- .../AbstractChainStage.java} | 11 +- .../topology/model/AbstractPipeline.java | 165 +++++ .../common/topology/model/AbstractRule.java | 32 +- .../common/topology/model/AbstractScript.java | 19 +- .../common/topology/model/AbstractStage.java | 250 ++++++-- ...geHandle.java => AbstractStageHandle.java} | 9 +- .../common/topology/model/ChainPipeline.java | 516 +++++++++++++++ .../common/topology/model/JobGraphCopy.java | 248 ++++++++ .../topology/model/MessageFinishCallBack.java | 103 +++ .../common/topology/model/Pipeline.java | 345 ---------- .../topology/{ => model}/SectionPipeline.java | 30 +- .../common/topology/model/StreamGraph.java | 156 +++++ .../streams/common/topology/model/Union.java | 7 +- .../topology/shuffle/ShuffleMQCreator.java | 406 ------------ .../stages/AbstractStatelessChainStage.java | 18 +- .../topology/stages/AbstractWindowStage.java | 98 --- .../common/topology/stages/DimChainStage.java | 180 ++++++ .../topology/stages/EmptyChainStage.java | 18 +- .../topology/stages/FilterChainStage.java | 194 +----- .../topology/stages/JoinChainStage.java | 161 ----- .../topology/stages/JoinStartChainStage.java | 55 +- .../topology/stages/NewSQLChainStage.java | 50 +- .../topology/stages/OpenAPIChainStage.java | 47 +- .../topology/stages/OutputChainStage.java | 171 ++--- .../topology/stages/PythonChainStage.java | 47 +- .../topology/stages/RightJoinChainStage.java | 92 --- .../common/topology/stages/SQLChainStage.java | 80 --- .../topology/stages/ScriptChainStage.java | 89 +-- ...ChainStage.java => ShuffleChainStage.java} | 42 +- .../stages/ShuffleConsumerChainStage.java | 193 ------ .../stages/ShuffleProducerChainStage.java | 345 ---------- ....java => SynchronousWindowChainStage.java} | 44 +- .../topology/stages/UnionStartChainStage.java | 15 +- .../topology/stages/ViewChainStage.java | 529 ---------------- .../topology/stages/udf/MapOperator.java | 8 +- .../topology/stages/udf/StageBuilder.java | 43 +- .../topology/stages/udf/UDFChainStage.java | 33 +- .../stages/udf/UDFUnionChainStage.java | 52 +- .../common/topology/task/StreamsTask.java | 160 ----- .../streams/common/utils/AESUtil.java | 9 +- .../streams/common/utils/ClassUtil.java | 20 +- .../streams/common/utils/CollectionUtil.java | 20 + .../common/utils/ConfigurableUtil.java | 76 +-- .../streams/common/utils/ContantsUtil.java | 66 +- .../utils/{DESUtils.java => DESUtil.java} | 6 +- .../streams/common/utils/DataTypeUtil.java | 56 +- .../streams/common/utils/DateUtil.java | 20 +- .../common/utils/DipperThreadLocalUtil.java | 2 +- .../utils/{ENVUtile.java => ENVUtil.java} | 6 +- .../streams/common/utils/FileUtil.java | 39 +- .../rocketmq/streams/common/utils/IOUtil.java | 2 +- .../rocketmq/streams/common/utils/IPUtil.java | 24 +- .../rocketmq/streams/common/utils/IdUtil.java | 53 ++ .../common/utils/InstantiationUtil.java | 36 +- .../{Ip2LongUtils.java => Ip2LongUtil.java} | 52 +- .../streams/common/utils/JarUtil.java | 2 +- .../streams/common/utils/JsonableUtil.java | 4 +- .../streams/common/utils/KryoUtil.java | 65 +- .../streams/common/utils/LogParserUtil.java | 18 +- .../streams/common/utils/MapKeyUtil.java | 29 +- .../streams/common/utils/MessageUtil.java | 12 +- .../common/utils/PipelineHTMLUtil.java | 446 +++++++------ .../streams/common/utils/PrintUtil.java | 103 ++- ...opertiesUtils.java => PropertiesUtil.java} | 57 +- .../streams/common/utils/RandomStrUtil.java | 6 +- .../streams/common/utils/ReflectUtil.java | 160 ++--- .../streams/common/utils/RocketmqUtil.java | 352 +++++++++++ .../streams/common/utils/RuntimeUtil.java | 4 +- .../common/utils/SQLFormatterUtil.java | 43 +- .../streams/common/utils/SQLUtil.java | 67 +- .../streams/common/utils/ScheduleUtil.java | 6 +- .../streams/common/utils/SerializeUtil.java | 22 +- .../streams/common/utils/ServiceLoadUtil.java | 19 +- .../streams/common/utils/ShellUtil.java | 11 +- .../streams/common/utils/StringUtil.java | 35 +- .../streams/common/utils/TraceUtil.java | 13 +- .../streams/common/MessasgeCacheTest.java | 4 +- .../cache/compress/FingerprintCacheTest.java | 33 + .../compress/HybridByteBufferTableTest.java | 7 +- .../cache/compress/NumberUtilsTest.java | 7 +- .../streams/common/channel/SourceTest.java | 2 +- .../streams/common/datatype/DataTypeTest.java | 4 +- .../streams/common/regex/RegexTest.java | 10 +- .../streams/common/serializa/CountAccum.java | 10 +- .../streams/common/serializa/Person.java | 3 +- .../common/stages/SelfChainStageTest.java | 71 ++- .../configurable/ConfigurableComponent.java | 184 ------ .../streams/configurable/model/Configure.java | 126 ---- .../service/AbstractConfigurableService.java | 510 --------------- ...AbstractSupportParentConfigureService.java | 198 ------ .../service/ConfigurableServiceFactory.java | 56 -- .../service/impl/FileConfigureService.java | 266 -------- .../FileSupportParentConfigureService.java | 36 -- .../service/impl/HttpConfigureService.java | 377 ----------- .../HttpSupportParentConfigureService.java | 44 -- .../service/impl/MemoryConfigureService.java | 120 ---- .../MemorySupportParentConfigureService.java | 36 -- .../configuable/ConfiguableComponentTest.java | 112 ---- .../streams/configuable/model/Person.java | 109 ---- .../ConfigurableComponentTest.java | 107 ---- .../streams/configurable/model/Person.java | 109 ---- .../src/test/resources/log4j.xml | 36 -- rocketmq-streams-connectors/pom.xml | 9 +- .../streams/connectors/IBoundedSource.java | 3 +- .../connectors/balance/ISourceBalance.java | 9 +- .../connectors/balance/SplitChanged.java | 7 +- .../balance/impl/LeaseBalanceImpl.java | 11 +- .../streams/connectors/model/PullMessage.java | 7 +- .../streams/connectors/model/QuerySplit.java | 91 +++ .../connectors/model/ReaderStatus.java | 60 +- .../reader/AbstractFlinkSplitReader.java | 19 +- .../reader/AbstractQueryReader.java | 68 ++ .../reader/AbstractSplitReader.java | 63 ++ .../connectors/reader/ISplitReader.java | 30 +- .../connectors/source/AbstractPullSource.java | 470 ++++++++------ .../source/AbstractQuerySource.java | 84 +++ .../connectors/source/IPullSource.java | 16 +- .../source/MutilBatchTaskSource.java | 158 ----- .../connectors/source/SourceInstance.java | 5 +- .../source/client/SplitConsumer.java | 162 +++++ .../source/filter/AbstractPatternFilter.java | 1 - .../source/filter/BoundedPatternFilter.java | 10 +- .../source/filter/CyclePatternFilter.java | 91 ++- .../connectors/source/filter/CyclePeriod.java | 75 ++- .../source/filter/CycleSchedule.java | 42 +- .../source/filter/CycleScheduleFilter.java | 2 +- .../filter/DataFormatPatternFilter.java | 29 +- .../source/filter/PatternFilter.java | 3 +- .../source/impl/FilesPullSource.java | 179 ++++++ .../source/impl/NonHomologousPullSource.java | 108 ++++ .../streams/connector/FileSourceTest.java | 90 +++ .../src/test/resources/log4j.xml | 0 .../pom.xml | 10 +- .../db/init}/mysql/delegate/DBDelegate.java | 7 +- .../mysql/delegate/DBDelegateFactory.java | 7 +- .../db/init}/mysql/delegate/DBType.java | 2 +- .../init}/mysql/delegate/MysqlDelegate.java | 16 +- .../main/resources/tables_mysql_innodb.sql | 227 +++++++ rocketmq-streams-db-operator/pom.xml | 8 +- .../db/configuable/DBConfigureService.java | 275 -------- .../DBSupportParentConfigureService.java | 36 -- .../streams/db/driver/DriverBuilder.java | 35 +- .../streams/db/driver/JDBCDriver.java | 182 +++--- .../db/driver/batchloader/BatchRowLoader.java | 104 +-- .../streams/db/driver/orm/ORMUtil.java | 122 ++-- .../streams/db/operator/SQLOperator.java | 17 +- .../apache/rocketmq/streams/db/Person.java | 4 +- .../DBSupportParentConfigureServiceTest.java | 74 --- .../streams/db/driver/orm/ORMUtilTest.java | 12 +- .../main/resources/tables_mysql_innodb.sql | 201 ------ rocketmq-streams-dim/pom.xml | 5 +- .../rocketmq/streams/dim/DimComponent.java | 62 -- .../dim/builder/AbstractDimParser.java | 7 +- .../streams/dim/builder/DimBuilder.java | 19 +- .../streams/dim/builder/FileDimSQLParser.java | 12 +- .../streams/dim/builder/IDimSQLParser.java | 4 +- .../expression/InExpressionResource.java | 78 --- .../expression/NotInExpressionResource.java | 44 -- .../function/script/IntelligenceFunction.java | 64 +- .../dim/function/script/NameListFunction.java | 202 ------ .../rocketmq/streams/dim/index/DimIndex.java | 47 +- .../streams/dim/index/IndexExecutor.java | 40 +- .../AbstractIntelligenceCache.java | 260 +++----- .../AccountIntelligenceCache.java | 9 +- .../intelligence/DomainIntelligenceCache.java | 21 +- .../dim/intelligence/IPIntelligenceCache.java | 48 +- .../intelligence/URLIntelligenceCache.java | 15 +- .../streams/dim/model/AbstractDim.java | 376 ++++++----- .../dim/model/AbstractProcShareDim.java | 61 +- .../rocketmq/streams/dim/model/DBDim.java | 107 ++-- .../rocketmq/streams/dim/model/FileDim.java | 18 +- .../streams/dim/model/IDataCache.java | 7 +- .../streams/dim/model/IDimSource.java | 7 +- .../dim/service/impl/DimServiceImpl.java | 21 +- .../java/com/aliyun/service/DBDimTest.java | 68 ++ .../aliyun/service/NameListFunctionTest.java | 4 +- rocketmq-streams-dispatcher/pom.xml | 25 + .../streams/dispatcher/IBalanceCallback.java | 31 + .../rocketmq/streams/dispatcher/ICache.java | 31 + .../streams/dispatcher/IDispatcher.java | 26 + .../dispatcher/IDispatcherCallback.java | 51 ++ .../rocketmq/streams/dispatcher/IMapper.java | 83 +++ .../streams/dispatcher/IStrategy.java | 26 + .../streams/dispatcher/IStrategyCallBack.java | 45 ++ .../streams/dispatcher/cache/DBCache.java | 175 ++++++ .../dispatcher/cache/RocketmqCache.java | 57 ++ .../callback/DefaultDispatcherCallback.java | 52 ++ .../constant/DispatcherConstant.java | 21 + .../dispatcher/entity/DispatcherMapper.java | 180 ++++++ .../dispatcher/enums/DispatchMode.java | 43 ++ .../dispatcher/impl/LeaseDispatcher.java | 238 +++++++ .../dispatcher/impl/RocketmqDispatcher.java | 362 +++++++++++ .../dispatcher/strategy/AllStrategy.java | 50 ++ .../strategy/AveragelyStrategy.java | 68 ++ .../strategy/ConsistencyStrategy.java | 81 +++ .../dispatcher/strategy/HashStrategy.java | 50 ++ .../dispatcher/strategy/LeastStrategy.java | 204 ++++++ .../dispatcher/strategy/StrategyFactory.java | 29 + rocketmq-streams-examples/README.md | 204 ++++++ rocketmq-streams-examples/pom.xml | 14 +- .../rocketmq/streams/examples/Test.java | 45 ++ .../aggregate/PageDimensionExample.java | 46 +- .../aggregate/RocketMQWindowExample.java | 50 +- .../aggregate/UsersDimensionExample.java | 32 +- .../checkpoint/RemoteCheckpointExample.java | 69 +- .../examples/join/RocketmqDimJoinExample.java | 4 +- .../examples/join/RocketmqJoinExample.java | 6 +- .../mutilconsumer/MultiStreamsExample.java | 36 +- .../examples/mutilconsumer/Producer.java | 8 +- .../examples/sink/MqttSinkExample.java | 4 +- .../examples/source/FileSourceExample.java | 6 +- .../examples/source/MqttSourceExample.java | 4 +- .../source/RocketmqSourceExample1.java | 15 +- .../source/RocketmqSourceExample2.java | 4 +- .../source/RocketmqSourceExample3.java | 4 +- rocketmq-streams-filter/pom.xml | 4 +- .../streams/filter/FilterComponent.java | 25 +- .../filter/builder/ExpressionBuilder.java | 90 ++- .../streams/filter/builder/RuleBuilder.java | 100 +-- .../filter/builder/RuleElementBuilder.java | 17 +- .../filter/context/ContextConfigure.java | 34 +- .../streams/filter/context/RuleContext.java | 189 +----- .../streams/filter/engine/SplitFlow.java | 22 +- .../filter/engine/impl/DefaultRuleEngine.java | 34 +- .../AbstractExpressionFunction.java | 11 +- .../function/expression/CompareFunction.java | 37 +- .../ContainsCaseInsensitiveFunction.java | 10 +- .../function/expression/ContainsFunction.java | 2 +- .../filter/function/expression/Equals.java | 48 +- .../function/expression/GreaterEquals.java | 2 +- .../function/expression/GreaterThan.java | 10 +- .../function/expression/InFunction.java | 6 +- .../function/expression/InMetaData.java | 2 +- .../expression/IpContainsFunction.java | 10 +- .../filter/function/expression/IsNotNull.java | 6 +- .../filter/function/expression/IsNull.java | 7 +- .../function/expression/LessEquals.java | 2 +- .../filter/function/expression/LessThan.java | 3 +- .../function/expression/LikeFunction.java | 118 ++-- .../NotContainsCaseInsensitiveFunction.java | 10 +- .../expression/NotContainsFunction.java | 2 +- .../filter/function/expression/NotEquals.java | 16 +- .../function/expression/NotInFunction.java | 2 +- .../function/expression/NotLikeFunction.java | 2 +- .../NotRegexCaseInsensitiveFunction.java | 2 +- .../function/expression/NotRegexFunction.java | 2 +- .../RegexCaseInsensitiveFunction.java | 2 +- .../function/expression/RegexFunction.java | 14 +- .../function/expression/ScriptFunction.java | 2 +- .../filter/function/script/CaseFunction.java | 3 + ...rser.java => GroupScriptOptimization.java} | 35 +- .../filter/monitor/rule/MessageMonitor.java | 4 +- .../filter/operator/FilterOperator.java | 5 +- .../streams/filter/operator/Rule.java | 220 ++----- .../filter/operator/action/Action.java | 1 + .../operator/action/IConfigurableAction.java | 3 +- .../operator/action/impl/SinkAction.java | 26 +- .../operator/expression/Expression.java | 150 ++--- .../expression/ExpressionPerformance.java | 52 +- .../expression/ExpressionRelationParser.java | 6 +- .../operator/expression/GroupExpression.java | 34 +- .../expression/GroupExpressionManager.java | 2 +- .../expression/RelationExpression.java | 133 ++-- .../operator/expression/SimpleExpression.java | 1 + .../filter/operator/var/ConstantVar.java | 7 +- .../filter/operator/var/ContextVar.java | 9 +- .../streams/filter/operator/var/InnerVar.java | 2 +- .../streams/filter/operator/var/Var.java | 5 - .../optimization/ExpressionOptimization.java | 22 +- .../PiplineLogFingerprintAnalysis.java | 46 +- .../casewhen/AbstractWhenExpression.java | 152 +++-- .../casewhen/CaseWhenBuilder.java | 38 +- .../casewhen/CaseWhenElement.java | 132 ++-- .../casewhen/GroupByVarCaseWhen.java | 171 ++--- .../casewhen/IFCaseWhenExpression.java | 121 ++++ .../casewhen/IFCaseWhenExpressionGroup.java | 179 ++++++ .../casewhen/IFExpressionOptimization.java | 78 +++ .../casewhen/MutilCaseWhenExpression.java | 13 +- .../casewhen/SingleCaseWhenExpression.java | 76 +-- .../dependency/BlinkRuleV2Expression.java | 132 ++-- .../dependency/DependencyTree.java | 44 +- .../dependency/EmptyTreeNode.java | 44 ++ .../dependency/FilterTreeNode.java | 16 +- .../optimization/dependency/PipelineTree.java | 10 +- .../dependency/ScriptDependent.java | 103 +-- .../dependency/ScriptTreeNode.java | 2 +- .../dependency/SimplePipelineTree.java | 12 +- .../dependency/StateLessDependencyTree.java | 23 +- .../optimization/dependency/TreeNode.java | 113 ++-- .../executor/AbstractExecutor.java | 3 +- .../executor/GroupByVarExecutor.java | 190 +++--- .../homologous/HomologousCompute.java | 29 +- .../homologous/HomologousOptimization.java | 108 +++- .../result/GroupQuickFilterResult.java | 48 +- .../script/ScriptOptimization.java | 23 +- .../filter/service/IRuleEngineService.java | 1 - .../service/impl/RuleEngineServiceImpl.java | 12 +- .../rocketmq/streams/filter/utils/IPUtil.java | 24 +- .../streams/filter/utils/RegexUtil.java | 111 ++-- .../streams/filter/FilterComponentTest.java | 2 +- .../streams/filter/operator/RuleTest.java | 39 -- .../src/test/resources/log4j.xml | 0 rocketmq-streams-lease/pom.xml | 10 +- .../streams/lease/LeaseComponent.java | 31 +- .../streams/lease/service/ILeaseService.java | 33 +- .../streams/lease/service/ILeaseStorage.java | 8 + .../streams/lease/service/ILeaseStorasge.java | 10 +- ...sedLesaseImpl.java => BasedLeaseImpl.java} | 178 +++--- .../lease/service/impl/LeaseServiceImpl.java | 68 +- .../lease/service/impl/MockLeaseImpl.java | 5 + .../service/storages/DBLeaseStorage.java | 24 +- .../streams/lease/LeaseComponentTest.java | 24 +- .../src/test/resources/log4j.xml | 0 .../pom.xml | 19 +- .../common/cache/compress/AdditionStore.java | 60 +- .../common/cache/compress/BigMapAddress.java | 0 .../common/cache/compress/BitSetCache.java | 106 ++-- .../common/cache/compress/ByteArray.java | 0 .../cache/compress/ByteArrayValueKV.java | 0 .../common/cache/compress/ByteStore.java | 46 +- .../common/cache/compress/CacheKV.java | 18 +- .../common/cache/compress/ICacheKV.java | 0 .../common/cache/compress/KVAddress.java | 90 +-- .../common/cache/compress/KVElement.java | 20 +- .../cache/compress/MappedByteBufferStore.java | 0 .../cache/compress/impl/AbstractListKV.java | 4 +- .../cache/compress/impl/IntValueKV.java | 72 +++ .../common/cache/compress/impl/KeySet.java | 48 +- .../cache/compress/impl/LightCache.java | 51 ++ .../cache/compress/impl/LongListKV.java | 20 - .../cache/compress/impl/LongValueKV.java | 0 .../cache/compress/impl/MapAddressListKV.java | 30 +- .../cache/compress/impl/MultiValueKV.java | 0 .../cache/compress/impl/MutilValueKV.java | 0 .../cache/compress/impl/SplitCache.java | 0 .../cache/compress/impl/StringValueKV.java | 75 +++ .../streams/common/utils/NumberUtils.java | 393 ++++++++++++ .../cache/compress/FingerprintCacheTest.java | 33 + .../common/cache/compress/IntValueKVTest.java | 66 ++ .../src/test/resources/log4j.xml | 0 .../assembly/distribution.xml | 69 -- .../assembly/standalone.xml | 72 --- rocketmq-streams-runner/bin/start.sh | 58 -- rocketmq-streams-runner/bin/stop.sh | 33 - rocketmq-streams-runner/log/catalina.out | 0 rocketmq-streams-runner/pom.xml | 66 -- .../src/main/resources/log4j.xml | 51 -- rocketmq-streams-schedule/pom.xml | 22 +- .../streams/schedule/ScheduleComponent.java | 16 +- .../schedule/job/ConfigurableExecutorJob.java | 9 - .../service/impl/ScheduleServiceImpl.java | 80 ++- .../schedule/ScheduleComponentTest.java | 2 +- .../src/test/resources/log4j.xml | 0 rocketmq-streams-script/pom.xml | 10 +- .../streams/script/ScriptComponent.java | 11 +- .../script/context/FunctionContext.java | 10 +- .../ScriptExpresionRuntimeExpression.java | 23 + .../aggregation/AverageAccumulator.java | 29 +- .../aggregation/ConcatAccumulator.java | 20 +- .../ConcatDistinctAccumulator.java | 21 +- .../aggregation/CountAccumulator.java | 8 +- .../aggregation/CountDistinctAccumulator.java | 3 +- .../aggregation/DistinctAccumulator.java | 9 +- .../aggregation/DistinctAccumulator2.java | 10 +- .../aggregation/FirstValueAccumulator.java | 8 +- .../aggregation/LastValueAccumulator.java | 8 +- .../function/aggregation/MaxAccumulator.java | 14 +- .../function/aggregation/MinAccumulator.java | 16 +- .../function/aggregation/SumAccumulator.java | 12 +- .../function/impl/bool/BooleanFunction.java | 2 +- .../function/impl/common/EchoFunction.java | 12 +- .../function/impl/common/EmptyFunction.java | 29 + .../impl/condition/EqualsFunction.java | 43 +- .../condition/ExecuteByConditionFuction.java | 4 +- .../impl/condition/GreateEqualsFunction.java | 12 +- .../impl/condition/GreaterFunction.java | 7 +- .../function/impl/condition/IFFunction.java | 3 +- .../impl/condition/InConditionFunction.java | 14 +- .../impl/condition/LessEqualsFunction.java | 8 +- .../function/impl/condition/LessFunction.java | 4 +- .../function/impl/date/AddMonthFunction.java | 10 +- .../impl/date/CurrentTimestampFunction.java | 46 +- .../function/impl/date/DateAddFunction.java | 16 +- .../function/impl/date/DateDiffFunction.java | 16 +- .../impl/date/DateFormatFunction.java | 25 +- .../function/impl/date/DatePartFunction.java | 52 +- .../impl/date/DateToCharFunction.java | 8 +- .../function/impl/date/DateTruncFunction.java | 22 +- .../function/impl/date/DateUtilFunction.java | 46 +- .../function/impl/date/DateUtileFunction.java | 17 +- .../function/impl/date/DayFunction.java | 4 +- .../impl/date/FromUnixTimeFunction.java | 18 +- .../function/impl/date/IsDateFunction.java | 6 +- .../function/impl/date/LastDayFunction.java | 6 +- .../function/impl/date/QuarterFuction.java | 1 - .../function/impl/date/QuarterFunction.java | 6 +- .../impl/date/UnixTimeStampFunction.java | 6 +- .../impl/date/WeekOfYearFunction.java | 6 +- .../function/impl/date/WeekdayFunction.java | 6 +- .../script/function/impl/env/ENVFunction.java | 47 -- .../function/impl/eval/EvalFunction.java | 27 +- .../impl/field/AdditionalFiledFunction.java | 50 +- .../function/impl/field/CoalesceFunction.java | 20 +- .../impl/field/ExtraFieldFunction.java | 8 +- .../function/impl/field/FieldFunction.java | 23 + .../impl/field/RemoveFieldFunction.java | 14 +- .../impl/field/RetainFieldFunction.java | 2 +- .../function/impl/filter/BreakFunction.java | 2 +- .../impl/flatmap/SplitArrayFunction.java | 9 +- .../ReturnFunction.java} | 20 +- .../function/impl/json/JsonArrayFunction.java | 42 +- .../impl/json/JsonCreatorFunction.java | 34 +- .../function/impl/json/JsonValueFunction.java | 60 +- .../impl/json/UDTFFieldNameFunction.java | 26 +- .../function/impl/math/AbsFunction.java | 40 +- .../function/impl/math/AcosFunction.java | 26 +- .../function/impl/math/AdditionFunction.java | 40 +- .../function/impl/math/AsinFunction.java | 26 +- .../function/impl/math/AtanFunction.java | 6 +- .../function/impl/math/BinFunction.java | 2 +- .../function/impl/math/BitFunction.java | 28 +- .../function/impl/math/BroundFunction.java | 12 +- .../impl/math/CardinalityFunction.java | 2 +- .../function/impl/math/CbrtFunction.java | 4 +- .../function/impl/math/CeilFunction.java | 26 +- .../function/impl/math/ConvFunction.java | 36 +- .../function/impl/math/CosFunction.java | 25 +- .../function/impl/math/CosHFunction.java | 25 +- .../function/impl/math/CotFunction.java | 25 +- .../function/impl/math/DegreesFunction.java | 23 +- .../function/impl/math/DivisionFunction.java | 4 +- .../function/impl/math/ExpFunction.java | 25 +- .../function/impl/math/FactorialFunction.java | 4 +- .../function/impl/math/FloorFunction.java | 25 +- .../function/impl/math/HexFunction.java | 4 +- .../function/impl/math/IsDecimalFunction.java | 35 +- .../script/function/impl/math/LnFunction.java | 25 +- .../function/impl/math/Log10Function.java | 24 +- .../function/impl/math/Log2Function.java | 24 +- .../function/impl/math/LogFunction.java | 32 +- .../function/impl/math/MathFunction.java | 31 +- .../impl/math/MultiplicationFunction.java | 4 +- .../function/impl/math/OperationFunction.java | 16 +- .../function/impl/math/PowerFunction.java | 33 +- .../function/impl/math/RoundFunction.java | 31 +- .../function/impl/math/ShiftLeftFunction.java | 4 +- .../impl/math/ShiftRightFunction.java | 4 +- .../impl/math/ShiftRightUnSignedFunction.java | 4 +- .../function/impl/math/SignFunction.java | 2 +- .../function/impl/math/SinFunction.java | 25 +- .../function/impl/math/SinHFunction.java | 25 +- .../function/impl/math/SqrtFunction.java | 25 +- .../impl/math/SubtractionFunction.java | 4 +- .../function/impl/math/TanFunction.java | 25 +- .../function/impl/math/TanHFunction.java | 25 +- .../function/impl/math/TruncFunction.java | 32 +- .../function/impl/math/UnHexFunction.java | 4 +- .../function/impl/offset/OffsetFunction.java | 2 +- .../function/impl/parser/GrokFunction.java | 50 +- .../impl/parser/LogParserFunction.java | 10 +- .../impl/parser/Paser2JsonFunction.java | 10 +- .../impl/parser/PaserBySplitFunction.java | 70 +-- .../impl/parser/RegexParserFunction.java | 45 +- .../function/impl/python/GroovyFunction.java | 2 +- .../function/impl/python/PythonFunction.java | 61 +- .../function/impl/relation/AndFunction.java | 20 +- .../script/function/impl/sql/SQLFunction.java | 6 +- .../function/impl/string/AsciiFunction.java | 4 +- .../function/impl/string/Base64Function.java | 6 +- .../impl/string/BlinkStringFunction.java | 6 +- .../impl/string/CharMatchCountFunction.java | 4 +- .../function/impl/string/ChrFunction.java | 4 +- .../impl/string/HashCodeFunction.java | 2 +- .../function/impl/string/InitCapFunction.java | 2 +- .../function/impl/string/InstrFunction.java | 26 +- .../function/impl/string/InstrtFunction.java | 13 +- .../impl/string/IsEncodingFunction.java | 6 +- .../impl/string/KeyValueFunction.java | 20 +- .../function/impl/string/LTrimFunction.java | 2 +- .../function/impl/string/LengthBFunction.java | 4 +- .../function/impl/string/LengthFunction.java | 4 +- .../function/impl/string/Md5Function.java | 10 +- .../impl/string/ParseUrlFunction.java | 16 +- .../function/impl/string/RTrimFunction.java | 2 +- .../impl/string/RegexCountFunction.java | 12 +- .../function/impl/string/RegexFunction.java | 38 +- .../impl/string/RegexInstrFunction.java | 60 +- .../impl/string/RegexReplaceFunction.java | 14 +- .../impl/string/RegexSubStrFunction.java | 20 +- .../function/impl/string/RepeatFunction.java | 4 +- .../function/impl/string/ReplaceFunction.java | 6 +- .../function/impl/string/ReverseFunction.java | 2 +- .../function/impl/string/SoundxFunction.java | 2 +- .../function/impl/string/SpaceFunction.java | 2 +- .../impl/string/SplitPartFunction.java | 20 +- .../function/impl/string/String2Map.java | 18 +- .../impl/string/SubStringIndexFunction.java | 30 +- .../function/impl/string/ToCharFunction.java | 10 +- .../function/impl/string/ToLowerFunction.java | 8 +- .../function/impl/string/ToUpperFunction.java | 2 +- .../impl/string/TransLateFunction.java | 6 +- .../function/impl/string/TrimFunction.java | 12 +- .../impl/string/UrlDecodeFunction.java | 6 +- .../impl/string/UrlEncodeFunction.java | 44 +- .../function/impl/type/CastFunction.java | 8 +- .../function/impl/udtf/UDTFFunction.java | 2 +- .../function/model/FunctionConfigure.java | 37 +- .../script/function/model/FunctionInfo.java | 5 +- .../script/function/model/FunctionType.java | 10 +- .../function/service/IFunctionService.java | 4 +- .../impl/DefaultFunctionServiceImpl.java | 101 +-- .../service/impl/ScanFunctionService.java | 15 +- .../expression/GroupScriptExpression.java | 136 ++-- ...ser.java => IGroupScriptOptimization.java} | 5 +- .../operator/expression/ScriptExpression.java | 84 +-- .../operator/expression/ScriptParameter.java | 2 +- .../operator/impl/AggregationScript.java | 131 ++-- .../script/operator/impl/FunctionScript.java | 185 ++++-- .../operator/impl/GroovyScriptOperator.java | 38 +- .../operator/impl/JPythonScriptOperator.java | 32 +- .../script/operator/impl/ScriptOperator.java | 3 +- .../compile/CompileParameter.java | 18 +- .../compile/CompileScriptExpression.java | 6 +- .../performance/IScriptOptimization.java | 32 +- .../parser/ScriptExpressionParserFactory.java | 6 +- .../parser/imp/ConditionExpressionParser.java | 8 +- .../parser/imp/FunctionExpressionParser.java | 12 +- .../script/parser/imp/FunctionParser.java | 22 +- .../service/impl/ScriptServiceImpl.java | 25 +- .../script/service/udf/SimpleUDAFScript.java | 4 +- .../script/service/udf/UDAFScript.java | 4 +- .../streams/script/service/udf/UDFScript.java | 181 +++--- .../streams/script/utils/FunctionUtils.java | 57 +- .../streams/script/utils/MatchUtil.java | 2 +- .../streams/script/utils/RandomStrUtil.java | 6 +- .../rocketmq/streams/script/utils/UniqID.java | 6 +- .../streams/script/function/FunctionTest.java | 36 +- .../function/aggregation/AccumulatorTest.java | 34 +- .../function/JavaObjectUDFFunction.java | 23 +- .../script/function/function/Person.java | 7 +- .../src/test/resources/log4j.xml | 0 rocketmq-streams-serviceloader/pom.xml | 6 +- .../serviceloader/ServiceLoaderComponent.java | 52 +- .../impl/AnnotationServiceNameGetter.java | 2 +- .../ServiceLoaderComponentTest.java | 2 +- .../src/test/resources/log4j.xml | 0 rocketmq-streams-state/pom.xml | 6 +- .../streams/state/IEntryProcessor.java | 2 +- .../rocketmq/streams/state/ISchama.java | 3 +- .../apache/rocketmq/streams/state/IState.java | 102 ++- .../rocketmq/streams/state/LruState.java | 29 +- .../streams/state/backend/IStateBackend.java | 49 +- .../rocketmq/streams/state/impl/MapState.java | 3 +- .../state/kv/rocksdb/RocksDBOperator.java | 20 +- .../state/kv/rocksdb/RocksdbState.java | 45 +- .../streams/state/kv/TestLruState.java | 4 +- .../src/test/resources/log4j.xml | 0 .../pom.xml | 46 +- .../rocketmq/streams/storage/IStorage.java | 28 + .../streams/storage/IStorageBlock.java | 49 ++ .../rocketmq/streams/storage/orc/ORCFile.java | 377 +++++++++++ .../streams/storage/orc/ORCStorageBlock.java | 130 ++++ .../streams/storage/orc/RocksDBFile.java | 249 ++++++++ .../streams/storage/orc/RowIterator.java | 227 +++++++ .../streams/storage/utils/TermMatchUtil.java | 114 ++++ .../streams/storage/BloomFilterTest.java | 220 +++++++ .../rocketmq/streams/storage/ORCFileTest.java | 177 ++++++ .../streams/storage/RocksDBFileTest.java | 133 ++++ rocketmq-streams-sts/.gitignore | 1 + rocketmq-streams-sts/pom.xml | 41 ++ .../streams/sts/CheckParametersUtils.java | 31 + .../streams/sts/ClientTransformer.java | 131 ++++ .../rocketmq/streams/sts/StsIdentity.java | 77 +++ .../rocketmq/streams/sts/StsService.java | 177 ++++++ rocketmq-streams-tasks/pom.xml | 31 + .../tasks/entity/WeightChainPipeline.java | 42 ++ .../streams/dispatcher/DispatcherTest.java | 42 ++ .../dispatcher/TestStreamsDispatcherV2.java | 56 ++ rocketmq-streams-transport-minio/pom.xml | 6 +- .../transport/minio/MinioFileTransport.java | 12 +- .../streams/configuable/model/Person.java | 2 +- .../streams/configurable/model/Person.java | 2 +- .../src/test/resources/log4j.xml | 0 rocketmq-streams-window/pom.xml | 12 +- .../stage/ShuffleOutputChainStage.java | 196 ++++++ .../stage/ShuffleSourceChainStage.java | 440 +++++++++++++ .../streams/window/WindowConstants.java | 45 ++ .../streams/window/builder/WindowBuilder.java | 16 +- .../streams/window/debug/DebugAnalysis.java | 2 +- .../streams/window/debug/DebugWriter.java | 26 +- .../streams/window/fire/EventTimeManager.java | 14 +- .../window/fire/SplitEventTimeManager.java | 99 ++- .../window/minibatch/MiniBatchMsgCache.java | 40 +- .../window/minibatch/ShuffleMessageCache.java | 135 ++-- .../streams/window/model/WindowCache.java | 204 ------ .../streams/window/model/WindowInstance.java | 142 ++--- .../window/offset/IWindowMaxValueManager.java | 14 +- .../streams/window/offset/WindowMaxValue.java | 38 +- .../window/offset/WindowMaxValueManager.java | 6 +- .../offset/WindowMaxValueProcessor.java | 7 +- .../operator/AbstractShuffleWindow.java | 199 ++++-- .../window/operator/AbstractWindow.java | 545 ++++++++-------- .../window/operator/IShuffleCallback.java | 64 ++ .../streams/window/operator/impl/OrderBy.java | 8 +- .../window/operator/impl/OverWindow.java | 40 +- .../window/operator/impl/SessionOperator.java | 52 +- .../operator/impl/ShuffleOverWindow.java | 117 ++-- .../window/operator/impl/TopNState.java | 277 ++++---- .../window/operator/impl/WindowOperator.java | 286 +++++---- .../window/operator/join/DBOperator.java | 128 ++-- .../window/operator/join/JoinWindow.java | 250 ++++---- .../window/shuffle/AbstractSystemChannel.java | 358 ----------- .../streams/window/shuffle/ShuffleCache.java | 46 +- .../window/shuffle/ShuffleChannel.java | 536 ---------------- .../window/shuffle/ShuffleContext.java | 29 + .../window/shuffle/ShuffleManager.java | 315 ++++++++++ .../streams/window/shuffle/ShuffleSink.java | 266 ++++++++ .../window/source/WindowShuffleSource.java | 87 --- .../streams/window/sqlcache/SQLCache.java | 20 +- .../sqlcache/impl/FiredNotifySQLElement.java | 7 +- .../window/sqlcache/impl/SQLElement.java | 10 +- .../window/sqlcache/impl/SplitSQLElement.java | 8 +- .../window/state/AbstractMapState.java | 13 +- .../streams/window/state/WindowBaseValue.java | 17 +- .../window/state/impl/WindowValue.java | 451 ++++++------- .../window/storage/AbstractWindowStorage.java | 30 +- .../window/storage/IRemoteStorage.java | 1 + .../streams/window/storage/IStorage.java | 17 +- .../storage/ShufflePartitionManager.java | 6 +- .../streams/window/storage/WindowStorage.java | 52 +- .../streams/window/storage/db/DBStorage.java | 34 +- .../storage/rocksdb/RocksdbStorage.java | 129 ++-- .../rocksdb/SelfAdaptionRocksdbStorage.java | 248 ++++++++ .../streams/window/trigger/WindowTrigger.java | 146 ++--- .../streams/window/util/ShuffleUtil.java | 46 +- .../window/util/WindowChannellUtil.java | 3 +- .../apache/rocketmq/streams/RocksdbTest.java | 2 +- .../SelfAdaptionRocksdbStorageTest.java | 99 +++ .../rocketmq/streams/WindowTriggerTest.java | 10 +- .../streams/configurable/model/Person.java | 2 +- .../streams/window/WindowInstanceTest.java | 17 +- .../src/test/resources/log4j.xml | 0 1010 files changed, 35788 insertions(+), 25262 deletions(-) create mode 100644 README-chinese.md delete mode 100644 SUMMARY.md create mode 100644 docs/README.md create mode 100644 docs/SUMMARY.md create mode 100644 docs/quick_start/README.md rename stream_sink.md => docs/stream_sink/README.md (99%) rename stream_source.md => docs/stream_source/README.md (99%) rename stream_transform.md => docs/stream_transform/README.md (100%) create mode 100644 rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/source/AbstractDynamicSplitSource.java rename {rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors => rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db}/source/CycleDynamicMultipleDBScanSource.java (87%) rename {rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader => rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/source}/DBScanReader.java (76%) create mode 100644 rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/source/DBSource.java rename {rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors => rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db}/source/DynamicMultipleDBScanSource.java (81%) create mode 100644 rocketmq-streams-channel-huawei/pom.xml create mode 100644 rocketmq-streams-channel-huawei/src/main/java/org/apache/rocketmq/streams/huawei/obs/ObsSource.java create mode 100644 rocketmq-streams-channel-huawei/src/main/java/org/apache/rocketmq/streams/huawei/obs/builder/ObsSourceBuilder.java create mode 100644 rocketmq-streams-channel-huawei/src/main/java/org/apache/rocketmq/streams/huawei/obs/iterator/ObsDataIterator.java create mode 100644 rocketmq-streams-channel-huawei/src/main/java/org/apache/rocketmq/streams/huawei/obs/split/ObsSplit.java create mode 100644 rocketmq-streams-channel-huawei/src/test/java/org/apache/rocketmq/streams/huawei/ObsTest.java create mode 100644 rocketmq-streams-channel-huawei/src/test/java/org/apache/rocketmq/streams/huawei/TimeTest.java mode change 100755 => 100644 rocketmq-streams-channel-kafka/src/test/resources/log4j.xml rename {rocketmq-streams-checkpoint => rocketmq-streams-channel-metaq}/pom.xml (53%) create mode 100644 rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/MetaqChannelBuilder.java create mode 100644 rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/MetaqOffset.java create mode 100644 rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/debug/DebugWriter.java create mode 100644 rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/queue/MetaqMessageQueue.java create mode 100644 rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/sink/MetaqSink.java create mode 100644 rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/source/MetaqSource.java create mode 100644 rocketmq-streams-channel-metaq/src/test/java/org/apache/rocketmq/streams/AbstractChannelTest.java create mode 100644 rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/factory/MessageProcess.java create mode 100644 rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/factory/MqttClientFactory.java create mode 100644 rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/factory/MqttConnection.java create mode 100644 rocketmq-streams-channel-openapi/.gitignore create mode 100755 rocketmq-streams-channel-openapi/pom.xml create mode 100644 rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/OpenAPIChannelBuilder.java create mode 100644 rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/OpenAPIDim.java create mode 100644 rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/OpenAPISQLParser.java create mode 100644 rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/OpenAPISource.java create mode 100644 rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/AbstractOpenAPIExecutor.java create mode 100644 rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/GenericAcsResponse.java create mode 100644 rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/GenericRpcAcsRequest.java create mode 100644 rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/HttpOpenAPIExecutor.java create mode 100644 rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/OpenAPIExecutor.java create mode 100644 rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/splits/OpenAPISplit.java create mode 100644 rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/utils/ParamUtil.java create mode 100644 rocketmq-streams-channel-openapi/src/test/java/org/apache/rocketmq/streams/openapi/OpenAPIExecutorTest.java create mode 100644 rocketmq-streams-channel-openapi/src/test/java/org/apache/rocketmq/streams/openapi/RuleEngineCenterTest.java rename rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/{ => rocketmq}/debug/DebugWriter.java (97%) rename rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/{ => rocketmq}/queue/RocketMQMessageQueue.java (74%) rename rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/{ => rocketmq}/sink/RocketMQSink.java (71%) rename rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/{ => rocketmq}/source/RocketMQSource.java (58%) create mode 100644 rocketmq-streams-channel-sls/pom.xml create mode 100644 rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/PullSlsChannelBuilder.java rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewChannelBuilder.java => rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/SlsChannelBuilder.java (61%) create mode 100644 rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/SlsSearchBuilder.java create mode 100644 rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/sink/SLSSink.java create mode 100644 rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/sink/SLSSplit.java create mode 100644 rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/LogHubProcessor.java create mode 100644 rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/LogHubProcessorFactory.java create mode 100644 rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/SLSPullSource.java create mode 100644 rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/SLSSearchSource.java create mode 100644 rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/SLSSource.java create mode 100644 rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/sts/StsClientWorker.java create mode 100644 rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/utils/SLSUtil.java create mode 100644 rocketmq-streams-channel-tencent/pom.xml create mode 100644 rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/TencentOpenAPIClient.java create mode 100644 rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/TencentOpenAPISplit.java create mode 100644 rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/common/TencentOpenAPIChannelBuilder.java create mode 100644 rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/common/TencentOpenAPISource.java create mode 100644 rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/waf/TencentWafAttackChannelBuilder.java create mode 100644 rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/waf/TencentWafAttackSource.java create mode 100644 rocketmq-streams-channel-tencent/src/test/java/org/apache/rocketmq/streams/tencent/waf/TestCommonAPI.java create mode 100644 rocketmq-streams-channel-tencent/src/test/java/org/apache/rocketmq/streams/tencent/waf/TestWafAPI.java create mode 100644 rocketmq-streams-channel-view/.gitignore create mode 100755 rocketmq-streams-channel-view/pom.xml create mode 100644 rocketmq-streams-channel-view/src/main/java/org/apache/rocketmq/streams/view/ViewChannelBuilder.java create mode 100644 rocketmq-streams-channel-view/src/test/java/org/apache/rocketmq/streams/db/sink/es/EsChannelTest.java delete mode 100644 rocketmq-streams-checkpoint/src/main/java/org/apache/rocketmq/streams/checkpoint/db/DBCheckPointStorage.java create mode 100644 rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/AbstractExecutionEnvironment.java delete mode 100644 rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/ScheduledStreamBuilder.java delete mode 100644 rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/ScheduledTask.java create mode 100644 rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/StreamExecutionEnvironment.java delete mode 100644 rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ApplicationTest.java delete mode 100644 rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/DBDriverTest.java create mode 100644 rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/MQTTTest.java create mode 100644 rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SourceMergerTest.java rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IAfterConfigurableRefreshListener.java => rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/model/JobModel.java (72%) create mode 100644 rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/model/RuleContainer.java create mode 100644 rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/model/TestRule.java create mode 100644 rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/SimpleWindowTest.java mode change 100755 => 100644 rocketmq-streams-clients/src/test/resources/log4j.xml delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/MappedByteBufferTableWithPrimaryIndex.java delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/IntValueKV.java delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/StringValueKV.java delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemoryChannel.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/RowListChannelBuilder.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/RowListSource.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractPushSource.java rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/{AbstractBatchSource.java => AbstractSingleSplitSource.java} (69%) create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/SplitProgress.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/systemmsg/WaterMarkNotifyMessage.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/split/CommonSplit.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/split/DynamicSplit.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/ISplitOffset.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/ENVField.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurableQuery.java delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurableService.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/annotation/ConfigurableReference.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configuration/ConfigurationKey.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configuration/EmitInsertConfig.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configuration/JobConfiguration.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configuration/SystemContext.java delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configure/ConfigureFileKey.java delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configure/StreamsConfigure.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/constant/State.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/enums/StageType.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/exception/CanNotSupportThisMethod.java rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/{channel/source/AbstractSupportShuffleSource.java => interfaces/IConfigurableLifeStart.java} (85%) create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IConfigurablePropertySetter.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IDim.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/INeedAalignWaiting.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IStage.java rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/{schedule/IScheduleCondition.java => interfaces/ITaskStart.java} (87%) rename rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/IScriptUDFInit.java => rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IUDF.java (90%) create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/JobConfigure.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/JobStage.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/Pair.java rename rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/ConfigurableServcieType.java => rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/SQLCompileContextForSource.java (58%) create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/StageRelation.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/IHomologousCalculate.java rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/{MessageGlobleTrace.java => MessageTrace.java} (67%) delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/TaskOptimization.java delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/schedule/ScheduleManager.java delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/schedule/ScheduleTask.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/threadpool/ScheduleFactory.java delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/AbstractMutilPipelineChainPipline.java delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ChainPipeline.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/IJobGraph.java rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/{model => }/ILifeCycle.java (95%) rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/{stages/udf => }/IReducer.java (94%) rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/{shuffle => }/IShuffleKeyGenerator.java (94%) rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/{builder => }/IStageBuilder.java (90%) create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ISynchronousWindow.java rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/{model => }/IWindow.java (53%) rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/{task => assigner}/TaskAssigner.java (78%) create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/graph/JobGraph.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/SourceMetric.java rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/{ChainStage.java => model/AbstractChainStage.java} (90%) create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractPipeline.java rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/{IStageHandle.java => AbstractStageHandle.java} (80%) create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/ChainPipeline.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/JobGraphCopy.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/MessageFinishCallBack.java delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/Pipeline.java rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/{ => model}/SectionPipeline.java (76%) create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/StreamGraph.java delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/shuffle/ShuffleMQCreator.java delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/AbstractWindowStage.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/DimChainStage.java delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/JoinChainStage.java delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/RightJoinChainStage.java delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/SQLChainStage.java rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/{UnionChainStage.java => ShuffleChainStage.java} (51%) delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ShuffleConsumerChainStage.java delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ShuffleProducerChainStage.java rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/{WindowChainStage.java => SynchronousWindowChainStage.java} (62%) delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ViewChainStage.java delete mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/task/StreamsTask.java rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/{DESUtils.java => DESUtil.java} (98%) rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/{ENVUtile.java => ENVUtil.java} (90%) create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/IdUtil.java rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/{Ip2LongUtils.java => Ip2LongUtil.java} (95%) rename rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/{PropertiesUtils.java => PropertiesUtil.java} (74%) create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/RocketmqUtil.java create mode 100644 rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/cache/compress/FingerprintCacheTest.java delete mode 100644 rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/ConfigurableComponent.java delete mode 100644 rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/model/Configure.java delete mode 100644 rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/AbstractConfigurableService.java delete mode 100644 rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/AbstractSupportParentConfigureService.java delete mode 100644 rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/ConfigurableServiceFactory.java delete mode 100644 rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/FileConfigureService.java delete mode 100644 rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/FileSupportParentConfigureService.java delete mode 100644 rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/HttpConfigureService.java delete mode 100644 rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/HttpSupportParentConfigureService.java delete mode 100644 rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/MemoryConfigureService.java delete mode 100644 rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/MemorySupportParentConfigureService.java delete mode 100644 rocketmq-streams-configurable/src/test/java/org/apache/rocketmq/streams/configuable/ConfiguableComponentTest.java delete mode 100644 rocketmq-streams-configurable/src/test/java/org/apache/rocketmq/streams/configuable/model/Person.java delete mode 100644 rocketmq-streams-configurable/src/test/java/org/apache/rocketmq/streams/configurable/ConfigurableComponentTest.java delete mode 100644 rocketmq-streams-configurable/src/test/java/org/apache/rocketmq/streams/configurable/model/Person.java delete mode 100755 rocketmq-streams-configurable/src/test/resources/log4j.xml create mode 100644 rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/model/QuerySplit.java rename rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/StreamBuilder.java => rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/AbstractFlinkSplitReader.java (62%) create mode 100644 rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/AbstractQueryReader.java create mode 100644 rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/AbstractSplitReader.java create mode 100644 rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/AbstractQuerySource.java delete mode 100644 rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/MutilBatchTaskSource.java create mode 100644 rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/client/SplitConsumer.java create mode 100644 rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/impl/FilesPullSource.java create mode 100644 rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/impl/NonHomologousPullSource.java create mode 100644 rocketmq-streams-connectors/src/test/java/org/apache/rocketmq/streams/connector/FileSourceTest.java rename {rocketmq-streams-cep => rocketmq-streams-connectors}/src/test/resources/log4j.xml (100%) mode change 100755 => 100644 rename {rocketmq-streams-dbinit => rocketmq-streams-db-init}/pom.xml (86%) rename {rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit => rocketmq-streams-db-init/src/main/java/org/apache/rocketmq/streams/db/init}/mysql/delegate/DBDelegate.java (82%) rename {rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit => rocketmq-streams-db-init/src/main/java/org/apache/rocketmq/streams/db/init}/mysql/delegate/DBDelegateFactory.java (83%) rename {rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit => rocketmq-streams-db-init/src/main/java/org/apache/rocketmq/streams/db/init}/mysql/delegate/DBType.java (93%) rename {rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit => rocketmq-streams-db-init/src/main/java/org/apache/rocketmq/streams/db/init}/mysql/delegate/MysqlDelegate.java (83%) create mode 100644 rocketmq-streams-db-init/src/main/resources/tables_mysql_innodb.sql delete mode 100644 rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/DBConfigureService.java delete mode 100644 rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/DBSupportParentConfigureService.java delete mode 100644 rocketmq-streams-db-operator/src/test/java/org/apache/rocketmq/streams/db/configuable/DBSupportParentConfigureServiceTest.java delete mode 100644 rocketmq-streams-dbinit/src/main/resources/tables_mysql_innodb.sql delete mode 100644 rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/DimComponent.java delete mode 100644 rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/expression/InExpressionResource.java delete mode 100644 rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/expression/NotInExpressionResource.java delete mode 100644 rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/NameListFunction.java create mode 100644 rocketmq-streams-dim/src/test/java/com/aliyun/service/DBDimTest.java create mode 100644 rocketmq-streams-dispatcher/pom.xml create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IBalanceCallback.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/ICache.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IDispatcher.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IDispatcherCallback.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IMapper.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IStrategy.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IStrategyCallBack.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/cache/DBCache.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/cache/RocketmqCache.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/callback/DefaultDispatcherCallback.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/constant/DispatcherConstant.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/entity/DispatcherMapper.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/enums/DispatchMode.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/impl/LeaseDispatcher.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/impl/RocketmqDispatcher.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/AllStrategy.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/AveragelyStrategy.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/ConsistencyStrategy.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/HashStrategy.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/LeastStrategy.java create mode 100644 rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/StrategyFactory.java create mode 100644 rocketmq-streams-examples/README.md create mode 100644 rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/Test.java rename rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/script/{CaseDependentParser.java => GroupScriptOptimization.java} (51%) create mode 100644 rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/IFCaseWhenExpression.java create mode 100644 rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/IFCaseWhenExpressionGroup.java create mode 100644 rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/IFExpressionOptimization.java create mode 100644 rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/EmptyTreeNode.java delete mode 100644 rocketmq-streams-filter/src/test/java/org/apache/rocketmq/streams/filter/operator/RuleTest.java mode change 100755 => 100644 rocketmq-streams-filter/src/test/resources/log4j.xml rename rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/{BasedLesaseImpl.java => BasedLeaseImpl.java} (80%) mode change 100755 => 100644 rocketmq-streams-lease/src/test/resources/log4j.xml rename {rocketmq-streams-configurable => rocketmq-streams-local-cache}/pom.xml (69%) rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/main/java/org/apache/rocketmq/streams/common/cache/compress/AdditionStore.java (99%) rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BigMapAddress.java (100%) rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BitSetCache.java (100%) rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArray.java (100%) rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArrayValueKV.java (100%) rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteStore.java (100%) rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/main/java/org/apache/rocketmq/streams/common/cache/compress/CacheKV.java (94%) rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ICacheKV.java (100%) rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVAddress.java (79%) rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVElement.java (100%) rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/main/java/org/apache/rocketmq/streams/common/cache/compress/MappedByteBufferStore.java (100%) rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/AbstractListKV.java (100%) create mode 100644 rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/IntValueKV.java rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/KeySet.java (100%) create mode 100644 rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LightCache.java rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LongListKV.java (78%) rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LongValueKV.java (100%) rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MapAddressListKV.java (100%) rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MultiValueKV.java (100%) rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MutilValueKV.java (100%) rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/SplitCache.java (100%) create mode 100644 rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/StringValueKV.java create mode 100644 rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/utils/NumberUtils.java create mode 100644 rocketmq-streams-local-cache/src/test/java/org/apache/rocketmq/streams/common/cache/compress/FingerprintCacheTest.java create mode 100644 rocketmq-streams-local-cache/src/test/java/org/apache/rocketmq/streams/common/cache/compress/IntValueKVTest.java rename {rocketmq-streams-commons => rocketmq-streams-local-cache}/src/test/resources/log4j.xml (100%) mode change 100755 => 100644 delete mode 100644 rocketmq-streams-runner/assembly/distribution.xml delete mode 100644 rocketmq-streams-runner/assembly/standalone.xml delete mode 100755 rocketmq-streams-runner/bin/start.sh delete mode 100755 rocketmq-streams-runner/bin/stop.sh delete mode 100644 rocketmq-streams-runner/log/catalina.out delete mode 100644 rocketmq-streams-runner/pom.xml delete mode 100644 rocketmq-streams-runner/src/main/resources/log4j.xml mode change 100755 => 100644 rocketmq-streams-schedule/src/test/resources/log4j.xml create mode 100644 rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/exception/ScriptExpresionRuntimeExpression.java create mode 100644 rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/common/EmptyFunction.java delete mode 100644 rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/env/ENVFunction.java rename rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/{mock/MockFunction.java => flow/ReturnFunction.java} (61%) rename rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/{ICaseDependentParser.java => IGroupScriptOptimization.java} (91%) mode change 100755 => 100644 rocketmq-streams-script/src/test/resources/log4j.xml mode change 100755 => 100644 rocketmq-streams-serviceloader/src/test/resources/log4j.xml mode change 100755 => 100644 rocketmq-streams-state/src/test/resources/log4j.xml rename {rocketmq-streams-cep => rocketmq-streams-storage}/pom.xml (55%) mode change 100755 => 100644 create mode 100644 rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/IStorage.java create mode 100644 rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/IStorageBlock.java create mode 100644 rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/orc/ORCFile.java create mode 100644 rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/orc/ORCStorageBlock.java create mode 100644 rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/orc/RocksDBFile.java create mode 100644 rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/orc/RowIterator.java create mode 100644 rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/utils/TermMatchUtil.java create mode 100644 rocketmq-streams-storage/src/test/java/org/apache/rocketmq/streams/storage/BloomFilterTest.java create mode 100644 rocketmq-streams-storage/src/test/java/org/apache/rocketmq/streams/storage/ORCFileTest.java create mode 100644 rocketmq-streams-storage/src/test/java/org/apache/rocketmq/streams/storage/RocksDBFileTest.java create mode 100644 rocketmq-streams-sts/.gitignore create mode 100755 rocketmq-streams-sts/pom.xml create mode 100644 rocketmq-streams-sts/src/main/java/org/apache/rocketmq/streams/sts/CheckParametersUtils.java create mode 100644 rocketmq-streams-sts/src/main/java/org/apache/rocketmq/streams/sts/ClientTransformer.java create mode 100644 rocketmq-streams-sts/src/main/java/org/apache/rocketmq/streams/sts/StsIdentity.java create mode 100644 rocketmq-streams-sts/src/main/java/org/apache/rocketmq/streams/sts/StsService.java create mode 100644 rocketmq-streams-tasks/pom.xml create mode 100644 rocketmq-streams-tasks/src/main/java/org/apache/rocketmq/streams/tasks/entity/WeightChainPipeline.java create mode 100644 rocketmq-streams-tasks/src/test/java/org/apache/rocketmq/streams/dispatcher/DispatcherTest.java create mode 100644 rocketmq-streams-tasks/src/test/java/org/apache/rocketmq/streams/dispatcher/TestStreamsDispatcherV2.java mode change 100755 => 100644 rocketmq-streams-transport-minio/src/test/resources/log4j.xml create mode 100644 rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/stage/ShuffleOutputChainStage.java create mode 100644 rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/stage/ShuffleSourceChainStage.java create mode 100644 rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/WindowConstants.java delete mode 100644 rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowCache.java create mode 100644 rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/IShuffleCallback.java delete mode 100644 rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/AbstractSystemChannel.java delete mode 100644 rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleChannel.java create mode 100644 rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleContext.java create mode 100644 rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleManager.java create mode 100644 rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleSink.java delete mode 100644 rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/source/WindowShuffleSource.java create mode 100644 rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocksdb/SelfAdaptionRocksdbStorage.java create mode 100644 rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/SelfAdaptionRocksdbStorageTest.java rename rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/IntelligenceNameListFunction.java => rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/WindowTriggerTest.java (82%) mode change 100755 => 100644 rocketmq-streams-window/src/test/resources/log4j.xml diff --git a/README-chinese.md b/README-chinese.md new file mode 100644 index 00000000..da7d4489 --- /dev/null +++ b/README-chinese.md @@ -0,0 +1,150 @@ +[![GitHub release](https://img.shields.io/badge/release-download-orange.svg)](https://github.com/apache/rocketmq-streams/releases) +[![License](https://img.shields.io/badge/license-Apache%202-4EB1BA.svg)](https://www.apache.org/licenses/LICENSE-2.0.html) +[![Average time to resolve an issue](http://isitmaintained.com/badge/resolution/apache/rocketmq-streams.svg)](http://isitmaintained.com/project/apache/rocketmq-streams "Average time to resolve an issue") +[![Percentage of issues still open](http://isitmaintained.com/badge/open/apache/rocketmq-streams.svg)](http://isitmaintained.com/project/apache/rocketmq-streams "Percentage of issues still open") +[![Twitter Follow](https://img.shields.io/twitter/follow/ApacheRocketMQ?style=social)](https://twitter.com/intent/follow?screen_name=ApacheRocketMQ) + +# Features + +* 轻量级部署:可以单独部署,也支持集群部署 +* 多种类型的数据输入以及输出,source 支持 rocketmq , sink 支持db, rocketmq 等 + +# DataStream Example + +```java +import org.apache.rocketmq.streams.client.transform.DataStream; + +DataStreamSource source=StreamBuilder.dataStream("namespace","pipeline"); + + source + .fromFile("~/admin/data/text.txt",false) + .map(message->message) + .toPrint(1) + .start(); +``` + +# Maven Repository + +```xml + + + org.apache.rocketmq + rocketmq-streams-clients + 1.0.2-SNAPSHOT + +``` + +# Core API + +rocketmq-stream 实现了一系列高级的API,可以让用户很方便的编写流计算的程序,实现自己的业务需求; + +## StreamBuilder + +StreamBuilder 用于构建流任务的源; + ++ [dataStream(nameSpaceName,pipelineName)]() 返回DataStreamSource实例,用于分段编程实现流计算任务; + +## DataStream API + +### Source + +DataStreamSource 是分段式编程的源头类,用于对接各种数据源, 从各大消息队列中获取数据; + ++ ```fromFile``` 从文件中读取数据, 该方法包含俩个参数 + + ```filePath``` 文件路径,必填参数 + + ```isJsonData``` 是否json数据, 非必填参数, 默认为```true``` + + ++ ```fromRocketmq``` 从rocketmq中获取数据,包含四个参数 + + ```topic``` rocketmq消息队列的topic名称,必填参数 + + ```groupName``` 消费者组的名称,必填参数 + + ```isJson``` 是否json格式,非必填参数 + + ```tags``` rocketmq消费的tags值,用于过滤消息,非必填参数 + ++ ```fromKafka``` 从kafka中获取数据,包含5个参数 + + ```bootstrapserver``` kafka的bootstrapserver 地址,包括ip和端口,多个值以逗号分隔, 必填参数 + + ```topic``` kafka的topic, 必填参数 + + ```groupName``` 消费组, 必填参数 + + ```isJson``` 是否json格式,非必填参数,默认为true + + ```maxThread``` 客户端最大线程数,非必填参数,默认为1 + ++ ```fromMqtt``` 从满足MQTT协议的终端读取数据, 满足边缘计算的场景,其中包含9个参数 + + ```url``` mqtt broker的地址,必填参数 + + ```clientId``` 客户端ID, 必填参数,相同的clientId有负载的作用 + + ```topic``` topic信息, 必填参数 + + ```username``` 用户名, 非必填,在mqtt端添加鉴权机制时使用 + + ```password``` 密码,非必填参数,在mqtt端添加鉴权机制时使用 + + ```cleanSession``` 是否清理session信息, 非必填,默认为true + + ```connectionTimeout``` 连接超时信息, 非必填,默认是10s + + ```aliveInterval``` 判断连接是否活跃的间隔时间,非必填,默认是60s + + ```automaticReconnect``` 连接断开后自动重连机制,非必填,默认是true + + ++ ```from``` 自定义的数据源, 通过实现ISource接口实现自己的数据源 + +### transform + +transform 允许在流计算过程中对输入源的数据进行修改,进行下一步的操作;DataStream API中包括```DataStream```,```JoinStream```, ```SplitStream```,```WindowStream```等多个transform类; + +DataStream实现了一系列常见的流计算算子 + ++ ```map``` 通过将源的每个记录传递给函数func来返回一个新的DataStream ++ ```flatmap``` 与map类似,一个输入项对应0个或者多个输出项 ++ ```filter``` 只选择func返回true的源DStream的记录来返回一个新的DStream ++ ```forEach``` 对每个记录执行一次函数func, 返回一个新的DataStream ++ ```selectFields``` 对每个记录返回对应的字段值,返回一个新的DataStream ++ ```operate``` 对每个记录执行一次自定义的函数,返回一个新的DataStream ++ ```script``` 针对每个记录的字段执行一段脚本,返回新的字段,生成一个新的DataStream ++ ```toPrint``` 将结果在控制台打印,生成新的DataStream实例 ++ ```toFile``` 将结果保存为文件,生成一个新的DataStream实例 ++ ```toMqtt``` 将结果输出到满足mqtt协议的设备中,生成一个新的DataStream实例 ++ ```toDB``` 将结果保存到数据库 ++ ```toRocketmq``` 将结果输出到rocketmq ++ ```toKafka``` 将结果输出到kafka ++ ```to``` 将结果经过自定义的ISink接口输出到指定的存储 ++ ```window``` 在窗口内进行相关的统计分析,一般会与```groupBy```连用, ```window()```用来定义窗口的大小, ```groupBy()```用来定义统计分析的主key,可以指定多个 + + ```count``` 在窗口内计数 + + ```min``` 获取窗口内统计值的最小值 + + ```max``` 获取窗口内统计值得最大值 + + ```avg``` 获取窗口内统计值的平均值 + + ```sum``` 获取窗口内统计值的加和值 + + ```reduce``` 在窗口内进行自定义的汇总运算 ++ ```join``` 根据条件将俩个流进行内关联 ++ ```leftJoin``` 根据条件将俩个流的数据进行左关联 ++ ```dimJoin``` 根据条件将流与维表进行内关联,维表的数据可以来自于文件,也可以来自于数据库 ++ ```dimLeftJoin``` 根据条件将流与维表进行左关联,维表的数据可以来自于文件,也可以来自于数据库 ++ ```union``` 将俩个流进行合并 ++ ```split``` 将一个数据流按照标签进行拆分,分为不同的数据流供下游进行分析计算 ++ ```with``` with算子用来指定计算过程中的相关策略,包括checkpoint的存储策略,state的存储策略等 + +#### Strategy + +策略机制主要用来控制计算引擎运行过程中的底层逻辑,如checkpoint,state的存储方式等,后续还会增加对窗口、双流join等的控制;所有的控制策略通过```with```算子传入,可以同时传入多个策略类型; + +```java +//指定checkpoint的存储策略 +source + .fromRocketmq("TSG_META_INFO","") + .map(message->message+"--") + .toPrint(1) + .with(CheckpointStrategy.db("jdbc:mysql://XXXXX:3306/XXXXX","","",0L)) + .start(); +``` + +# 运行 + +Rocketmq-Streams 作为典型的java应用,既可以集成在业务系统里运行,也可以作为一个独立的jar包来运行; + +首先对应用的源码进行编译 + +```shell +mvn -Prelease-all -DskipTests clean install -U +``` + +然后直接通过java指令来运行 + +```shell + java -jar jarName mainClass +``` + +更多详细的案例可以看[这里](docs/SUMMARY.md) \ No newline at end of file diff --git a/README.md b/README.md index e4205699..05c7e8d3 100644 --- a/README.md +++ b/README.md @@ -1,6 +1,145 @@ -# Summary +# RocketMQ Streams [![Build Status](https://app.travis-ci.com/apache/rocketmq-streams.svg?branch=main)](https://app.travis-ci.com/apache/rocketmq-streams) [![CodeCov](https://codecov.io/gh/apache/rocketmq-stream/branch/main/graph/badge.svg)](https://app.codecov.io/gh/apache/rocketmq-streams) -* [Quick Start](quick\_start.md) -* [创建实时任务数据源](stream\_source.md) -* [创建实时任务数据输出](stream\_sink.md) -* [数据处理逻辑](stream\_transform.md) +[![GitHub release](https://img.shields.io/badge/release-download-orange.svg)](https://github.com/apache/rocketmq-streams/releases) +[![License](https://img.shields.io/badge/license-Apache%202-4EB1BA.svg)](https://www.apache.org/licenses/LICENSE-2.0.html) +[![Average time to resolve an issue](http://isitmaintained.com/badge/resolution/apache/rocketmq-streams.svg)](http://isitmaintained.com/project/apache/rocketmq-streams "Average time to resolve an issue") +[![Percentage of issues still open](http://isitmaintained.com/badge/open/apache/rocketmq-streams.svg)](http://isitmaintained.com/project/apache/rocketmq-streams "Percentage of issues still open") +[![Twitter Follow](https://img.shields.io/twitter/follow/ApacheRocketMQ?style=social)](https://twitter.com/intent/follow?screen_name=ApacheRocketMQ) + +## [中文文档](./README-Chinese.md) + +## [Quick Start](./quick_start.md) + +## Features + +* Lightweight deployment: RocketMQ Streams can be deployed separately or in cluster mode. +* Various types of data input and output: source supports RocketMQ while sink supports databases and RocketMQ, etc. + +## DataStream Example + +```java +import org.apache.rocketmq.streams.client.transform.DataStream; + +DataStreamSource source=StreamBuilder.dataStream("namespace","pipeline"); + source + .fromFile("~/admin/data/text.txt",false) + .map(message->message) + .toPrint(1) + .start(); +``` + +## Maven Repository + +```xml + + + org.apache.rocketmq + rocketmq-streams-clients + 1.0.2-SNAPSHOT + +``` + +# Core API + +RocketMQ Streams implements a series of advanced APIs, allowing users to write stream computing programs conveniently and achieve their own business requirements. + +## StreamBuilder + +StreamBuilder is used to build the source of stream tasks. It contains two methods: ```dataStream()``` and ```tableStream()```, which return two sources, DataStreamSource and TableStreamSource, respectively. + ++ [dataStream(nameSpaceName,pipelineName)]() returns an instance of DataStreamSource, used for segmented programming to achieve stream computing tasks. ++ [tableStream(nameSpaceName,pipelineName)]() returns an instance of TableStreamSource, used for script programming to achieve stream computing tasks. + +## DataStream API + +### Source + +DataStreamSource is a source class of segmented programming, used to interface with various data sources and obtain data from major message queues. + ++ ```fromFile```: reads data from the file. This method contains two parameters: + + ```filePath```: specifies which file path to read. Required. + + ```isJsonData```: specifies whether data is in JSON format. Optional. Default value: ```true```. + + ```tags```: the tags for filtering messages used by the RocketMQ consumer. Optional. + + ++ ```fromRocketmq```: obtains data from RocketMQ, including four parameters: + + ```topic```: the topic name of RocketMQ. Required. + + ```groupName```: the name of the consumer group. Required. + + ```isJson```: specifies whether data is in JSON format. Optional. + + ```tags```: the tags for filtering messages used by the RocketMQ consumer. Optional. + ++ ```fromKafka``` read data from the Kafka, including five parameters: + + ```bootstrapserver``` the Kafka bootstrap servers. Required. + + ```topic``` the topic name of Kafka. Required. + + ```groupName``` the name of the consumer group. Required. + + ```isJson``` specifies whether data is in JSON format. Optional. + + ```maxThread``` the number of the Kafka consumer max Threads.Optional. + ++ ```fromMqtt``` reads data from MQTT service, including nine parameters: + + ```url``` the broker of the MQTT service. Required. + + ```clientId``` the client id. Required + + ```topic``` the name of the MQTT topic. Required. + + ```username``` username. Optional. + + ```password``` password. Optional. + + ```cleanSession``` specifies Whether to clear the session during the restart. Optional. + + ```connectionTimeout``` the connection timeout. Optional. + + ```aliveInterval``` Survival time interval. Optional. + + ```automaticReconnect``` specifies Whether to reconnect. Optional. + ++ ```from```: custom data source. You can specify your own data source by implementing ISource interface. + +### transform + +transform allows the input source data to be modified during the stream calculation process for the next step; DataStream API includes ```DataStream```, ```JoinStream```, ```SplitStream```, ```WindowStream```, and many other transform +classes. + +#### DataStream + +DataStream implements a series of common stream calculation operators as follows: + ++ ```map```: returns a new DataStream by passing each record of the source to the **func** function. ++ ```flatmap```: similar to map. One input item corresponds to 0 or more output items. ++ ```filter```: returns a new DataStream based on the record of the source DataStream only when the ** func** function returns **true**. ++ ```forEach```: executes the **func** function once for each record and returns a new DataStream. ++ ```selectFields```: returns the corresponding field value for each record, and returns a new DataStream. ++ ```operate```: executes a custom function for each record and returns a new DataStream. ++ ```script```: executes a script for each recorded field, returns new fields, and generates a new DataStream. ++ ```toPrint```: prints the result on the console and generates a new DataStreamAction instance. ++ ```toFile```: saves the result as a file and generates a new DataStreamAction instance. ++ ```toDB```: saves the result to the database. ++ ```toRocketmq```: outputs the result to RocketMQ. ++ ```toKafka```: outputs the result to Kafka. ++ ```to```: outputs the result to the specified storage through the custom ISink interface. ++ ```window```: performs relevant statistical analysis in the window, generally used in conjunction with ```groupBy```. ```window()``` is used to define the size of the window, and ```groupBy( )``` used to define the main key of statistical + analysis. You can specify multiple main keys: + + ```count```: counts in the window. + + ```min```: gets the minimum of the statistical value in the window. + + ```max```: gets the maximum of the statistical value in the window. + + ```avg```: gets the average of the statistical values in the window. + + ```sum```: gets the sum of the statistical values in the window. + + ```reduce```: performs custom summary calculations in the window. ++ ```join```: associates the two streams or one stream and one physical table according to the conditions and merges them into a large stream for related calculations. + + ```dimJoin``` associate a stream with a physical table which can be a file or a db table, and all matching records are retained + + ```dimLeftJoin``` After a flow is associated with a physical table, all data of the flow is reserved and fields that do not match the physical table are left blank + + ```join``` + + ```leftJoin``` ++ ```union```: merges the two streams. ++ ```split```: splits a data stream into different data streams according to tags for downstream analysis and calculation. ++ ```with```: specifies related strategies during the calculation, including Checkpoint and state storage strategies, etc. + +# Strategy + +The Strategy mechanism is mainly used to control the underlying logic during the operation of the computing engine, such as the storage methods of Checkpoint and state etc. Subsequent controls for windows, dual-stream joins, and so on will +be added. All control strategies are transmitted through the ```with``` operator. Multiple policy types can be transmitted at the same time. + +```java +//Specify the storage strategy for Checkpoint. +source + .fromRocketmq("TSG_META_INFO","") + .map(message->message+"--") + .toPrint(1) + .with(CheckpointStrategy.db("jdbc:mysql://XXXXX:3306/XXXXX","","",0L)) + .start(); +``` + +—————————————— diff --git a/SUMMARY.md b/SUMMARY.md deleted file mode 100644 index 2c991438..00000000 --- a/SUMMARY.md +++ /dev/null @@ -1,7 +0,0 @@ -# Table of contents - -* [Summary](README.md) -* [快速开发](quick\_start.md) -* [stream\_source](stream\_source.md) -* [stream\_sink](stream\_sink.md) -* [数据处理逻辑](stream\_transform.md) diff --git a/docs/README.md b/docs/README.md new file mode 100644 index 00000000..5282bb1d --- /dev/null +++ b/docs/README.md @@ -0,0 +1,150 @@ +[![GitHub release](https://img.shields.io/badge/release-download-orange.svg)](https://github.com/apache/rocketmq-streams/releases) +[![License](https://img.shields.io/badge/license-Apache%202-4EB1BA.svg)](https://www.apache.org/licenses/LICENSE-2.0.html) +[![Average time to resolve an issue](http://isitmaintained.com/badge/resolution/apache/rocketmq-streams.svg)](http://isitmaintained.com/project/apache/rocketmq-streams "Average time to resolve an issue") +[![Percentage of issues still open](http://isitmaintained.com/badge/open/apache/rocketmq-streams.svg)](http://isitmaintained.com/project/apache/rocketmq-streams "Percentage of issues still open") +[![Twitter Follow](https://img.shields.io/twitter/follow/ApacheRocketMQ?style=social)](https://twitter.com/intent/follow?screen_name=ApacheRocketMQ) + +# Features + +* 轻量级部署:可以单独部署,也支持集群部署 +* 多种类型的数据输入以及输出,source 支持 rocketmq , sink 支持db, rocketmq 等 + +# DataStream Example + +```java +import org.apache.rocketmq.streams.client.transform.DataStream; + +DataStreamSource source=StreamBuilder.dataStream("namespace","pipeline"); + + source + .fromFile("~/admin/data/text.txt",false) + .map(message->message) + .toPrint(1) + .start(); +``` + +# Maven Repository + +```xml + + + org.apache.rocketmq + rocketmq-streams-clients + 1.0.0-SNAPSHOT + +``` + +# Core API + +rocketmq-stream 实现了一系列高级的API,可以让用户很方便的编写流计算的程序,实现自己的业务需求; + +## StreamBuilder + +StreamBuilder 用于构建流任务的源; + ++ [dataStream(nameSpaceName,pipelineName)]() 返回DataStreamSource实例,用于分段编程实现流计算任务; + +## DataStream API + +### Source + +DataStreamSource 是分段式编程的源头类,用于对接各种数据源, 从各大消息队列中获取数据; + ++ ```fromFile``` 从文件中读取数据, 该方法包含俩个参数 + + ```filePath``` 文件路径,必填参数 + + ```isJsonData``` 是否json数据, 非必填参数, 默认为```true``` + + ++ ```fromRocketmq``` 从rocketmq中获取数据,包含四个参数 + + ```topic``` rocketmq消息队列的topic名称,必填参数 + + ```groupName``` 消费者组的名称,必填参数 + + ```isJson``` 是否json格式,非必填参数 + + ```tags``` rocketmq消费的tags值,用于过滤消息,非必填参数 + ++ ```fromKafka``` 从kafka中获取数据,包含5个参数 + + ```bootstrapserver``` kafka的bootstrapserver 地址,包括ip和端口,多个值以逗号分隔, 必填参数 + + ```topic``` kafka的topic, 必填参数 + + ```groupName``` 消费组, 必填参数 + + ```isJson``` 是否json格式,非必填参数,默认为true + + ```maxThread``` 客户端最大线程数,非必填参数,默认为1 + ++ ```fromMqtt``` 从满足MQTT协议的终端读取数据, 满足边缘计算的场景,其中包含9个参数 + + ```url``` mqtt broker的地址,必填参数 + + ```clientId``` 客户端ID, 必填参数,相同的clientId有负载的作用 + + ```topic``` topic信息, 必填参数 + + ```username``` 用户名, 非必填,在mqtt端添加鉴权机制时使用 + + ```password``` 密码,非必填参数,在mqtt端添加鉴权机制时使用 + + ```cleanSession``` 是否清理session信息, 非必填,默认为true + + ```connectionTimeout``` 连接超时信息, 非必填,默认是10s + + ```aliveInterval``` 判断连接是否活跃的间隔时间,非必填,默认是60s + + ```automaticReconnect``` 连接断开后自动重连机制,非必填,默认是true + + ++ ```from``` 自定义的数据源, 通过实现ISource接口实现自己的数据源 + +### transform + +transform 允许在流计算过程中对输入源的数据进行修改,进行下一步的操作;DataStream API中包括```DataStream```,```JoinStream```, ```SplitStream```,```WindowStream```等多个transform类; + +DataStream实现了一系列常见的流计算算子 + ++ ```map``` 通过将源的每个记录传递给函数func来返回一个新的DataStream ++ ```flatmap``` 与map类似,一个输入项对应0个或者多个输出项 ++ ```filter``` 只选择func返回true的源DStream的记录来返回一个新的DStream ++ ```forEach``` 对每个记录执行一次函数func, 返回一个新的DataStream ++ ```selectFields``` 对每个记录返回对应的字段值,返回一个新的DataStream ++ ```operate``` 对每个记录执行一次自定义的函数,返回一个新的DataStream ++ ```script``` 针对每个记录的字段执行一段脚本,返回新的字段,生成一个新的DataStream ++ ```toPrint``` 将结果在控制台打印,生成新的DataStream实例 ++ ```toFile``` 将结果保存为文件,生成一个新的DataStream实例 ++ ```toMqtt``` 将结果输出到满足mqtt协议的设备中,生成一个新的DataStream实例 ++ ```toDB``` 将结果保存到数据库 ++ ```toRocketmq``` 将结果输出到rocketmq ++ ```toKafka``` 将结果输出到kafka ++ ```to``` 将结果经过自定义的ISink接口输出到指定的存储 ++ ```window``` 在窗口内进行相关的统计分析,一般会与```groupBy```连用, ```window()```用来定义窗口的大小, ```groupBy()```用来定义统计分析的主key,可以指定多个 + + ```count``` 在窗口内计数 + + ```min``` 获取窗口内统计值的最小值 + + ```max``` 获取窗口内统计值得最大值 + + ```avg``` 获取窗口内统计值的平均值 + + ```sum``` 获取窗口内统计值的加和值 + + ```reduce``` 在窗口内进行自定义的汇总运算 ++ ```join``` 根据条件将俩个流进行内关联 ++ ```leftJoin``` 根据条件将俩个流的数据进行左关联 ++ ```dimJoin``` 根据条件将流与维表进行内关联,维表的数据可以来自于文件,也可以来自于数据库 ++ ```dimLeftJoin``` 根据条件将流与维表进行左关联,维表的数据可以来自于文件,也可以来自于数据库 ++ ```union``` 将俩个流进行合并 ++ ```split``` 将一个数据流按照标签进行拆分,分为不同的数据流供下游进行分析计算 ++ ```with``` with算子用来指定计算过程中的相关策略,包括checkpoint的存储策略,state的存储策略等 + +#### Strategy + +策略机制主要用来控制计算引擎运行过程中的底层逻辑,如checkpoint,state的存储方式等,后续还会增加对窗口、双流join等的控制;所有的控制策略通过```with```算子传入,可以同时传入多个策略类型; + +```java +//指定checkpoint的存储策略 +source + .fromRocketmq("TSG_META_INFO","") + .map(message->message+"--") + .toPrint(1) + .with(CheckpointStrategy.db("jdbc:mysql://XXXXX:3306/XXXXX","","",0L)) + .start(); +``` + +# 运行 + +Rocketmq-Streams 作为典型的java应用,既可以集成在业务系统里运行,也可以作为一个独立的jar包来运行; + +首先对应用的源码进行编译 + +```shell +mvn -Prelease-all -DskipTests clean install -U +``` + +然后直接通过java指令来运行 + +```shell + java -jar jarName mainClass +``` + +更多详细的案例可以看[这里](docs/SUMMARY.md) \ No newline at end of file diff --git a/docs/SUMMARY.md b/docs/SUMMARY.md new file mode 100644 index 00000000..950f99bd --- /dev/null +++ b/docs/SUMMARY.md @@ -0,0 +1,8 @@ +# Summary + +* [Introduction](README.md) +* [Quick Start](quick_start/README.md) +* [创建实时任务数据源](stream_source/README.md) +* [创建实时任务数据输出](stream_sink/README.md) +* [数据处理逻辑](stream_transform/README.md) + diff --git a/docs/quick_start/README.md b/docs/quick_start/README.md new file mode 100644 index 00000000..a60dbb95 --- /dev/null +++ b/docs/quick_start/README.md @@ -0,0 +1,46 @@ +# 快速开发 + +## 引入相关的jar包 + +```xml + + + org.apache.rocketmq + rocketmq-streams-clients + + +``` + +## 开发实时应用程序 + +```java + +public class RocketmqExample { + + public static void main(String[] args) { + + DataStreamSource dataStream = StreamBuilder.dataStream("test_namespace", "graph_pipeline"); + + dataStream + .fromFile("data.csv", false) //构建实时任务的数据源 + .map(message -> message.split(",")) //构建实时任务处理的逻辑过程 + .toPrint(1) //构建实时任务的输出 + .start(); //启动实时任务 + } +} + +``` + +## 运行 + +打包 + +```shell +mvn -Prelease-all -DskipTests clean install -U +``` + +运行 + +```shell + java -jar jarName mainClass +``` diff --git a/stream_sink.md b/docs/stream_sink/README.md similarity index 99% rename from stream_sink.md rename to docs/stream_sink/README.md index d7718865..fd560aa1 100644 --- a/stream_sink.md +++ b/docs/stream_sink/README.md @@ -80,14 +80,15 @@ DataStream dataStream=dataStream.toRocketmq(topic,tags,groupName,namesrvAddress); ``` -## kafka + +##kafka + ```java String bootstrapServers = ......;//kafka的bootstrap server String topic = ......; //kafka的topic DataStream dataStream = dataStream.toKafka(bootstrapServers, topic); ``` - # MQTT协议 ```java diff --git a/stream_source.md b/docs/stream_source/README.md similarity index 99% rename from stream_source.md rename to docs/stream_source/README.md index 8ae7b7e4..13831285 100644 --- a/stream_source.md +++ b/docs/stream_source/README.md @@ -17,6 +17,7 @@ String filepath = .....; //文件路径 DataStream dataStream = dataStreamSource.fromFile(filePath); ``` + 或者 ```java @@ -25,7 +26,6 @@ DataStream dataStream = dataStreamSource.fromFile(filePath, isJsonData); ``` - ## Rocketmq ```java @@ -36,6 +36,7 @@ DataStream dataStream = dataStreamSource.fromRocketmq(topic, groupName, namesrvAddress); ``` + 或者 ```java @@ -47,7 +48,9 @@ DataStream dataStream = dataStreamSource.fromRocketmq(topic, groupName, isJsonData, namesrvAddress); ``` + 或者 + ```java String topic = .....; //rocketmq 的topic @@ -59,7 +62,8 @@ ``` -## kafka +##kafka + ```java String bootstrapServers = ......;//kafka的bootstrap server String topic = ......; //kafka的topic @@ -68,7 +72,9 @@ Integer maxThread = 1; //客户端线程数 DataStream dataStream = dataStreamSource.fromKafka(bootstrapServers, topic, groupName, isJsonData, maxThread); ``` + 或者 + ```java String bootstrapServers = ......;//kafka的bootstrap server String topic = ......; //kafka的topic @@ -76,7 +82,9 @@ Boolean isJsonData = true; //是否json DataStream dataStream = dataStreamSource.fromKafka(bootstrapServers, topic, groupName, isJsonData); ``` + 或者 + ```java String bootstrapServers = ......;//kafka的bootstrap server String topic = ......; //kafka的topic @@ -84,10 +92,8 @@ DataStream dataStream = dataStreamSource.fromKafka(bootstrapServers, topic, groupName); ``` - - - # MQTT协议 + ```java String url = ......; @@ -96,7 +102,9 @@ DataStream dataStream = dataStreamSource.fromMqtt(url, cliientId, topic); ``` + 或者 + ```java String url = ......; @@ -107,7 +115,9 @@ DataStream dataStream = dataStreamSource.fromMqtt(url, cliientId, topic, username, password); ``` + 或者 + ````java String url = ......; @@ -124,6 +134,7 @@ ```` ##自定义Source + ````java DataStream dataStream = dataStreamSource.from(new ISource() {}); ```` diff --git a/stream_transform.md b/docs/stream_transform/README.md similarity index 100% rename from stream_transform.md rename to docs/stream_transform/README.md diff --git a/pom.xml b/pom.xml index b2792adc..200b6e32 100644 --- a/pom.xml +++ b/pom.xml @@ -1,4 +1,4 @@ - + - 4.0.0 @@ -26,9 +26,10 @@ 18 + org.apache.rocketmq rocketmq-streams - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT Apache RocketMQ Streams ${project.version} pom https://rocketmq.apache.org/ @@ -38,27 +39,35 @@ rocketmq-streams-dim rocketmq-streams-transport-minio rocketmq-streams-script - rocketmq-streams-configurable rocketmq-streams-serviceloader rocketmq-streams-filter rocketmq-streams-schedule rocketmq-streams-lease rocketmq-streams-db-operator - rocketmq-streams-dbinit + rocketmq-streams-db-init rocketmq-streams-window + rocketmq-streams-channel-metaq rocketmq-streams-clients rocketmq-streams-channel-rocketmq rocketmq-streams-channel-db rocketmq-streams-channel-http rocketmq-streams-state rocketmq-streams-examples - rocketmq-streams-checkpoint rocketmq-streams-connectors rocketmq-streams-channel-syslog rocketmq-streams-channel-es rocketmq-streams-channel-kafka rocketmq-streams-channel-mqtt - rocketmq-streams-cep + rocketmq-streams-channel-sls + rocketmq-streams-channel-openapi + rocketmq-streams-channel-view + rocketmq-streams-channel-tencent + rocketmq-streams-dispatcher + rocketmq-streams-tasks + rocketmq-streams-sts + rocketmq-streams-storage + rocketmq-streams-local-cache + rocketmq-streams-channel-huawei @@ -67,42 +76,54 @@ 1.13.0 2.11 false - 1.8 - 1.8 - 1.8 + 11 + 11 + 11 UTF-8 ${java.encoding} - 1.1 + 5.1.14.RELEASE 1.0-rc5 - 5.1.40 - 1.2.25 - 2.2.1 + 8.0.31 + 1.2.83 + 2.3.2 4.5.13 + 1.2 2.7 + 3.2.2 + 3.12.0 + 1.15 4.13.1 - 25.1-jre + 31.1-jre 2.1.8 3.2.0 - 6.6.4 - 4.9.3 + 8.3.2 + 5.0.0 5.4.0-2.0.0 3.5.2 2.8.5 0.1.9 2.7.0 2.12.4 - 1.2.3 3.0.10 - 6.6.4 - 3.2.1 - 2.5 7.4.0 1.1.0 1.2.2 5.3.0 2.56 + 1.6 1.7.36 + 1.7.36 + 1.6.0 + 2.5.0 + 0.6.74 + 0.3.11 + 0.6.33 + 3.2.0 + 1.2.8 + 1.0.0 + 4.5.1 + 3.10.2 @@ -128,7 +149,10 @@ build_without_test.sh NOTICE LICENSE - *.md + README.md + README-chinese.md + QUICKSTART.md + quick_start.md .github/** */target/** */*.iml @@ -199,17 +223,61 @@ - + + org.apache.rocketmq + rocketmq-streams-channel-tencentcloudapi + ${project.version} + + + org.apache.rocketmq + rocketmq-streams-ocsf + ${project.version} + + + org.apache.rocketmq + rocketmq-streams-shuffle-store + ${project.version} + + + org.apache.rocketmq + rocketmq-streams-local-cache + ${project.version} + + + org.apache.rocketmq + rocketmq-streams-dispatcher + ${project.version} + org.apache.rocketmq rocketmq-streams-commons ${project.version} + + org.apache.rocketmq + rocketmq-streams-channel-metaq + ${project.version} + + + org.apache.rocketmq + rocketmq-streams-channel-view + ${project.version} + + + org.apache.rocketmq + rocketmq-streams-sts + ${project.version} + org.apache.rocketmq rocketmq-streams-channel-es ${project.version} + + org.apache.rocketmq + rocketmq-streams-tasks + ${project.version} + org.apache.rocketmq rocketmq-streams-channel-mqtt @@ -217,7 +285,22 @@ org.apache.rocketmq - rocketmq-streams-dbinit + rocketmq-streams-db-init + ${project.version} + + + org.apache.rocketmq + rocketmq-streams-channel-openapi + ${project.version} + + + org.apache.rocketmq + rocketmq-streams-channel-tencent + ${project.version} + + + org.apache.rocketmq + rocketmq-streams-channel-huawei ${project.version} @@ -244,16 +327,6 @@ org.apache.rocketmq rocketmq-streams-configurable ${project.version} - - - ch.qos.logback - logback-classic - - - ch.qos.logback - logback-core - - org.apache.rocketmq @@ -284,22 +357,17 @@ org.apache.rocketmq rocketmq-streams-script ${project.version} - - - ch.qos.logback - logback-classic - - - ch.qos.logback - logback-core - - org.apache.rocketmq rocketmq-streams-script-python ${project.version} + + org.apache.rocketmq + rocketmq-streams-channel-rocketmq + ${project.version} + org.apache.rocketmq rocketmq-streams-serviceloader @@ -327,12 +395,12 @@ org.apache.rocketmq - rocketmq-streams-channel-rocketmq + rocketmq-streams-channel-kafka ${project.version} org.apache.rocketmq - rocketmq-streams-channel-kafka + rocketmq-streams-channel-sls ${project.version} @@ -345,24 +413,20 @@ rocketmq-streams-examples ${project.version} - - - - - org.apache.rocketmq - rocketmq-tools - ${rocketmq.version} + rocketmq-streams-localcache + ${project.version} + org.apache.rocketmq - rocketmq-common + rocketmq-client ${rocketmq.version} org.apache.rocketmq - rocketmq-client + rocketmq-tools ${rocketmq.version} @@ -370,45 +434,85 @@ rocketmq-acl ${rocketmq.version} - + + com.taobao.metaq.final + metaq-client + 4.0.2.Final + + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + + + + + com.alibaba.rocketmq + rocketmq-common + 4.0.2 + - + - - com.alibaba - fastjson - ${fastjson.version} + commons-io + commons-io + ${commons-io.version} - - junit - junit - ${junit.version} + commons-codec + commons-codec + ${commons-codec.version} - - commons-logging - commons-logging - ${commons-logging.version} + org.apache.commons + commons-lang3 + ${commons-lang3.version} - org.apache.rocketmq - commons - ${project.version} + commons-collections + commons-collections + ${commons-collections.version} + + + + + - commons-io - commons-io - ${commons-io.version} + org.slf4j + slf4j-api + ${slf4j-api.version} - org.slf4j slf4j-log4j12 ${slf4j-log4j12.version} + + org.elasticsearch.client + elasticsearch-rest-high-level-client + ${elasticsearch.version} + + + org.elasticsearch.client + elasticsearch-rest-client + ${elasticsearch.version} + + + + org.elasticsearch + elasticsearch + ${elasticsearch.version} + + + + + com.google.code.gson gson @@ -433,6 +537,25 @@ + + com.google.re2j + re2j + ${re2j.version} + + + + + com.alibaba + fastjson + ${fastjson.version} + + + + junit + junit + ${junit.version} + + com.lmax disruptor @@ -513,22 +636,11 @@ - - ch.qos.logback - logback-core - ${logback-core.version} - - - ch.qos.logback - logback-classic - ${logback-core.version} - io.minio minio ${minio.version} - org.rocksdb rocksdbjni @@ -540,53 +652,88 @@ 0.9.48 - commons-collections - commons-collections - ${commons-collections.version} - - - commons-lang - commons-lang - ${commons-lang.version} + org.apache.kafka + kafka_2.12 + ${kafka.version} - org.elasticsearch.client - elasticsearch-rest-high-level-client - ${elasticsearch.version} + de.ruedigermoeller + fst + ${fst.version} - org.elasticsearch.client - elasticsearch-rest-client - ${elasticsearch.version} + com.esotericsoftware + kryo + ${kryo.version} - - org.elasticsearch - elasticsearch - ${elasticsearch.version} + net.agkn + hll + ${hll.version} - - org.apache.kafka - kafka_2.12 - ${kafka.version} - - org.eclipse.paho org.eclipse.paho.client.mqttv3 ${paho.version} - + - de.ruedigermoeller - fst - ${fst.version} + com.google.protobuf + protobuf-java + ${protobuf.version} - com.esotericsoftware - kryo - ${kryo.version} + com.aliyun.openservices + aliyun-log + ${aliyun-log.version} + + + com.aliyun.openservices + aliyun-log-producer + ${aliyun-log-producer.version} + + + com.aliyun.openservices + loghub-client-lib + ${loghub.version} + + + + com.aliyun + aliyun-java-sdk-sts-internal + ${sts.version} + + + com.aliyun.ram + ram-auth-client + ${ram-auth-client.version} + + + com.aliyun + aliyun-java-sdk-ram-inner + ${ram-inner.version} + + + com.aliyun + aliyun-java-sdk-core + ${aliyun-java-sdk-core.version} + + + com.univocity + univocity-parsers + 2.9.1 + + + com.aliyun.oss + aliyun-sdk-oss + ${oss.version} + + + + org.springframework.boot + spring-boot-loader + 2.5.12 diff --git a/quick_start.md b/quick_start.md index a60dbb95..adcb529d 100644 --- a/quick_start.md +++ b/quick_start.md @@ -1,46 +1,84 @@ -# 快速开发 +## rocketmq-streams 快速搭建 +--- -## 引入相关的jar包 +### 前言 -```xml +本文档主要介绍如何基于rocketmq-streams快速搭建流处理任务,搭建过程中某些例子会用到rocketmq,可以参考[rocketmq搭建文档](https://rocketmq.apache.org/docs/quick-start/) - - org.apache.rocketmq - rocketmq-streams-clients - +### 1、源码构建 -``` +#### 1.1、构建环境 -## 开发实时应用程序 +- JDK 1.8 and above +- Maven 3.2 and above -```java +#### 1.2、构建Rocketmq-streams -public class RocketmqExample { +```shell +git clone https://github.com/apache/rocketmq-streams.git +cd rocketmq-streams +mvn clean -DskipTests install -U - public static void main(String[] args) { +``` - DataStreamSource dataStream = StreamBuilder.dataStream("test_namespace", "graph_pipeline"); +### 2、基于rocketmq-streams创建应用 - dataStream - .fromFile("data.csv", false) //构建实时任务的数据源 - .map(message -> message.split(",")) //构建实时任务处理的逻辑过程 - .toPrint(1) //构建实时任务的输出 - .start(); //启动实时任务 - } -} +#### 2.1、pom依赖 +```xml + + + org.apache.rocketmq + rocketmq-streams-clients + ``` -## 运行 +#### 2.2、shade clients依赖包 -打包 +```xml -```shell -mvn -Prelease-all -DskipTests clean install -U + + + + org.apache.maven.plugins + maven-shade-plugin + 3.2.1 + + + package + + shade + + + false + true + + + org.apache.rocketmq:rocketmq-streams-clients + + + + + + + + ``` -运行 +#### 2.3、编写业务代码 + +快速编写一个统计页面点击次数的小程序:Please see the [rocketmq-streams-examples](rocketmq-streams-examples/README.md) + +#### 2.4、运行 + +- 前提:在从rocketmq中读取数据做流处理时,需要运行topic在rocketmq中自动创建,因为做groupBy操作时,需要用到rocketmq作为shuffle数据的读写目的地。 +- 命令: -```shell - java -jar jarName mainClass ``` + java -jar XXXX-shade.jar \ + -Dlog4j.level=ERROR \ + -Dlog4j.home=/logs \ + -Xms1024m \ + -Xmx1024m +``` + diff --git a/rocketmq-streams-channel-db/pom.xml b/rocketmq-streams-channel-db/pom.xml index b31b7110..7a5ac938 100755 --- a/rocketmq-streams-channel-db/pom.xml +++ b/rocketmq-streams-channel-db/pom.xml @@ -15,14 +15,14 @@ See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 org.apache.rocketmq rocketmq-streams - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT rocketmq-streams-channel-db ROCKETMQ STREAMS :: channel-db @@ -30,7 +30,8 @@ org.apache.rocketmq - rocketmq-streams-db-operator + rocketmq-streams-connectors + diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/CycleSplit.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/CycleSplit.java index 706f07e1..4c9ac5a4 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/CycleSplit.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/CycleSplit.java @@ -26,11 +26,11 @@ public class CycleSplit extends DynamicMultipleDBSplit implements Serializable { private static final long serialVersionUID = 4309494143340650195L; String cyclePeriod; - public CycleSplit(){ + public CycleSplit() { } - public CycleSplit(String version){ + public CycleSplit(String version) { this.cyclePeriod = version; } diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/DynamicMultipleDBSplit.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/DynamicMultipleDBSplit.java index bf3662ea..32478373 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/DynamicMultipleDBSplit.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/DynamicMultipleDBSplit.java @@ -56,7 +56,6 @@ public String getQueueId() { return logicTableName + "_" + suffix; } - @Override public String getQueue() { return logicTableName + "_" + suffix; diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSink.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSink.java index 24b4ad71..4c9bd469 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSink.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSink.java @@ -24,14 +24,12 @@ import java.sql.ResultSet; import java.util.List; import java.util.Set; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.channel.IChannel; import org.apache.rocketmq.streams.common.channel.sink.AbstractSink; import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageCache; import org.apache.rocketmq.streams.common.channel.sinkcache.impl.MessageCache; -import org.apache.rocketmq.streams.common.component.AbstractComponent; import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.metadata.MetaData; import org.apache.rocketmq.streams.common.metadata.MetaDataField; @@ -41,35 +39,32 @@ import org.apache.rocketmq.streams.db.driver.DriverBuilder; import org.apache.rocketmq.streams.db.driver.JDBCDriver; import org.apache.rocketmq.streams.db.driver.orm.ORMUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * 主要用于写db,输入可以是一个insert/replace 模版,也可以是metadata对象,二者选一即可。都支持批量插入,提高吞吐 sql 模版:insert into table(column1,column2,column3)values('#{var1}',#{var2},'#{var3}') MetaData:主要是描述每个字段的类型,是否必须 二者选一个即可。sql模式,系统会把一批(batchSize)数据拼成一个大sql。metadata模式,基于字段描述,最终也是拼成一个大sql */ public class DBSink extends AbstractSink { - static final Log logger = LogFactory.getLog(DBSink.class); - public static final String SQL_MODE_DEFAULT = "default"; public static final String SQL_MODE_REPLACE = "replace"; public static final String SQL_MODE_IGNORE = "ignore"; - - @ENVDependence protected String jdbcDriver = AbstractComponent.DEFAULT_JDBC_DRIVER; + static final Logger LOGGER = LoggerFactory.getLogger(DBSink.class); + /** + * 解析出insert value数据部分,对于批量的插入,效果会更佳 + */ + private static final String VALUES_NAME = "values"; + @ENVDependence protected String jdbcDriver = ConfigurationKey.DEFAULT_JDBC_DRIVER; @ENVDependence protected String url; @ENVDependence protected String userName; @ENVDependence protected String tableName; //指定要插入的数据表 @ENVDependence protected String password; @ENVDependence protected String sqlMode; - - protected MetaData metaData;//可以指定meta data,和insertSQL二选一 - protected String insertSQLTemplate;//完成插入部分的工作,和metadata二选一。insert into table(column1,column2,column3)values('#{var1}',#{var2},'#{var3}') - protected boolean openSqlCache = true; - protected transient IMessageCache sqlCache;//cache sql, batch submit sql - boolean isMultiple = false; //是否多表 - /** * db串多数是名字,可以取个名字前缀,如果值为空,默认为此类的name,name为空,默认为简单类名 * @@ -79,7 +74,7 @@ public class DBSink extends AbstractSink { public DBSink(String insertSQL, String dbInfoNamePrefix) { setType(IChannel.TYPE); if (StringUtil.isEmpty(dbInfoNamePrefix)) { - dbInfoNamePrefix = getConfigureName(); + dbInfoNamePrefix = getName(); } if (StringUtil.isEmpty(dbInfoNamePrefix)) { dbInfoNamePrefix = this.getClass().getSimpleName(); @@ -115,7 +110,7 @@ public DBSink(String url, String userName, String password, String tableName, St @Override protected boolean initConfigurable() { if (this.metaData == null) { try { - Class.forName("com.mysql.jdbc.Driver"); + Class.forName(this.jdbcDriver); if (StringUtil.isNotEmpty(this.tableName)) { Connection connection = DriverManager.getConnection(this.url, this.userName, this.password); DatabaseMetaData connectionMetaData = connection.getMetaData(); @@ -131,6 +126,7 @@ public DBSink(String url, String userName, String password, String tableName, St List insertFields = Lists.newArrayList(); List insertValues = Lists.newArrayList(); List duplicateKeys = Lists.newArrayList(); + fieldList.forEach(field -> { String fieldName = field.getFieldName(); insertFields.add("`" + fieldName + "`"); @@ -154,7 +150,7 @@ public DBSink(String url, String userName, String password, String tableName, St this.sqlCache = new MessageCache<>(sqls -> { JDBCDriver dataSource = DriverBuilder.createDriver(jdbcDriver, url, userName, password); try { - dataSource.executSqls(sqls); + dataSource.executeSqls(sqls); } catch (Exception e) { e.printStackTrace(); throw new RuntimeException(e); @@ -224,15 +220,10 @@ protected void executeSQL(JDBCDriver dbDataSource, List sqls) { this.sqlCache.addCache(sql); } } else { - dbDataSource.executSqls(sqls); + dbDataSource.executeSqls(sqls); } } - /** - * 解析出insert value数据部分,对于批量的插入,效果会更佳 - */ - private static final String VALUES_NAME = "values"; - protected String parseInsertValues(String insertSQL) { int start = insertSQL.toLowerCase().indexOf(VALUES_NAME); if (start == -1) { @@ -306,14 +297,6 @@ public void setSqlMode(String sqlMode) { this.sqlMode = sqlMode; } - public MetaData getMetaData() { - return metaData; - } - - public void setMetaData(MetaData metaData) { - this.metaData = metaData; - } - public boolean isOpenSqlCache() { return openSqlCache; } @@ -352,11 +335,11 @@ private final String getCreateTableSqlFromOther(String sourceTableName, String t String createTableSql = MetaDataUtils.getCreateTableSqlByTableName(url, userName, password, sourceTableName); if (createTableSql == null) { String errMsg = String.format("source table is not exist. multiple db sink must be dependency logic table meta for auto create sub table. logic table name is ", sourceTableName); - logger.error(errMsg); + LOGGER.error(errMsg); throw new RuntimeException(errMsg); } createTableSql = createTableSql.replace(sourceTableName, targetTableName); - logger.info(String.format("createTableSql is %s", createTableSql)); + LOGGER.info(String.format("createTableSql is %s", createTableSql)); return createTableSql; } diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSinkBuilder.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSinkBuilder.java index 0c36b629..53ae49d9 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSinkBuilder.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSinkBuilder.java @@ -24,6 +24,8 @@ 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; +import org.apache.rocketmq.streams.db.source.DBSource; @AutoService(IChannelBuilder.class) @ServiceName(value = DBSinkBuilder.TYPE, aliasName = "db") @@ -32,15 +34,8 @@ public class DBSinkBuilder implements IChannelBuilder { @Override public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { - DBSink sink = new DBSink(); - JSONObject proJson = createFormatProperty(properties); - sink.setUrl(proJson.getString("url")); - sink.setUserName(proJson.getString("userName")); - sink.setPassword(proJson.getString("password")); - sink.setTableName(proJson.getString("tableName")); - sink.setSqlMode(proJson.getString("sqlMode")); - sink.setMetaData(metaData); - return sink; + return (ISink) ConfigurableUtil.create(DBSink.class.getName(), namespace, name, createFormatProperty(properties), null); + } /** @@ -66,7 +61,7 @@ protected JSONObject createFormatProperty(Properties properties) { @Override public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) { - throw new RuntimeException("can not support this method"); + return (ISource) ConfigurableUtil.create(DBSource.class.getName(), namespace, name, createFormatProperty(properties), null); } @Override diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DynamicMultipleDBSink.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DynamicMultipleDBSink.java index d57fe015..c1a24aee 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DynamicMultipleDBSink.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DynamicMultipleDBSink.java @@ -17,8 +17,6 @@ package org.apache.rocketmq.streams.db.sink; import org.apache.rocketmq.streams.common.channel.split.ISplit; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.functions.MultiTableSplitFunction; import org.apache.rocketmq.streams.db.DynamicMultipleDBSplit; @@ -26,7 +24,7 @@ /** * @description */ -public class DynamicMultipleDBSink extends AbstractMultiTableSink implements IAfterConfigurableRefreshListener { +public class DynamicMultipleDBSink extends AbstractMultiTableSink { private static final long serialVersionUID = -4570659943689358381L; String logicTableName; @@ -35,6 +33,12 @@ public class DynamicMultipleDBSink extends AbstractMultiTableSink implements IAf public DynamicMultipleDBSink() { } + public DynamicMultipleDBSink(String url, String userName, String password, String logicTableName, String fieldName) { + super(url, userName, password); + this.logicTableName = logicTableName; + this.fieldName = fieldName; + } + public String getLogicTableName() { return logicTableName; } @@ -51,12 +55,6 @@ public void setFieldName(String fieldName) { this.fieldName = fieldName; } - public DynamicMultipleDBSink(String url, String userName, String password, String logicTableName, String fieldName) { - super(url, userName, password); - this.logicTableName = logicTableName; - this.fieldName = fieldName; - } - @Override protected String createTableName(String splitId) { return this.multiTableSplitFunction.createTableFromSplitId(splitId); @@ -68,7 +66,7 @@ protected String createTableName(String splitId) { } @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { + protected boolean initConfigurable() { if (this.multiTableSplitFunction == null) { @@ -85,6 +83,7 @@ public String createTableFromSplitId(String splitId) { }; } + return true; } } diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/EnhanceDBSink.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/EnhanceDBSink.java index d7897032..5dad2d0f 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/EnhanceDBSink.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/EnhanceDBSink.java @@ -21,17 +21,13 @@ import java.util.ArrayList; import java.util.Date; import java.util.List; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.channel.IChannel; import org.apache.rocketmq.streams.common.channel.sink.AbstractSink; import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageCache; import org.apache.rocketmq.streams.common.channel.sinkcache.impl.MessageCache; import org.apache.rocketmq.streams.common.channel.source.systemmsg.ChangeTableNameMessage; -import org.apache.rocketmq.streams.common.component.AbstractComponent; -import org.apache.rocketmq.streams.common.component.ComponentCreator; import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.interfaces.ISystemMessage; import org.apache.rocketmq.streams.common.metadata.MetaData; @@ -41,27 +37,25 @@ import org.apache.rocketmq.streams.db.driver.orm.ORMUtil; import org.apache.rocketmq.streams.db.sink.sqltemplate.ISqlTemplate; import org.apache.rocketmq.streams.db.sink.sqltemplate.SqlTemplateFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * @description enhance db sink, support atomic sink and multiple sink */ public class EnhanceDBSink extends AbstractSink { - static final Log logger = LogFactory.getLog(EnhanceDBSink.class); + static final Logger LOGGER = LoggerFactory.getLogger(EnhanceDBSink.class); protected MetaData metaData;//可以指定meta data,和insertSQL二选一 protected String tableName; //指定要插入的数据表 - boolean isAtomic = false; //是否原子写入 - boolean isMultiple = false; //是否多表 protected boolean isContainsId = false; protected boolean openSqlCache = true; - /** * for atomic sink. default is null */ protected String tmpTableName; - @ENVDependence - protected String jdbcDriver = AbstractComponent.DEFAULT_JDBC_DRIVER; + protected String jdbcDriver = ConfigurationKey.DEFAULT_JDBC_DRIVER; @ENVDependence protected String url; @ENVDependence @@ -70,10 +64,10 @@ public class EnhanceDBSink extends AbstractSink { protected String password; @ENVDependence protected String sqlMode; - protected transient IMessageCache sqlCache;//cache sql, batch submit sql - protected transient ISqlTemplate iSqlTemplate; + boolean isAtomic = false; //是否原子写入 + boolean isMultiple = false; //是否多表 public EnhanceDBSink() { this(null, null, null, null); @@ -100,47 +94,45 @@ public EnhanceDBSink(String url, String userName, String password, String tableN @Override protected boolean initConfigurable() { - if(isAtomic && isMultiple){ + if (isAtomic && isMultiple) { String errMsg = String.format("atomic is not support multiple."); - logger.error(errMsg); + LOGGER.error(errMsg); throw new RuntimeException(errMsg); } //如果是多表, 根据逻辑表名创建分区表 - if(isMultiple){ + if (isMultiple) { createMultiTable(); } //如果是原子写入,根据结果表创建临时表 - if(isAtomic){ + if (isAtomic) { createTmpTable(); } //如果未设置metadata, 则从db搜索元数据, 创建metadata - if(metaData == null){ + if (metaData == null) { createMetaData(); } - if(iSqlTemplate == null){ + if (iSqlTemplate == null) { try { iSqlTemplate = SqlTemplateFactory.newSqlTemplate(sqlMode, metaData, isContainsId); } catch (Exception e) { - e.printStackTrace(); - logger.error(e); + LOGGER.error("get sql template error", e); } } - if(openSqlCache){ + if (openSqlCache) { initSqlCache(); } return super.initConfigurable(); } - - private void initSqlCache(){ + private void initSqlCache() { this.sqlCache = new MessageCache<>(sqls -> { JDBCDriver dataSource = DriverBuilder.createDriver(jdbcDriver, url, userName, password); try { - dataSource.executSqls(sqls); + dataSource.executeSqls(sqls); } catch (Exception e) { e.printStackTrace(); throw new RuntimeException(e); @@ -157,53 +149,52 @@ private void initSqlCache(){ /** * create */ - private void createMetaData(){ + private void createMetaData() { String realInsertTableName = isAtomic ? tmpTableName : tableName; metaData = MetaDataUtils.createMetaData(url, userName, password, realInsertTableName); } - private void createMultiTable(){ + private void createMultiTable() { String logicTable = subStrLogicTableName(tableName); copyAndCreateTableSchema(logicTable, tableName); } - private void createTmpTable(){ + private void createTmpTable() { String tmpTable = createTmpTableName(tableName); copyAndCreateTableSchema(tableName, tmpTable); } - private void copyAndCreateTableSchema(String sourceTableName, String targetTableName){ + private void copyAndCreateTableSchema(String sourceTableName, String targetTableName) { List tables = MetaDataUtils.listTableNameByPattern(url, userName, password, targetTableName); - if(tables == null || tables.size() == 0){ + if (tables == null || tables.size() == 0) { String createTableSql = getCreateTableSqlFromOther(sourceTableName, tableName); createTable(createTableSql); } } - private final String getCreateTableSqlFromOther(String sourceTableName, String targetTableName){ + private final String getCreateTableSqlFromOther(String sourceTableName, String targetTableName) { String createTableSql = MetaDataUtils.getCreateTableSqlByTableName(url, userName, password, sourceTableName); createTableSql = createTableSql.replace(sourceTableName, targetTableName); - logger.info(String.format("createTableSql is %s", createTableSql)); + LOGGER.info(String.format("createTableSql is %s", createTableSql)); return createTableSql; } - private final String subStrLogicTableName(String realTableName){ + private final String subStrLogicTableName(String realTableName) { int len = realTableName.lastIndexOf("_"); String logicTableName = realTableName.substring(0, len); return logicTableName; } - private final String createTmpTableName(String tableName){ + private final String createTmpTableName(String tableName) { return "tmp" + "_" + tableName; } /** - * * @param createTableSql */ - private final void createTable(String createTableSql){ + private final void createTable(String createTableSql) { ORMUtil.executeSQL(url, userName, password, createTableSql, null); } @@ -214,14 +205,15 @@ protected boolean batchInsert(List messages) { return false; } - private String genInsertSql(List messages){ + private String genInsertSql(List messages) { String sql = iSqlTemplate.createSql(convertJsonObjectFromMessage(messages)); return sql; } - protected List convertJsonObjectFromMessage(List messageList){ + @Override + protected List convertJsonObjectFromMessage(List messageList) { List messages = new ArrayList<>(); - for(IMessage message:messageList){ + for (IMessage message : messageList) { messages.add(message.getMessageBody()); } return messages; @@ -234,7 +226,7 @@ protected void executeSQL(String sql) { JDBCDriver dbDataSource = DriverBuilder.createDriver(jdbcDriver, url, userName, password); try { dbDataSource.execute(sql); - }finally { + } finally { dbDataSource.destroy(); } } @@ -336,21 +328,21 @@ public void setJdbcDriver(String jdbcDriver) { this.jdbcDriver = jdbcDriver; } - public void rename(String suffix){ + public void rename(String suffix) { SimpleDateFormat format = new SimpleDateFormat("yyyyMMddHHmmss"); String rename1 = String.format("rename table %s to %s", tableName, tmpTableName.replace("tmp_", "re_") + "_" + suffix + "_" + format.format(new Date())); String rename2 = String.format("rename table %s to %s", tmpTableName, tableName); - logger.info(String.format("exec rename1 %s", rename1)); - logger.info(String.format("exec rename2 %s", rename2)); + LOGGER.info(String.format("exec rename1 %s", rename1)); + LOGGER.info(String.format("exec rename2 %s", rename2)); ORMUtil.executeSQL(rename1, null); ORMUtil.executeSQL(rename2, null); } @Override - public void atomicSink(ISystemMessage iMessage){ - if(isAtomic){ - ChangeTableNameMessage message = (ChangeTableNameMessage)iMessage; + public void atomicSink(ISystemMessage iMessage) { + if (isAtomic) { + ChangeTableNameMessage message = (ChangeTableNameMessage) iMessage; rename(message.getScheduleCycle()); try { super.finish(); @@ -359,12 +351,4 @@ public void atomicSink(ISystemMessage iMessage){ } } } - - public static boolean isAtomicConfiguration(){ - String isAtomicDBSink = ComponentCreator.getProperties().getProperty(ConfigureFileKey.IS_ATOMIC_DB_SINK); - if(isAtomicDBSink == null){ - return false; - } - return Boolean.parseBoolean(isAtomicDBSink); - } } diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/SelfMultiTableSink.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/SelfMultiTableSink.java index 709fb77b..33bfa16f 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/SelfMultiTableSink.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/SelfMultiTableSink.java @@ -17,22 +17,15 @@ package org.apache.rocketmq.streams.db.sink; import org.apache.rocketmq.streams.common.channel.split.ISplit; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.functions.MultiTableSplitFunction; -import org.apache.rocketmq.streams.common.utils.Base64Utils; -import org.apache.rocketmq.streams.common.utils.InstantiationUtil; -public class SelfMultiTableSink extends AbstractMultiTableSink implements IAfterConfigurableRefreshListener { - protected String multiTableSplitFunctionSerializeValue;//用户自定义的operator的序列化字节数组,做了base64解码 +public class SelfMultiTableSink extends AbstractMultiTableSink { protected transient MultiTableSplitFunction multiTableSplitFunction; public SelfMultiTableSink(String url, String userName, String password, MultiTableSplitFunction multiTableSplitFunction) { super(url, userName, password); this.multiTableSplitFunction = multiTableSplitFunction; - byte[] bytes = InstantiationUtil.serializeObject(multiTableSplitFunction); - multiTableSplitFunctionSerializeValue = Base64Utils.encode(bytes); } @Override @@ -45,9 +38,4 @@ protected String createTableName(String splitId) { return multiTableSplitFunction.createSplit(message); } - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - byte[] bytes = Base64Utils.decode(multiTableSplitFunctionSerializeValue); - this.multiTableSplitFunction = InstantiationUtil.deserializeObject(bytes); - } } diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/sqltemplate/ISqlTemplate.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/sqltemplate/ISqlTemplate.java index 9d2007ae..f02c4ed6 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/sqltemplate/ISqlTemplate.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/sqltemplate/ISqlTemplate.java @@ -28,7 +28,7 @@ public interface ISqlTemplate { public static final String SQL_MODE_DUPLICATE = "duplicate"; public static final String SQL_MODE_IGNORE = "ignore"; - static final String[] SUPPORTS = new String[]{ + static final String[] SUPPORTS = new String[] { ISqlTemplate.SQL_MODE_DEFAULT, SQL_MODE_DUPLICATE, SQL_MODE_IGNORE @@ -37,10 +37,11 @@ public interface ISqlTemplate { /** * create sql prefix * eg : - * insert into table(`a`,`b`,`c`) - * or insert ignore into table(`a`, `b`, `c`) - * or insert into table(`a`,`b`,`c`) on duplicate key update + * insert into table(`a`,`b`,`c`) + * or insert ignore into table(`a`, `b`, `c`) + * or insert into table(`a`,`b`,`c`) on duplicate key update * `a` = values(`a`), `b` = values(`b`), `c` = values(`c`) + * * @return */ void initSqlTemplate(); diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/sqltemplate/MysqlInsertIgnoreIntoSqlTemplate.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/sqltemplate/MysqlInsertIgnoreIntoSqlTemplate.java index cbd50a16..04f8c90d 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/sqltemplate/MysqlInsertIgnoreIntoSqlTemplate.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/sqltemplate/MysqlInsertIgnoreIntoSqlTemplate.java @@ -22,7 +22,7 @@ /** * @description */ -public class MysqlInsertIgnoreIntoSqlTemplate extends MysqlInsertIntoSqlTemplate{ +public class MysqlInsertIgnoreIntoSqlTemplate extends MysqlInsertIntoSqlTemplate { public MysqlInsertIgnoreIntoSqlTemplate(MetaData metaData, boolean isContainsId) { super(metaData, isContainsId); @@ -30,7 +30,7 @@ public MysqlInsertIgnoreIntoSqlTemplate(MetaData metaData, boolean isContainsId) @Override public void initSqlTemplate() { - if(sqlPrefix != null){ + if (sqlPrefix != null) { return; } sqlPrefix = SQLUtil.createInsertIgnoreSegment(metaData, isContainsId); diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/sqltemplate/MysqlInsertIntoWithDuplicateKeySqlTemplate.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/sqltemplate/MysqlInsertIntoWithDuplicateKeySqlTemplate.java index c0150afa..d0bee49c 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/sqltemplate/MysqlInsertIntoWithDuplicateKeySqlTemplate.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/sqltemplate/MysqlInsertIntoWithDuplicateKeySqlTemplate.java @@ -21,13 +21,8 @@ import org.apache.rocketmq.streams.common.metadata.MetaData; import org.apache.rocketmq.streams.common.utils.SQLUtil; -/** - * @author zengyu.cw - * @program apache-rocketmq-streams-fork - * @create 2021-09-28 13:49:40 - * @description createDuplicateKeyUpdateSegment - */ -public class MysqlInsertIntoWithDuplicateKeySqlTemplate extends MysqlInsertIntoSqlTemplate{ + +public class MysqlInsertIntoWithDuplicateKeySqlTemplate extends MysqlInsertIntoSqlTemplate { String sqlSuffix; @@ -36,17 +31,17 @@ public MysqlInsertIntoWithDuplicateKeySqlTemplate(MetaData metaData, boolean isC } @Override - public void initSqlTemplate(){ - if(sqlPrefix == null){ + public void initSqlTemplate() { + if (sqlPrefix == null) { sqlPrefix = SQLUtil.createInsertSegment(metaData, isContainsId); } - if(sqlSuffix == null){ + if (sqlSuffix == null) { sqlSuffix = SQLUtil.createDuplicateKeyUpdateSegment(metaData, isContainsId); } } @Override - public String createSql(List> rows){ + public String createSql(List> rows) { initSqlTemplate(); return String.join(" ", sqlPrefix, SQLUtil.createValuesSegment(metaData, rows, isContainsId), sqlSuffix); } diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/sqltemplate/SqlTemplateFactory.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/sqltemplate/SqlTemplateFactory.java index 8837c9c6..00e86f68 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/sqltemplate/SqlTemplateFactory.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/sqltemplate/SqlTemplateFactory.java @@ -26,13 +26,13 @@ public class SqlTemplateFactory { public static ISqlTemplate newSqlTemplate(String type, MetaData metaData, boolean isContainsId) throws Exception { - if(ISqlTemplate.SQL_MODE_DEFAULT.equalsIgnoreCase(type)){ + if (ISqlTemplate.SQL_MODE_DEFAULT.equalsIgnoreCase(type)) { return new MysqlInsertIntoSqlTemplate(metaData, isContainsId); - }else if(ISqlTemplate.SQL_MODE_DUPLICATE.equalsIgnoreCase(type)){ + } else if (ISqlTemplate.SQL_MODE_DUPLICATE.equalsIgnoreCase(type)) { return new MysqlInsertIntoWithDuplicateKeySqlTemplate(metaData, isContainsId); - }else if(ISqlTemplate.SQL_MODE_IGNORE.equalsIgnoreCase(type)){ + } else if (ISqlTemplate.SQL_MODE_IGNORE.equalsIgnoreCase(type)) { return new MysqlInsertIgnoreIntoSqlTemplate(metaData, isContainsId); - }else{ + } else { throw new Exception(String.format("unsupported type %s, only support %s. ", type, Arrays.toString(ISqlTemplate.SUPPORTS))); } } diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/source/AbstractDynamicSplitSource.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/source/AbstractDynamicSplitSource.java new file mode 100644 index 00000000..fb916949 --- /dev/null +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/source/AbstractDynamicSplitSource.java @@ -0,0 +1,45 @@ +/* + * 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.db.source; + +import java.util.ArrayList; +import java.util.List; +import org.apache.rocketmq.streams.common.channel.split.CommonSplit; +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.connectors.source.AbstractPullSource; + +; + +public abstract class AbstractDynamicSplitSource extends AbstractPullSource { + protected int splitCount = 1; + + @Override public List> fetchAllSplits() { + List> splits = new ArrayList<>(); + for (int i = 0; i < 1; i++) { + splits.add(new CommonSplit(i + "")); + } + return splits; + } + + public int getSplitCount() { + return splitCount; + } + + public void setSplitCount(int splitCount) { + this.splitCount = splitCount; + } +} diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/CycleDynamicMultipleDBScanSource.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/source/CycleDynamicMultipleDBScanSource.java similarity index 87% rename from rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/CycleDynamicMultipleDBScanSource.java rename to rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/source/CycleDynamicMultipleDBScanSource.java index 561b48f2..20003462 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/CycleDynamicMultipleDBScanSource.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/source/CycleDynamicMultipleDBScanSource.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.connectors.source; +package org.apache.rocketmq.streams.db.source; import com.alibaba.fastjson.JSONObject; import java.io.Serializable; @@ -24,8 +24,6 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; 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.source.systemmsg.ChangeTableNameMessage; @@ -40,6 +38,8 @@ import org.apache.rocketmq.streams.connectors.source.filter.CycleSchedule; import org.apache.rocketmq.streams.connectors.source.filter.CycleScheduleFilter; import org.apache.rocketmq.streams.db.CycleSplit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * @description @@ -47,7 +47,7 @@ public class CycleDynamicMultipleDBScanSource extends DynamicMultipleDBScanSource implements IBoundedSource, Serializable { private static final long serialVersionUID = 6840988298037061128L; - private static final Log logger = LogFactory.getLog(CycleDynamicMultipleDBScanSource.class); + private static final Logger LOGGER = LoggerFactory.getLogger(CycleDynamicMultipleDBScanSource.class); Map initReaderMap = new ConcurrentHashMap<>(); CycleSchedule.Cycle cycle; @@ -62,6 +62,12 @@ public CycleDynamicMultipleDBScanSource(CycleSchedule.Cycle cycle) { this.cycle = cycle; } + public static String createKey(ISource iSource) { + AbstractSource source = (AbstractSource) iSource; + CycleSchedule.Cycle cycle = ((CycleDynamicMultipleDBScanSource) iSource).getCycle(); + return MapKeyUtil.createKey(source.getNameSpace(), source.getGroupName(), source.getName(), source.getTopic(), cycle.getCycleDateStr()); + } + public AtomicInteger getSize() { return size; } @@ -75,7 +81,7 @@ public void setSize(AtomicInteger size) { */ //todo @Override - public synchronized List fetchAllSplits() { + public synchronized List> fetchAllSplits() { if (this.filter == null) { filter = new CycleScheduleFilter(cycle.getAllPattern()); @@ -88,13 +94,13 @@ public synchronized List fetchAllSplits() { String sourceName = createKey(this); List tableNames = MetaDataUtils.listTableNameByPattern(url, userName, password, logicTableName + "%"); - logger.info(String.format("load all logic table : %s", Arrays.toString(tableNames.toArray()))); + LOGGER.info(String.format("load all logic table : %s", Arrays.toString(tableNames.toArray()))); Iterator it = tableNames.iterator(); while (it.hasNext()) { String s = it.next(); String suffix = s.replace(logicTableName + "_", ""); if (filter.filter(sourceName, logicTableName, suffix)) { - logger.info(String.format("filter add %s", s)); + LOGGER.info(String.format("filter add %s", s)); CycleSplit split = new CycleSplit(); split.setLogicTableName(logicTableName); split.setSuffix(suffix); @@ -106,7 +112,7 @@ public synchronized List fetchAllSplits() { size.incrementAndGet(); } } else { - logger.info(String.format("filter remove %s", s)); + LOGGER.info(String.format("filter remove %s", s)); it.remove(); } } @@ -129,8 +135,8 @@ public void finish() { for (Map.Entry entry : initReaderMap.entrySet()) { String key = entry.getKey(); Boolean value = entry.getValue(); - if (value == false) { - logger.error(String.format("split[%s] reader is not finish, exit with error. ", key)); + if (!value) { + LOGGER.error(String.format("split[%s] reader is not finish, exit with error. ", key)); } } this.initReaderMap.clear(); @@ -142,9 +148,6 @@ public void finish() { @Override public boolean isFinished() { List readerStatuses = ReaderStatus.queryReaderStatusListBySourceName(createKey(this)); - if (readerStatuses == null) { - return false; - } return readerStatuses.size() == size.get(); } @@ -154,20 +157,20 @@ protected ISplitReader createSplitReader(ISplit iSplit) { } private void sendChangeTableNameMessage() { - logger.info(String.format("start send change table name message.")); + LOGGER.info(String.format("start send change table name message.")); ChangeTableNameMessage changeTableNameMessage = new ChangeTableNameMessage(); changeTableNameMessage.setScheduleCycle(cycle.getCycleDateStr()); Message message = createMessage(new JSONObject(), null, null, false); message.setSystemMessage(changeTableNameMessage); message.getHeader().setSystemMessage(true); executeMessage(message); - logger.info(String.format("finish send change table name message.")); + LOGGER.info(String.format("finish send change table name message.")); } @Override public synchronized void boundedFinishedCallBack(ISplit iSplit) { this.initReaderMap.put(iSplit.getQueueId(), true); - logger.info(String.format("current map is %s, key is %s. ", initReaderMap, iSplit.getQueueId())); + LOGGER.info(String.format("current map is %s, key is %s. ", initReaderMap, iSplit.getQueueId())); if (statusCheckerStart.compareAndSet(false, true)) { Thread thread = new Thread(new Runnable() { @Override @@ -175,7 +178,7 @@ public void run() { while (!isFinished()) { ThreadUtil.sleep(3 * 1000); } - logger.info(String.format("source will be closed.")); + LOGGER.info(String.format("source will be closed.")); sendChangeTableNameMessage(); //下发修改name的消息 ThreadUtil.sleep(1 * 1000); finish(); @@ -204,10 +207,4 @@ public synchronized int getTotalReader() { return size.get(); } - public static String createKey(ISource iSource) { - AbstractSource source = (AbstractSource) iSource; - CycleSchedule.Cycle cycle = ((CycleDynamicMultipleDBScanSource) iSource).getCycle(); - return MapKeyUtil.createKey(source.getNameSpace(), source.getGroupName(), source.getConfigureName(), source.getTopic(), cycle.getCycleDateStr()); - } - } diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/DBScanReader.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/source/DBScanReader.java similarity index 76% rename from rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/DBScanReader.java rename to rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/source/DBScanReader.java index 268e891e..5558b9df 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/DBScanReader.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/source/DBScanReader.java @@ -14,56 +14,70 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.connectors.reader; +package org.apache.rocketmq.streams.db.source; import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.JSONObject; import java.io.Serializable; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.channel.source.ISource; import org.apache.rocketmq.streams.common.channel.split.ISplit; -import org.apache.rocketmq.streams.common.component.AbstractComponent; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; import org.apache.rocketmq.streams.common.context.MessageOffset; import org.apache.rocketmq.streams.common.utils.ThreadUtil; import org.apache.rocketmq.streams.connectors.IBoundedSource; import org.apache.rocketmq.streams.connectors.IBoundedSourceReader; import org.apache.rocketmq.streams.connectors.model.PullMessage; import org.apache.rocketmq.streams.connectors.model.ReaderStatus; -import org.apache.rocketmq.streams.connectors.source.CycleDynamicMultipleDBScanSource; +import org.apache.rocketmq.streams.connectors.reader.ISplitReader; +import org.apache.rocketmq.streams.connectors.reader.SplitCloseFuture; import org.apache.rocketmq.streams.db.driver.DriverBuilder; import org.apache.rocketmq.streams.db.driver.JDBCDriver; import org.apache.rocketmq.streams.db.driver.orm.ORMUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * @description */ public class DBScanReader implements ISplitReader, IBoundedSourceReader, Serializable { - private static final long serialVersionUID = 8172403250050893288L; - private static final Log logger = LogFactory.getLog(DBScanReader.class); static final String sqlTemplate = "select * from %s where id >= %d and id < %d"; - + private static final long serialVersionUID = 8172403250050893288L; + private static final Logger LOGGER = LoggerFactory.getLogger(DBScanReader.class); //是否完成了source的call back调用 - transient volatile boolean isFinishedCall = false; - ISource iSource; - String url; - String userName; - String password; - String tableName; - int batchSize; - long offset; - long offsetStart; - long offsetEnd; - long maxOffset; - long minOffset; - ISplit iSplit; - transient List pullMessages; - volatile boolean interrupt = false; - volatile boolean isClosed = false; + private boolean isFinishedCall = false; + private ISource iSource; + private String url; + private String userName; + private String password; + private final ThreadLocal threadLocal = new ThreadLocal<>() { + + @Override + public JDBCDriver initialValue() { + LOGGER.info(String.format("%s initial jdbcDriver. ", Thread.currentThread().getName())); + return DriverBuilder.createDriver(ConfigurationKey.DEFAULT_JDBC_DRIVER, url, userName, password); + } + + }; + private String tableName; + private int batchSize; + private long offset; + private long offsetStart; + private long offsetEnd; + private long maxOffset; + private long minOffset; + private ISplit iSplit; + private List> pullMessages; + private boolean interrupt = false; + private boolean isClosed = false; + + public DBScanReader() { + + } public String getUrl() { return url; @@ -105,38 +119,23 @@ public void setBatchSize(int batchSize) { this.batchSize = batchSize; } - public ISplit getISplit() { + public ISplit getISplit() { return iSplit; } - public void setISplit(ISplit iSplit) { + public void setISplit(ISplit iSplit) { this.iSplit = iSplit; } - public DBScanReader() { - - } - - transient ThreadLocal threadLocal = new ThreadLocal() { - - @Override - public JDBCDriver initialValue() { - logger.info(String.format("%s initial jdbcDriver. ", Thread.currentThread().getName())); - return DriverBuilder.createDriver(AbstractComponent.DEFAULT_JDBC_DRIVER, url, userName, password); - } - - }; - @Override - public void open(ISplit split) { - this.iSplit = split; + public void open() { JDBCDriver jdbcDriver = threadLocal.get(); Map range = jdbcDriver.queryOneRow("select min(id) as min_id, max(id) as max_id from " + tableName); minOffset = Long.parseLong(String.valueOf(range.get("min_id"))); maxOffset = Long.parseLong(String.valueOf(range.get("max_id"))); offsetStart = minOffset; offset = minOffset; - logger.info(String.format("table %s min id [ %d ], max id [ %d ]", tableName, minOffset, maxOffset)); + LOGGER.info(String.format("table %s min id [ %d ], max id [ %d ]", tableName, minOffset, maxOffset)); pullMessages = new ArrayList<>(); } @@ -153,15 +152,15 @@ public boolean next() { JDBCDriver jdbcDriver = threadLocal.get(); offsetEnd = offsetStart + batchSize; String batchQuery = String.format(sqlTemplate, tableName, offsetStart, offsetEnd); - logger.debug(String.format("execute sql : %s", batchQuery)); + LOGGER.debug(String.format("execute sql : %s", batchQuery)); List> resultData = jdbcDriver.queryForList(batchQuery); offsetStart = offsetEnd; pullMessages.clear(); for (Map r : resultData) { - PullMessage msg = new PullMessage(); + PullMessage msg = new PullMessage<>(); JSONObject data = JSONObject.parseObject(JSON.toJSONString(r)); msg.setMessage(data); - offset = offset > Long.parseLong(data.getString("id")) ? offset : Long.parseLong(data.getString("id")); + offset = Math.max(offset, Long.parseLong(data.getString("id"))); msg.setMessageOffset(new MessageOffset(String.valueOf(offset), true)); pullMessages.add(msg); } @@ -169,9 +168,9 @@ public boolean next() { } @Override - public List getMessage() { + public Iterator> getMessage() { // logger.info(String.format("output messages %d", pullMessages.size())); - return pullMessages; + return pullMessages.iterator(); } @Override @@ -193,7 +192,7 @@ public void seek(String cursor) { offset = minOffset; } offsetStart = offset; - logger.info(String.format("split %s seek %d.", iSplit.getQueueId(), offset)); + LOGGER.info(String.format("split %s seek %d.", iSplit.getQueueId(), offset)); } @Override @@ -217,7 +216,7 @@ public boolean isClose() { } @Override - public ISplit getSplit() { + public ISplit getSplit() { return iSplit; } @@ -227,9 +226,8 @@ public boolean isInterrupt() { } @Override - public boolean interrupt() { + public void interrupt() { interrupt = true; - return true; } @Override @@ -249,20 +247,20 @@ public void finish() { isFinishedCall = true; } - public ISource getISource() { + public ISource getISource() { return iSource; } - public void setISource(ISource iSource) { + public void setISource(ISource iSource) { this.iSource = iSource; } private final void updateReaderStatus() { String sourceName = CycleDynamicMultipleDBScanSource.createKey(this.getISource()); - int finish = Integer.valueOf(1); + int finish = 1; int total = ((CycleDynamicMultipleDBScanSource) iSource).getTotalReader(); ReaderStatus readerStatus = ReaderStatus.create(sourceName, iSplit.getQueueId(), finish, total); - logger.info(String.format("create reader status %s.", readerStatus)); + LOGGER.info(String.format("create reader status %s.", readerStatus)); ORMUtil.batchReplaceInto(readerStatus); } diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/source/DBSource.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/source/DBSource.java new file mode 100644 index 00000000..49d3e6ff --- /dev/null +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/source/DBSource.java @@ -0,0 +1,108 @@ +/* + * 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.db.source; + +import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; +import java.util.Date; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.context.MessageOffset; +import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.connectors.model.PullMessage; +import org.apache.rocketmq.streams.connectors.reader.AbstractQueryReader; +import org.apache.rocketmq.streams.connectors.reader.ISplitReader; +import org.apache.rocketmq.streams.connectors.source.AbstractPullSource; +import org.apache.rocketmq.streams.connectors.source.AbstractQuerySource; +import org.apache.rocketmq.streams.db.driver.DriverBuilder; +import org.apache.rocketmq.streams.db.driver.JDBCDriver; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DBSource extends AbstractQuerySource { + private static final Logger LOGGER = LoggerFactory.getLogger(DBSource.class); + @ENVDependence protected String jdbcDriver = ConfigurationKey.DEFAULT_JDBC_DRIVER; + @ENVDependence protected String url; + @ENVDependence protected String userName; + @ENVDependence protected String tableName; //指定要插入的数据表 + @ENVDependence protected String password; + + protected int batchSize=1000; + + protected String idField="id"; + protected String modifyTimeField; + protected boolean isTimestamp=false; + protected String modifyTimeFormat=DateUtil.DEFAULT_FORMAT; + + @Override protected ISplitReader createSplitReader(ISplit split) { + return new AbstractQueryReader(split) { + @Override public Iterator> getMessage() { + long endTime=startTime+dateAdd*60*1000; + String startTimeForSQL=getTimeSQL(startTime); + String endTimeForSQL=getTimeSQL(endTime); + String sql="select * from `"+tableName+"` where `"+modifyTimeField+"`>="+startTimeForSQL+" and `"+modifyTimeField+"`<"+endTimeForSQL+" and `"+idField+"`>"+pageNum+" order by `"+idField+"` limit "+batchSize; + JDBCDriver dataSource = DriverBuilder.createDriver(jdbcDriver, url, userName, password); + List> rows= dataSource.queryForList(sql); + int rowSize=(rows==null||rows.size()==0)?0:rows.size(); + LOGGER.info("pull data from db:"+getTimeSQL(startTime)+"-"+getTimeSQL(endTime)+". pull data count is "+rowSize+""); + if(rowSize==0){ + startTime=startTime+pollingMinute*60*1000; + pageNum=pageInit; + return null; + } + List> messages=new ArrayList<>(); + long maxIdValue=0; + + for(Map row:rows){ + long idValue=Long.valueOf(row.get(idField).toString()); + if(idValue>maxIdValue){ + maxIdValue=idValue; + } + JSONObject msg=new JSONObject(); + msg.putAll(row); + PullMessage pullMessage=new PullMessage(); + pullMessage.setMessage(msg); + MessageOffset messageOffset=new MessageOffset(startTime); + messageOffset.addLayerOffset(idValue); + pullMessage.setMessageOffset(messageOffset); + messages.add(pullMessage); + } + if(rows.size() tableNames; - List splits; + List> splits; transient volatile AtomicBoolean statusCheckerStart = new AtomicBoolean(false); //todo @@ -58,20 +57,12 @@ public DynamicMultipleDBScanSource() { splits = new ArrayList<>(); } - @Override - protected boolean initConfigurable() { + @Override protected boolean initConfigurable() { setTopic(logicTableName); return super.initConfigurable(); } - @Override - protected boolean isNotDataSplit(String queueId) { - return tableNames.contains(queueId); - } - - @Override - protected ISplitReader createSplitReader(ISplit split) { - + @Override protected ISplitReader createSplitReader(ISplit split) { DBScanReader reader = new DBScanReader(); reader.setISplit(split); reader.setUrl(url); @@ -79,15 +70,14 @@ protected ISplitReader createSplitReader(ISplit split) { reader.setPassword(password); reader.setTableName(String.valueOf(split.getQueue())); int local = batchSize <= 0 ? DEFAULT_BATCH_SIZE : batchSize; - local = local > MAX_BATCH_SIZE ? MAX_BATCH_SIZE : local; + local = Math.min(local, MAX_BATCH_SIZE); reader.setBatchSize(local); reader.setISource(this); - logger.info(String.format("create reader for split %s", split.getQueueId())); + LOGGER.info(String.format("create reader for split %s", split.getQueueId())); return reader; } - @Override - public List fetchAllSplits() { + @Override public List> fetchAllSplits() { if (filter == null) { filter = new DataFormatPatternFilter(); @@ -97,18 +87,18 @@ public List fetchAllSplits() { tableNames = MetaDataUtils.listTableNameByPattern(url, userName, password, logicTableName + "%"); - logger.info(String.format("load all logic table : %s", Arrays.toString(tableNames.toArray()))); + LOGGER.info(String.format("load all logic table : %s", Arrays.toString(tableNames.toArray()))); for (String s : tableNames) { String suffix = s.replace(logicTableName + "_", ""); if (filter.filter(null, logicTableName, suffix)) { - logger.info(String.format("filter add %s", s)); + LOGGER.info(String.format("filter add %s", s)); DynamicMultipleDBSplit split = new DynamicMultipleDBSplit(); split.setLogicTableName(logicTableName); split.setSuffix(suffix); splits.add(split); } else { - logger.info(String.format("filter remove %s", s)); + LOGGER.info(String.format("filter remove %s", s)); } } @@ -171,11 +161,11 @@ public void setTableNames(List tableNames) { this.tableNames = tableNames; } - public List getSplits() { + public List> getSplits() { return splits; } - public void setSplits(List splits) { + public void setSplits(List> splits) { this.splits = splits; } diff --git a/rocketmq-streams-channel-db/src/test/java/org/apache/rocketmq/streams/db/sink/db/DBWriteOnlyChannelTest.java b/rocketmq-streams-channel-db/src/test/java/org/apache/rocketmq/streams/db/sink/db/DBWriteOnlyChannelTest.java index 6a214cfb..77979307 100644 --- a/rocketmq-streams-channel-db/src/test/java/org/apache/rocketmq/streams/db/sink/db/DBWriteOnlyChannelTest.java +++ b/rocketmq-streams-channel-db/src/test/java/org/apache/rocketmq/streams/db/sink/db/DBWriteOnlyChannelTest.java @@ -26,9 +26,9 @@ public class DBWriteOnlyChannelTest { - private String URL = "*"; protected String USER_NAME = "*"; protected String PASSWORD = "*"; + private String URL = "*"; @Test public void testOutputBySQL() { diff --git a/rocketmq-streams-channel-db/src/test/java/org/apache/rocketmq/streams/db/sink/db/ISqlTemplateTest.java b/rocketmq-streams-channel-db/src/test/java/org/apache/rocketmq/streams/db/sink/db/ISqlTemplateTest.java index 41091491..70b18de1 100644 --- a/rocketmq-streams-channel-db/src/test/java/org/apache/rocketmq/streams/db/sink/db/ISqlTemplateTest.java +++ b/rocketmq-streams-channel-db/src/test/java/org/apache/rocketmq/streams/db/sink/db/ISqlTemplateTest.java @@ -34,7 +34,7 @@ public class ISqlTemplateTest { static List rows; - static{ + static { JSONObject jsonObject1 = new JSONObject(); jsonObject1.put("id", 1); jsonObject1.put("name", "chris"); @@ -59,7 +59,7 @@ public class ISqlTemplateTest { } @Test - public void testInsertInto(){ + public void testInsertInto() { MysqlInsertIntoSqlTemplate templateWithId = new MysqlInsertIntoSqlTemplate(metaData, true); System.out.println(templateWithId.createSql(rows)); MysqlInsertIntoSqlTemplate templateWithOutId = new MysqlInsertIntoSqlTemplate(metaData, false); @@ -67,7 +67,7 @@ public void testInsertInto(){ } @Test - public void testInsertIgnoreInto(){ + public void testInsertIgnoreInto() { MysqlInsertIgnoreIntoSqlTemplate templateWithId = new MysqlInsertIgnoreIntoSqlTemplate(metaData, true); System.out.println(templateWithId.createSql(rows)); MysqlInsertIgnoreIntoSqlTemplate templateWithOutId = new MysqlInsertIgnoreIntoSqlTemplate(metaData, false); @@ -75,7 +75,7 @@ public void testInsertIgnoreInto(){ } @Test - public void testInsertDuplicate(){ + public void testInsertDuplicate() { MysqlInsertIntoWithDuplicateKeySqlTemplate templateWithId = new MysqlInsertIntoWithDuplicateKeySqlTemplate(metaData, true); System.out.println(templateWithId.createSql(rows)); MysqlInsertIntoWithDuplicateKeySqlTemplate templateWithOutId = new MysqlInsertIntoWithDuplicateKeySqlTemplate(metaData, false); diff --git a/rocketmq-streams-channel-es/pom.xml b/rocketmq-streams-channel-es/pom.xml index 9e872a51..c5e410d6 100755 --- a/rocketmq-streams-channel-es/pom.xml +++ b/rocketmq-streams-channel-es/pom.xml @@ -1,13 +1,13 @@ - 4.0.0 org.apache.rocketmq rocketmq-streams - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT rocketmq-streams-channel-es ROCKETMQ STREAMS :: channel-es @@ -24,7 +24,7 @@ org.apache.rocketmq - rocketmq-streams-configurable + rocketmq-streams-commons diff --git a/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/ESChannelBuilder.java b/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/ESChannelBuilder.java index b85a48f3..784a05cf 100644 --- a/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/ESChannelBuilder.java +++ b/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/ESChannelBuilder.java @@ -20,7 +20,6 @@ import com.google.auto.service.AutoService; import java.util.Properties; import org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder; -import org.apache.rocketmq.streams.common.channel.builder.IShuffleChannelBuilder; 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; @@ -29,7 +28,7 @@ @AutoService(IChannelBuilder.class) @ServiceName(value = ESChannelBuilder.TYPE, aliasName = "ES") -public class ESChannelBuilder implements IChannelBuilder { +public class ESChannelBuilder implements IChannelBuilder { public static final String TYPE = "es"; protected JSONObject createFormatProperty(Properties properties) { @@ -43,8 +42,8 @@ protected JSONObject createFormatProperty(Properties properties) { } IChannelBuilder.formatPropertiesName(formatProperties, properties, "host", "endPoint"); IChannelBuilder.formatPropertiesName(formatProperties, properties, "esIndex", "index"); - IChannelBuilder.formatPropertiesName(formatProperties, properties, "esIndexType", "typeName");; - IChannelBuilder.formatPropertiesName(formatProperties, properties, "esMsgId", "es_msg_id");; + IChannelBuilder.formatPropertiesName(formatProperties, properties, "esIndexType", "typeName"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "esMsgId", "es_msg_id"); return formatProperties; } diff --git a/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/ESSinkOnlyChannel.java b/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/ESSinkOnlyChannel.java index c14063ab..646acc4a 100644 --- a/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/ESSinkOnlyChannel.java +++ b/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/ESSinkOnlyChannel.java @@ -22,14 +22,11 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.http.HttpHost; import org.apache.http.auth.AuthScope; import org.apache.http.auth.UsernamePasswordCredentials; import org.apache.http.client.CredentialsProvider; import org.apache.http.impl.client.BasicCredentialsProvider; -import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; import org.apache.rocketmq.streams.common.channel.sink.AbstractSink; import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; import org.apache.rocketmq.streams.common.context.IMessage; @@ -41,36 +38,33 @@ import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.client.RestHighLevelClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ESSinkOnlyChannel extends AbstractSink { - private static final Log LOG = LogFactory.getLog(ESSinkOnlyChannel.class); + private static final Logger LOGGER = LoggerFactory.getLogger(ESSinkOnlyChannel.class); private static final String PREFIX = "xxx"; - + protected String esMsgId; @ENVDependence private String host; @ENVDependence private String port; - private boolean needAuth = false; @ENVDependence private String authUsername; @ENVDependence private String authPassword; - private int socketTimeOut = 5 * 60 * 1000;; - - private int connectTimeOut = 5 * 60 * 1000;; - - private int connectionRequestTimeOut = 5 * 60 * 1000;; - + ; + private int socketTimeOut = 5 * 60 * 1000; + ; + private int connectTimeOut = 5 * 60 * 1000; + ; + private int connectionRequestTimeOut = 5 * 60 * 1000; private String schema = "http"; @ENVDependence private String esIndex; - private String esIndexType = "log"; - - protected String esMsgId; - private transient RestHighLevelClient client; public ESSinkOnlyChannel() { @@ -123,42 +117,39 @@ protected boolean initConfigurable() { try { client = new RestHighLevelClient(builder); } catch (Exception e) { - setInitSuccess(false); throw new RuntimeException("unknowhost exception ", e); } } return true; } - private List generateRequests(List messages) { List requests = new ArrayList<>(); messages.forEach(message -> { IndexRequest indexRequest = new IndexRequest(esIndex); Object object = message.getMessageValue(); - if(object!=null&&!(object instanceof Map)){ - String str=object.toString(); - if(str.startsWith("{")&&str.endsWith("}")){ + if (object != null && !(object instanceof Map)) { + String str = object.toString(); + if (str.startsWith("{") && str.endsWith("}")) { try { - JSONObject jsonObject= JSON.parseObject(str); - object=jsonObject; - }catch (Exception e){ - LOG.warn("the sink msg is not json, convert error"); + JSONObject jsonObject = JSON.parseObject(str); + object = jsonObject; + } catch (Exception e) { + LOGGER.warn("the sink msg is not json, convert error"); } } } if (object instanceof Map) { indexRequest.source((Map) object); - if(StringUtil.isNotEmpty(esMsgId)){ - Map map=(Map)object; - Object msgId=map.get(esMsgId); - if(msgId!=null){ + if (StringUtil.isNotEmpty(esMsgId)) { + Map map = (Map) object; + Object msgId = map.get(esMsgId); + if (msgId != null) { indexRequest.id(msgId.toString()); } } - } else { indexRequest.source(object.toString()); } @@ -178,11 +169,11 @@ public boolean batchInsert(List messages) { response = client.bulk(bulkRequest, RequestOptions.DEFAULT); } catch (IOException e) { e.printStackTrace(); - LOG.error("batch insert message to es exception " + e); + LOGGER.error("batch insert message to es exception " + e); return false; } - LOG.info("esChannel sendLogs logSize=" + messages.size() + " response size" + LOGGER.info("esChannel sendLogs logSize=" + messages.size() + " response size" + response.getItems().length + " status " + response.status() + " cost=" + response.getTook() + " esIndex=" + esIndex + " host=" + host); return true; diff --git a/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/EsClient.java b/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/EsClient.java index ce800909..e5eb961c 100644 --- a/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/EsClient.java +++ b/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/EsClient.java @@ -17,9 +17,6 @@ package org.apache.rocketmq.streams.es.sink; import java.io.IOException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.http.HttpHost; import org.apache.http.auth.AuthScope; import org.apache.http.auth.UsernamePasswordCredentials; @@ -29,6 +26,8 @@ import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.client.RestHighLevelClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * @author zhangliang @@ -37,32 +36,21 @@ */ public class EsClient { - private static final Log logger = LogFactory.getLog(EsClient.class); + private static final Logger LOGGER = LoggerFactory.getLogger(EsClient.class); //private static final Logger logger = LogManager.getLogger(EsClient.class); - - private RestHighLevelClient esClient; - - private Object object = new Object(); - private static final int socketTimeOut = 5 * 60 * 1000; - private static final int connectTimeOut = 5 * 60 * 1000; - private static final int connectionRequestTimeOut = 5 * 60 * 1000; - private static final boolean needAuth = true; - + private static String SCHEME = "http"; + private RestHighLevelClient esClient; + private Object object = new Object(); private String host; - private String port; - private String username; - private String password; - private static String SCHEME = "http"; - public EsClient(String host, String port, String username, String password) { this.host = host; this.port = port; @@ -86,7 +74,7 @@ public RestHighLevelClient getEsClient() { } private RestHighLevelClient createClient() { - logger.info("esClient createClient host=" + host + " port=" + port); + LOGGER.info("esClient createClient host=" + host + " port=" + port); try { //解决netty冲突问题 System.setProperty("es.set.netty.runtime.available.processors", "false"); @@ -109,7 +97,7 @@ public HttpAsyncClientBuilder customizeHttpClient(HttpAsyncClientBuilder httpCli } esClient = new RestHighLevelClient(builder); } catch (Exception e) { - logger.error("esClient createClient host=" + host + " port=" + port + " error=" + e.getMessage(), e); + LOGGER.error("esClient createClient host=" + host + " port=" + port + " error=" + e.getMessage(), e); throw e; } return esClient; @@ -121,14 +109,14 @@ public void rebuildEsCilent() { } private void closeClient() { - logger.error("esClient closeClient host=" + host + " port=" + port); + LOGGER.error("esClient closeClient host=" + host + " port=" + port); if (esClient != null) { try { esClient.close(); esClient = null; - logger.error("esClient closeClient success host=" + host + " port=" + port); + LOGGER.error("esClient closeClient success host=" + host + " port=" + port); } catch (IOException e) { - logger.error("esClient closeClient failed host=" + host + " port=" + port + " error=" + e.getMessage(), e); + LOGGER.error("esClient closeClient failed host=" + host + " port=" + port + " error=" + e.getMessage(), e); } } } diff --git a/rocketmq-streams-channel-http/pom.xml b/rocketmq-streams-channel-http/pom.xml index 149b0507..9757516b 100644 --- a/rocketmq-streams-channel-http/pom.xml +++ b/rocketmq-streams-channel-http/pom.xml @@ -21,7 +21,7 @@ org.apache.rocketmq rocketmq-streams - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT 4.0.0 diff --git a/rocketmq-streams-channel-http/src/main/java/org/apache/rocketmq/streams/http/source/HttpSource.java b/rocketmq-streams-channel-http/src/main/java/org/apache/rocketmq/streams/http/source/HttpSource.java index da772fb4..b10ec09b 100644 --- a/rocketmq-streams-channel-http/src/main/java/org/apache/rocketmq/streams/http/source/HttpSource.java +++ b/rocketmq-streams-channel-http/src/main/java/org/apache/rocketmq/streams/http/source/HttpSource.java @@ -19,8 +19,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.channel.source.AbstractUnreliableSource; import org.apache.rocketmq.streams.common.utils.IPUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; @@ -30,7 +28,6 @@ * receive http(s) post data **/ public class HttpSource extends AbstractUnreliableSource { - private static final Log LOG = LogFactory.getLog(HttpSource.class); protected boolean isHttps = false;//发送的请求是否是https的 /** @@ -100,6 +97,10 @@ protected boolean startSource() { return true; } + @Override protected void destroySource() { + + } + public boolean isHttps() { return isHttps; } diff --git a/rocketmq-streams-channel-http/src/main/java/org/apache/rocketmq/streams/http/source/server/HttpServer.java b/rocketmq-streams-channel-http/src/main/java/org/apache/rocketmq/streams/http/source/server/HttpServer.java index 1d95bd92..e5a22696 100644 --- a/rocketmq-streams-channel-http/src/main/java/org/apache/rocketmq/streams/http/source/server/HttpServer.java +++ b/rocketmq-streams-channel-http/src/main/java/org/apache/rocketmq/streams/http/source/server/HttpServer.java @@ -33,23 +33,22 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.ExecutorService; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; import javax.net.ssl.KeyManagerFactory; import javax.net.ssl.SSLContext; import javax.net.ssl.SSLEngine; import javax.net.ssl.SSLParameters; import javax.net.ssl.TrustManagerFactory; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.channel.source.AbstractUnreliableSource; import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.threadpool.ThreadPoolFactory; import org.apache.rocketmq.streams.http.source.HttpSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class HttpServer extends AbstractUnreliableSource { - private static final Log LOG = LogFactory.getLog(HttpServer.class); + private static final Logger LOGGER = LoggerFactory.getLogger(HttpServer.class); + protected transient List channels = new ArrayList<>(); private transient com.sun.net.httpserver.HttpServer server; @ENVDependence private int port = 8000; @@ -61,8 +60,6 @@ public class HttpServer extends AbstractUnreliableSource { private int stopDelaySecond; private boolean useHttps = false; - protected transient List channels = new ArrayList<>(); - public HttpServer() { setJsonData(false); setMsgIsJsonArray(false); @@ -108,7 +105,7 @@ public void configure(HttpsParameters params) { params.setSSLParameters(sslParameters); } catch (Exception ex) { - LOG.error("Failed to create HTTPS port", ex); + LOGGER.error("Failed to create HTTPS port", ex); } } }); @@ -119,15 +116,15 @@ public void configure(HttpsParameters params) { } catch (IOException e) { // System.out.println(e); - LOG.error("http channel init get io exception", e); + LOGGER.error("http channel init get io exception", e); return false; } catch (NoSuchAlgorithmException e) { // System.out.println(e); - LOG.error("http channel init https ssl context exception", e); + LOGGER.error("http channel init https ssl context exception", e); return false; } catch (Exception e) { // System.out.println(e); - LOG.error("http channel init http cert exception", e); + LOGGER.error("http channel init http cert exception", e); return false; } return true; @@ -135,9 +132,7 @@ public void configure(HttpsParameters params) { @Override protected boolean startSource() { - ExecutorService cachedThreadPool = new ThreadPoolExecutor(maxThread, maxThread, - 0L, TimeUnit.MILLISECONDS, - new LinkedBlockingQueue(1000)); + ExecutorService cachedThreadPool = ThreadPoolFactory.createFixedThreadPool(maxThread, HttpServer.class.getName() + "-" + getName()); server.setExecutor(cachedThreadPool); server.createContext("/", new DataHandler()); setReceiver((message, context) -> { @@ -147,10 +142,6 @@ protected boolean startSource() { List destroyChannels = new ArrayList<>(); for (HttpSource channel : channels) { if (channel.match(clientIp, query, uri)) { - if (channel.isDestroy()) { - destroyChannels.add(channel); - continue; - } channel.doReceiveMessage(message.getMessageBody()); } } @@ -161,9 +152,9 @@ protected boolean startSource() { }); server.start(); if (isUseHttps()) { - LOG.info("https server is start"); + LOGGER.info("https server is start"); } else { - LOG.info("http server is start"); + LOGGER.info("http server is start"); } return true; @@ -209,34 +200,8 @@ public void setStopDelaySecond(int delaySecond) { this.stopDelaySecond = delaySecond; } - private class DataHandler implements HttpHandler { - @Override - public void handle(HttpExchange exchange) throws IOException { - OutputStream os = exchange.getResponseBody(); - InputStream in = exchange.getRequestBody(); - try { - JSONObject jsonObject = createHttpHeader(exchange); - jsonObject.put("query", exchange.getRequestURI().getQuery()); - String data = getContent(in); - jsonObject.put(IMessage.DATA_KEY, data); - - doReceiveMessage(jsonObject); - String response = "{\"code\": \"200\", \"data\" :\"received\", \"message\" :\"\"}"; - exchange.sendResponseHeaders(200, 0); - os.write(response.getBytes()); - os.flush(); - } catch (Exception e) { - LOG.error(""); - } finally { - if (os != null) { - os.close(); - } - if (in != null) { - in.close(); - } - } + @Override protected void destroySource() { - } } protected JSONObject createHttpHeader(HttpExchange exchange) { @@ -266,4 +231,34 @@ protected String getContent(InputStream in) throws IOException { } return stringBuilder.toString(); } + + private class DataHandler implements HttpHandler { + @Override + public void handle(HttpExchange exchange) throws IOException { + OutputStream os = exchange.getResponseBody(); + InputStream in = exchange.getRequestBody(); + try { + JSONObject jsonObject = createHttpHeader(exchange); + jsonObject.put("query", exchange.getRequestURI().getQuery()); + String data = getContent(in); + jsonObject.put(IMessage.DATA_KEY, data); + + doReceiveMessage(jsonObject); + String response = "{\"code\": \"200\", \"data\" :\"received\", \"message\" :\"\"}"; + exchange.sendResponseHeaders(200, 0); + os.write(response.getBytes()); + os.flush(); + } catch (Exception e) { + LOGGER.error(""); + } finally { + if (os != null) { + os.close(); + } + if (in != null) { + in.close(); + } + } + + } + } } \ No newline at end of file diff --git a/rocketmq-streams-channel-http/src/main/java/org/apache/rocketmq/streams/http/source/server/HttpServerManager.java b/rocketmq-streams-channel-http/src/main/java/org/apache/rocketmq/streams/http/source/server/HttpServerManager.java index cf0e4eeb..e6381c27 100644 --- a/rocketmq-streams-channel-http/src/main/java/org/apache/rocketmq/streams/http/source/server/HttpServerManager.java +++ b/rocketmq-streams-channel-http/src/main/java/org/apache/rocketmq/streams/http/source/server/HttpServerManager.java @@ -17,15 +17,12 @@ package org.apache.rocketmq.streams.http.source.server; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.http.source.HttpSource; /** * receive http(s) post data **/ public class HttpServerManager { - private static final Log LOG = LogFactory.getLog(HttpServerManager.class); protected transient static HttpServer HTTP_SERVER = new HttpServer();//http server,全局只有一个 protected transient static HttpServer HTTPS_SERVER = new HttpServer();//https server 全局只有一个 protected transient static AtomicBoolean isHttpStart = new AtomicBoolean(false);//只启动一次 diff --git a/rocketmq-streams-channel-huawei/pom.xml b/rocketmq-streams-channel-huawei/pom.xml new file mode 100644 index 00000000..b695d505 --- /dev/null +++ b/rocketmq-streams-channel-huawei/pom.xml @@ -0,0 +1,38 @@ + + + 4.0.0 + + rocketmq-streams + org.apache.rocketmq + 2.0.1-SNAPSHOT + + + rocketmq-streams-channel-huawei + ROCKETMQ STREAMS :: channel-huawei + + + + + org.apache.rocketmq + rocketmq-streams-connectors + + + junit + junit + test + + + com.huaweicloud + esdk-obs-java-bundle + 3.23.9 + + + org.apache.commons + commons-compress + 1.21 + + + + \ No newline at end of file diff --git a/rocketmq-streams-channel-huawei/src/main/java/org/apache/rocketmq/streams/huawei/obs/ObsSource.java b/rocketmq-streams-channel-huawei/src/main/java/org/apache/rocketmq/streams/huawei/obs/ObsSource.java new file mode 100644 index 00000000..cca588bf --- /dev/null +++ b/rocketmq-streams-channel-huawei/src/main/java/org/apache/rocketmq/streams/huawei/obs/ObsSource.java @@ -0,0 +1,361 @@ +/* + * 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.huawei.obs; + +import com.google.common.collect.Lists; +import com.obs.services.ObsClient; + +import java.io.IOException; +import java.util.Date; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.lang3.concurrent.BasicThreadFactory; +import org.apache.commons.lang3.time.DateFormatUtils; +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.threadpool.ScheduleFactory; +import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.common.utils.IdUtil; +import org.apache.rocketmq.streams.connectors.model.PullMessage; +import org.apache.rocketmq.streams.connectors.reader.AbstractFlinkSplitReader; +import org.apache.rocketmq.streams.connectors.reader.ISplitReader; +import org.apache.rocketmq.streams.connectors.source.AbstractPullSource; +import org.apache.rocketmq.streams.huawei.obs.iterator.ObsDataIterator; +import org.apache.rocketmq.streams.huawei.obs.split.ObsSplit; +import org.apache.rocketmq.streams.lease.service.ILeaseStorage; +import org.apache.rocketmq.streams.lease.service.impl.LeaseServiceImpl; +import org.apache.rocketmq.streams.lease.service.storages.DBLeaseStorage; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ObsSource extends AbstractPullSource { + + private static final Logger LOGGER = LoggerFactory.getLogger(ObsSource.class); + + private String accessId; + + private String accessKey; + + private String endPoint; + + private String bucketName; + + private String filePath; + + private String fileType; + + private String compressType; + + private String startTime; + + private String endTime; + + private Integer cycle; + + private String cycleUnit; + + private Integer cycleT = 1; + + /****************************其上为成员变量*****************************************/ + private ObsClient obsClient; + + private LeaseServiceImpl leaseService; + + private long cycleTime = 0L; + + private ScheduledExecutorService scheduledExecutorService; + + @Override + protected boolean initConfigurable() { + this.isTest = true; + super.initConfigurable(); + if (this.cache.getKeyConfig(this.getNameSpace() + "_" + this.getName(), START_TIME) != null) { + this.startTime = this.cache.getKeyConfig(this.getNameSpace() + "_" + this.getName(), START_TIME); + } else if (this.startTime == null) { + this.startTime = DateUtil.getCurrentTimeString("yyyy-MM-dd HH:mm:ss"); + } + + if (this.obsClient == null) { + this.obsClient = new ObsClient(accessId, accessKey, endPoint); + } + + if (cycleUnit == null || cycleUnit.isEmpty()) { + cycleUnit = TimeUnit.SECONDS.toString(); + } + if (cycleUnit.equalsIgnoreCase(TimeUnit.DAYS.toString())) { + cycleTime = cycle * 24 * 60 * 60 * 1000; + } else if (cycleUnit.equalsIgnoreCase(TimeUnit.HOURS.toString())) { + cycleTime = cycle * 60 * 60 * 1000; + } else if (cycleUnit.equalsIgnoreCase(TimeUnit.MINUTES.toString())) { + cycleTime = cycle * 60 * 1000; + } else if (cycleUnit.equalsIgnoreCase(TimeUnit.SECONDS.toString())) { + cycleTime = cycle * 1000; + } + + if (fileType == null || fileType.isEmpty()) { + fileType = "json"; + } + + if (compressType == null || compressType.isEmpty()) { + compressType = "gzip"; + } + + String jdbcDriver = getConfiguration().getProperty(ConfigurationKey.JDBC_DRIVER, ConfigurationKey.DEFAULT_JDBC_DRIVER); + String url = getConfiguration().getProperty(ConfigurationKey.JDBC_URL); + String userName = getConfiguration().getProperty(ConfigurationKey.JDBC_USERNAME); + String password = getConfiguration().getProperty(ConfigurationKey.JDBC_PASSWORD); + + this.leaseService = new LeaseServiceImpl(); + ILeaseStorage storage = new DBLeaseStorage(jdbcDriver, url, userName, password); + this.leaseService.setLeaseStorage(storage); + + //启动Master选举 + this.leaseService.startLeaseTask("Batch_Job_Lock_" + this.getNameSpace() + "_" + this.getName(), 60, nextLeaseDate -> { + LOGGER.info("[{}][{}][{}] Batch_Job_Locked_Success", this.getNameSpace(), this.getName(), IdUtil.objectId(this)); + }); + + if (this.scheduledExecutorService == null) { + this.scheduledExecutorService = new ScheduledThreadPoolExecutor(10, new BasicThreadFactory.Builder().namingPattern(getNameSpace() + "-" + getName() + "-schedule-%d").build()); + } + this.scheduledExecutorService.scheduleWithFixedDelay(() -> { + if (this.leaseService.hasLease("Batch_Job_Lock_" + this.getNameSpace() + "_" + this.getName())) { + LOGGER.info("[{}][{}][{}] Batch_Job_Master", this.getNameSpace(), this.getName(), IdUtil.objectId(this)); + + long start = DateUtil.parse(this.startTime).getTime(); + long end = (this.endTime == null ? System.currentTimeMillis() : DateUtil.parse(this.endTime).getTime()) - cycleT * cycleTime; + if (start > end) { + LOGGER.warn("[{}][{}][{}] Batch_Jobs_Start({})_Greater_Then_End({})", this.getNameSpace(), this.getName(), IdUtil.objectId(this), DateFormatUtils.format(start, "yyyyMMddHHmmss"), DateFormatUtils.format(end, "yyyyMMddHHmmss")); + this.cache.putKeyConfig(this.getNameSpace() + "_" + this.getName(), START_TIME, this.startTime); + } else { + List dateList = DateUtil.getWindowBeginTime(end, cycleTime, end - start); + + LOGGER.info("[{}][{}][{}] Batch_Jobs_From({})_To({})_Size({})", this.getNameSpace(), this.getName(), IdUtil.objectId(this), DateFormatUtils.format(start, "yyyyMMddHHmmss"), DateFormatUtils.format(end, "yyyyMMddHHmmss"), dateList.size()); + if (!dateList.isEmpty()) { + for (Date date : dateList) { + String partition = partitionPath(date, filePath); + this.cache.putKeyConfig(this.getNameSpace() + "_" + this.getName() + "_partition", partition + "___" + DateUtil.format(date), DateUtil.format(new Date(start))); + } + this.startTime = DateUtil.format(new Date(end)); + this.cache.putKeyConfig(this.getNameSpace() + "_" + this.getName(), START_TIME, this.startTime); + } + } + } + }, 0, 60, TimeUnit.SECONDS); + + return true; + } + + private String partitionPath(Date partitionTime, String filePath) { + String year = DateUtil.getYear(partitionTime); + String month = DateUtil.getMonth(partitionTime) < 10 ? "0" + DateUtil.getMonth(partitionTime) : String.valueOf(DateUtil.getMonth(partitionTime)); + String day = DateUtil.getDay(partitionTime) < 10 ? "0" + DateUtil.getDay(partitionTime) : String.valueOf(DateUtil.getDay(partitionTime)); + String hour = DateUtil.getHour(partitionTime) < 10 ? "0" + DateUtil.getHour(partitionTime) : String.valueOf(DateUtil.getHour(partitionTime)); + String minute = DateUtil.getMinute(partitionTime) < 10 ? "0" + DateUtil.getMinute(partitionTime) : String.valueOf(DateUtil.getMinute(partitionTime)); + return filePath.replaceAll("%Y", year).replaceAll("%m", month).replaceAll("%d", day).replaceAll("%H", hour).replaceAll("%M", minute); + } + + @Override + public String loadSplitOffset(ISplit split) { + return this.startTime; + } + + @Override + protected ISplitReader createSplitReader(ISplit split) { + + return new AbstractFlinkSplitReader(split) { + + private String queueId; + private String partition; + private Boolean hasNext = true; + + @Override + public String getCursor() { + return null; + } + + @Override + public void open() { + ObsSplit obsSplit = (ObsSplit)split; + this.queueId = obsSplit.getQueueId(); + this.partition = this.queueId.substring(0, this.queueId.indexOf("___")); + } + + @Override + public boolean next() { + return this.hasNext; + } + + @Override + public Iterator> getMessage() { + return new ObsDataIterator(obsClient, bucketName, this.partition, fileType, compressType, fieldDelimiter, getMetaData(), () -> { + doSplitRelease(Lists.newArrayList(split)); + cache.deleteKeyConfig(getNameSpace() + "_" + getName() + "_partition", this.queueId); + this.hasNext = false; + }); + } + + @Override + public void seek(String cursor) { + } + + @Override + public long getDelay() { + return 0; + } + + @Override + public long getFetchedDelay() { + return 0; + } + + }; + } + + @Override + public void destroySource() { + if (this.obsClient != null) { + try { + this.obsClient.close(); + ScheduleFactory.getInstance().cancel("ObsSource_Job_" + this.getNameSpace() + "_" + this.getName()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } + + @Override + public List> fetchAllSplits() { + Map partitions = this.cache.getKVListByNameSpace(this.getNameSpace() + "_" + this.getName() + "_partition"); + List> splits = Lists.newArrayList(); + if (partitions != null && !partitions.isEmpty()) { + for (Map.Entry entry : partitions.entrySet()) { + ObsSplit split = new ObsSplit(entry.getKey()); + splits.add(split); + } + } + return splits; + } + + public String getAccessId() { + return accessId; + } + + public void setAccessId(String accessId) { + this.accessId = accessId; + } + + public String getAccessKey() { + return accessKey; + } + + public void setAccessKey(String accessKey) { + this.accessKey = accessKey; + } + + public String getEndPoint() { + return endPoint; + } + + public void setEndPoint(String endPoint) { + this.endPoint = endPoint; + } + + public String getBucketName() { + return bucketName; + } + + public void setBucketName(String bucketName) { + this.bucketName = bucketName; + } + + public Integer getCycle() { + return cycle; + } + + public void setCycle(Integer cycle) { + this.cycle = cycle; + } + + public String getCycleUnit() { + return cycleUnit; + } + + public void setCycleUnit(String cycleUnit) { + this.cycleUnit = cycleUnit; + } + + public String getFilePath() { + return filePath; + } + + public void setFilePath(String filePath) { + this.filePath = filePath; + } + + public String getFileType() { + return fileType; + } + + public void setFileType(String fileType) { + this.fileType = fileType; + } + + public String getCompressType() { + return compressType; + } + + public void setCompressType(String compressType) { + this.compressType = compressType; + } + + public String getStartTime() { + return startTime; + } + + public void setStartTime(String startTime) { + this.startTime = startTime; + } + + public long getCycleTime() { + return cycleTime; + } + + public void setCycleTime(long cycleTime) { + this.cycleTime = cycleTime; + } + + public String getEndTime() { + return endTime; + } + + public void setEndTime(String endTime) { + this.endTime = endTime; + } + + public Integer getCycleT() { + return cycleT; + } + + public void setCycleT(Integer cycleT) { + this.cycleT = cycleT; + } +} diff --git a/rocketmq-streams-channel-huawei/src/main/java/org/apache/rocketmq/streams/huawei/obs/builder/ObsSourceBuilder.java b/rocketmq-streams-channel-huawei/src/main/java/org/apache/rocketmq/streams/huawei/obs/builder/ObsSourceBuilder.java new file mode 100644 index 00000000..72476584 --- /dev/null +++ b/rocketmq-streams-channel-huawei/src/main/java/org/apache/rocketmq/streams/huawei/obs/builder/ObsSourceBuilder.java @@ -0,0 +1,89 @@ +/* + * 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.huawei.obs.builder; + +import com.alibaba.fastjson.JSONObject; + +import com.google.auto.service.AutoService; + +import java.util.Properties; + +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.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; +import org.apache.rocketmq.streams.huawei.obs.ObsSource; + +@AutoService(IChannelBuilder.class) +@ServiceName(value = ObsSourceBuilder.TYPE, aliasName = "ObsSource") +public class ObsSourceBuilder extends AbstractSupportShuffleChannelBuilder { + + public static final String TYPE = "huawei_obs"; + + @Override + public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) { + return (ObsSource)ConfigurableUtil.create(ObsSource.class.getName(), namespace, name, createFormatProperty(properties), null); + } + + @Override + public String getType() { + return TYPE; + } + + @Override + public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { + throw new RuntimeException("can not support this method"); + } + + @Override + public ISink createBySource(ISource pipelineSource) { + return null; + } + + @Override + 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.get(key)); + } + + IChannelBuilder.formatPropertiesName(formatProperties, properties, "accessId", "ak"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "accessKey", "sk"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "endPoint", "end_point"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "bucketName", "bucket_name"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "filePath", "file_path"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "filePrefix", "file_prefix"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "fileType", "file_type"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "fieldDelimiter", "field_delimiter"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "compressType", "compress_type"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "cycle", "cycle"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "cycleUnit", "cycle_unit"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "cycleT", "cycle_t"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "startTime", "start_time"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "endTime", "end_time"); + + return formatProperties; + } + +} diff --git a/rocketmq-streams-channel-huawei/src/main/java/org/apache/rocketmq/streams/huawei/obs/iterator/ObsDataIterator.java b/rocketmq-streams-channel-huawei/src/main/java/org/apache/rocketmq/streams/huawei/obs/iterator/ObsDataIterator.java new file mode 100644 index 00000000..71948a45 --- /dev/null +++ b/rocketmq-streams-channel-huawei/src/main/java/org/apache/rocketmq/streams/huawei/obs/iterator/ObsDataIterator.java @@ -0,0 +1,237 @@ +/* + * 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.huawei.obs.iterator; + +import com.alibaba.fastjson.JSONObject; + +import com.google.common.collect.Lists; +import com.obs.services.ObsClient; +import com.obs.services.model.ListObjectsRequest; +import com.obs.services.model.ObjectListing; +import com.obs.services.model.ObsObject; + +import java.io.BufferedReader; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStreamReader; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.zip.ZipInputStream; + +import org.apache.commons.compress.archivers.zip.ZipArchiveEntry; +import org.apache.commons.compress.archivers.zip.ZipArchiveInputStream; +import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream; +import org.apache.rocketmq.streams.common.context.MessageOffset; +import org.apache.rocketmq.streams.common.metadata.MetaData; +import org.apache.rocketmq.streams.common.metadata.MetaDataField; +import org.apache.rocketmq.streams.connectors.model.PullMessage; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Obs数据迭代器 + */ +public class ObsDataIterator implements Iterator> { + + private static final Logger LOGGER = LoggerFactory.getLogger(ObsDataIterator.class); + + private final String fileType; + private final MetaData metaData; + private final String fieldDelimiter; + private final String partitionPath; + private final FinishedCallback callback; + List bufferedReaderList = Lists.newArrayList(); + private String nextLine = null; + private int hasNextIndex; + + public ObsDataIterator(ObsClient obsClient, String bucketName, String partitionPath, String fileType, String compressType, String fieldDelimiter, MetaData metaData, FinishedCallback callback) { + this.fileType = fileType; + this.metaData = metaData; + this.fieldDelimiter = fieldDelimiter; + this.partitionPath = partitionPath; + this.callback = callback; + + ListObjectsRequest request = new ListObjectsRequest(bucketName); + // 设置每页100个对象 + request.setPrefix(partitionPath); + request.setMaxKeys(100); + ObjectListing result = null; + do { + try { + result = obsClient.listObjects(request); + List resultObjects = result.getObjects(); + if (!resultObjects.isEmpty()) { + LOGGER.info("Get_Obs_Objects_Success: {} | {} : ", bucketName, partitionPath); + for (ObsObject obsObject : resultObjects) { + ObsObject obsTemp = obsClient.getObject(obsObject.getBucketName(), obsObject.getObjectKey()); + if (compressType.equalsIgnoreCase("gzip")) { + bufferedReaderList.add(getGzipContent(obsTemp)); + } else { + bufferedReaderList.add(getZipContent(obsTemp)); + } + } + } + request.setMarker(result.getNextMarker()); + } catch (Exception e) { + LOGGER.error("Get_Obs_Objects_Error", e); + } + } + while (result != null && result.isTruncated()); + try { + for (hasNextIndex = 0; hasNextIndex < this.bufferedReaderList.size(); hasNextIndex++) { + this.nextLine = this.bufferedReaderList.get(hasNextIndex).readLine(); + if (this.nextLine != null) { + break; + } + } + } catch (IOException e) { + LOGGER.error("Read_Obs_File_Error", e); + } + } + + @Override + public boolean hasNext() { + if (this.nextLine != null) { + return true; + } else { + try { + for (; hasNextIndex < this.bufferedReaderList.size(); hasNextIndex++) { + this.nextLine = this.bufferedReaderList.get(hasNextIndex).readLine(); + if (this.nextLine != null) { + break; + } + } + if (this.nextLine != null) { + return true; + } else { + close(); + return false; + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + @Override + public PullMessage next() { + if (this.nextLine != null || hasNext()) { + String line = this.nextLine; + this.nextLine = null; + JSONObject msg; + if (fileType.equalsIgnoreCase("json")) { + msg = JSONObject.parseObject(line); + } else { //原始数据 + List> dataFields = metaData.getMetaDataFields(); + String[] data = line.split(fieldDelimiter); + msg = new JSONObject(); + for (int i = 0; i < dataFields.size(); i++) { + MetaDataField metaDataField = dataFields.get(i); + msg.put(metaDataField.getFieldName(), i < data.length ? data[i] : ""); + } + } + MessageOffset messageOffset = new MessageOffset(this.partitionPath); + messageOffset.addLayerOffset(0); + PullMessage pullMessage = new PullMessage<>(); + pullMessage.setMessage(msg); + pullMessage.setMessageOffset(messageOffset); + return pullMessage; + } else { + throw new NoSuchElementException(); + } + } + + private void close() { + try { + for (BufferedReader bufferedReader : bufferedReaderList) { + bufferedReader.close(); + } + } catch (IOException e) { + throw new RuntimeException(e); + } finally { + this.callback.finish(); + } + } + + private BufferedReader getGzipContent(ObsObject obsObject) { + // 读取对象内容 + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + GzipCompressorInputStream gzIn = null; + try { + gzIn = new GzipCompressorInputStream(obsObject.getObjectContent()); + byte[] b = new byte[1024]; + int len; + while ((len = gzIn.read(b)) != -1) { + bos.write(b, 0, len); + } + + // 将CSV数据解析为Java对象 + ByteArrayInputStream fileInputStream = new ByteArrayInputStream(bos.toByteArray()); + InputStreamReader inputStreamReader = new InputStreamReader(fileInputStream); + return new BufferedReader(inputStreamReader); + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + try { + if (gzIn != null) { + gzIn.close(); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } + + private BufferedReader getZipContent(ObsObject obsObject) { + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + ZipArchiveInputStream zipArchiveInputStream = new ZipArchiveInputStream(new ZipInputStream(obsObject.getObjectContent())); + try { + ZipArchiveEntry tarEntry; + while ((tarEntry = zipArchiveInputStream.getNextZipEntry()) != null) { + if (tarEntry.isDirectory()) { + continue; + } + byte[] buffer = new byte[1024]; + int n; + while ((n = zipArchiveInputStream.read(buffer, 0, 1024)) >= 0) { + bos.write(buffer, 0, n); + } + + // 将CSV数据解析为Java对象 + ByteArrayInputStream fileInputStream = new ByteArrayInputStream(bos.toByteArray()); + InputStreamReader inputStreamReader = new InputStreamReader(fileInputStream); + return new BufferedReader(inputStreamReader); + } + } catch (IOException e) { + throw new RuntimeException(e); + } finally { + try { + zipArchiveInputStream.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return null; + } + + public interface FinishedCallback { + void finish(); + } + +} diff --git a/rocketmq-streams-channel-huawei/src/main/java/org/apache/rocketmq/streams/huawei/obs/split/ObsSplit.java b/rocketmq-streams-channel-huawei/src/main/java/org/apache/rocketmq/streams/huawei/obs/split/ObsSplit.java new file mode 100644 index 00000000..d4ef6831 --- /dev/null +++ b/rocketmq-streams-channel-huawei/src/main/java/org/apache/rocketmq/streams/huawei/obs/split/ObsSplit.java @@ -0,0 +1,48 @@ +/* + * 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.huawei.obs.split; + +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; + +public class ObsSplit extends BasedConfigurable implements ISplit { + + private String queueId; + + public ObsSplit() { + } + + public ObsSplit(String queueId) { + this.queueId = queueId; + } + + @Override + public String getQueueId() { + return this.queueId; + } + + @Override + public String getQueue() { + return this.queueId; + } + + @Override + public int compareTo(ObsSplit o) { + return this.queueId.compareTo(o.getQueueId()); + } + +} \ No newline at end of file diff --git a/rocketmq-streams-channel-huawei/src/test/java/org/apache/rocketmq/streams/huawei/ObsTest.java b/rocketmq-streams-channel-huawei/src/test/java/org/apache/rocketmq/streams/huawei/ObsTest.java new file mode 100644 index 00000000..39feb23a --- /dev/null +++ b/rocketmq-streams-channel-huawei/src/test/java/org/apache/rocketmq/streams/huawei/ObsTest.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.huawei; + +import com.google.common.collect.Lists; +import com.obs.services.ObsClient; +import com.obs.services.exception.ObsException; +import com.obs.services.model.DownloadFileRequest; +import com.obs.services.model.DownloadFileResult; +import com.obs.services.model.ListObjectsRequest; +import com.obs.services.model.ObjectListing; +import com.obs.services.model.ObjectMetadata; +import com.obs.services.model.ObsObject; + +import java.io.BufferedReader; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStreamReader; +import java.util.List; + +import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream; +import org.junit.Before; +import org.junit.Test; + +public class ObsTest { + + String endPoint = ""; + String ak = ""; + String sk = ""; + + private ObsClient obsClient; + + @Before + public void init() { + obsClient = new ObsClient(ak, sk, endPoint); + } + + @Test + public void test() { + ListObjectsRequest request = new ListObjectsRequest("waf-alert"); + // 设置每页100个对象 + request.setPrefix("LogTanks/cn-east-3/2023/07/19/waf-project-test/vpc-flow-log-eip/"); + request.setMaxKeys(100); + + ObjectListing result; + do { + result = obsClient.listObjects(request); + for (ObsObject obsObject : result.getObjects()) { + System.out.println(obsObject.getBucketName() + "---" + obsObject.getObjectKey()); + ObjectMetadata metadata = obsClient.getObjectMetadata("waf-alert", obsObject.getObjectKey()); + System.out.println("\t" + metadata.getContentType()); + System.out.println("\t" + metadata.getContentLength()); + System.out.println("\t" + metadata.getUserMetadata("property")); + System.out.println("\t" + metadata.getOriginalHeaders()); + + // getGzipContent(obsClient.getObject(obsObject.getBucketName(), obsObject.getObjectKey())); + } + request.setMarker(result.getNextMarker()); + } + while (result.isTruncated()); + + } + + private void download(ObsObject obsObject) { + DownloadFileRequest request = new DownloadFileRequest("waf-alert", obsObject.getObjectKey()); + request.setDownloadFile("/Users/junjie.cheng/tmp/" + obsObject.getObjectKey()); + request.setTaskNum(5); + request.setPartSize(10 * 1024 * 1024); + request.setEnableCheckpoint(true); + try { + // 进行断点续传下载 + DownloadFileResult result = obsClient.downloadFile(request); + System.out.println("Etag:" + result.getObjectMetadata().getEtag()); + } catch (ObsException e) { + System.out.println("Error Code:" + e.getErrorCode()); + System.out.println("Error Message: " + e.getErrorMessage()); + } + try { + Thread.sleep(10000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + private byte[] getGzipContent(ObsObject obsObject) { + // 读取对象内容 + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + GzipCompressorInputStream gzIn = null; + try { + gzIn = new GzipCompressorInputStream(obsObject.getObjectContent()); + byte[] b = new byte[1024]; + int len; + while ((len = gzIn.read(b)) != -1) { + bos.write(b, 0, len); + } + + // 将CSV数据解析为Java对象 + ByteArrayInputStream fileInputStream = new ByteArrayInputStream(bos.toByteArray()); + InputStreamReader inputStreamReader = new InputStreamReader(fileInputStream); + BufferedReader bufferedReader = new BufferedReader(inputStreamReader); + String line = bufferedReader.readLine(); + while (line != null) { + System.out.println(line); + line = bufferedReader.readLine(); + } + + } catch (Exception e) { + e.printStackTrace(); + } finally { + try { + bos.close(); + if (gzIn != null) { + gzIn.close(); + } + } catch (IOException e) { + e.printStackTrace(); + } + } + return new byte[0]; + } + +} \ No newline at end of file diff --git a/rocketmq-streams-channel-huawei/src/test/java/org/apache/rocketmq/streams/huawei/TimeTest.java b/rocketmq-streams-channel-huawei/src/test/java/org/apache/rocketmq/streams/huawei/TimeTest.java new file mode 100644 index 00000000..f621bd5d --- /dev/null +++ b/rocketmq-streams-channel-huawei/src/test/java/org/apache/rocketmq/streams/huawei/TimeTest.java @@ -0,0 +1,154 @@ +/* + * 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.huawei; + +import java.util.Date; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.junit.Before; +import org.junit.Test; +import org.quartz.CronScheduleBuilder; +import org.quartz.Job; +import org.quartz.JobBuilder; +import org.quartz.JobDetail; +import org.quartz.JobExecutionContext; +import org.quartz.JobExecutionException; +import org.quartz.Scheduler; +import org.quartz.SchedulerException; +import org.quartz.SchedulerFactory; +import org.quartz.Trigger; +import org.quartz.TriggerBuilder; +import org.quartz.impl.StdSchedulerFactory; + +public class TimeTest { + + private String startTime = "2021-07-12 12:13:10"; + + private Integer cycle = 3; + + private String cycleUnit = "hours"; + + private long cycleTime = 0L; + + @Before + public void init() { + if (cycleUnit == null || cycleUnit.isEmpty()) { + cycleUnit = TimeUnit.SECONDS.toString(); + } + if (cycleUnit.equalsIgnoreCase(TimeUnit.DAYS.toString())) { + cycleTime = cycle * 24 * 60 * 60 * 1000; + } else if (cycleUnit.equalsIgnoreCase(TimeUnit.HOURS.toString())) { + cycleTime = cycle * 60 * 60 * 1000; + } else if (cycleUnit.equalsIgnoreCase(TimeUnit.MINUTES.toString())) { + cycleTime = cycle * 60 * 1000; + } else { + if (cycleUnit.equalsIgnoreCase(TimeUnit.SECONDS.toString())) { + cycleTime = cycle * 1000; + } + } + } + + @Test + public void test3() { + long start = DateUtil.parse("2021-07-11 11:59:10").getTime(); + long current = DateUtil.parse("2021-07-13 00:01:10").getTime(); + Long offset = DateUtil.getWindowStartWithOffset(start, 0, 24 * 60 * 60 * 1000); + System.out.println(new Date(offset)); + } + + @Test + public void test() { + long start = DateUtil.parse("2021-07-14 11:59:10").getTime(); + long current = DateUtil.parse("2021-07-14 08:01:10").getTime() - 8 * 5 * 60 * 1000; + + List dateList = DateUtil.getWindowBeginTime(current, 5 * 60 * 1000, current - start); + + for (Date date : dateList) { + System.out.println(DateUtil.format(date)); + } + } + + @Test + public void test1() throws SchedulerException { + SchedulerFactory factory = new StdSchedulerFactory(); + Scheduler scheduler = factory.getScheduler(); + scheduler.start(); + // 2.创建JobDetail实例,并与MyJob类绑定(Job执行内容) + JobDetail job = JobBuilder.newJob(MyJob.class) + .withIdentity("job1", "group1") + .build(); + + // 3.构建Trigger实例,每隔30s执行一次 + Trigger trigger = TriggerBuilder.newTrigger() + .withIdentity("trigger1", "group1") + .startNow() + .withSchedule(CronScheduleBuilder.cronSchedule("0/5 * * * * ?")) + .build(); + scheduler.scheduleJob(job, trigger); + + while (true) { + try { + Thread.sleep(60000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + } + + @Test + public void test2() { + String path = "LTS-test/%Y/%m/%done/%H/%m"; + System.out.println(path.replaceAll("%Y", "2023").replaceAll("%M", "")); + + } + + public static class MyJob implements Job { + + private Long startTime = DateUtil.parseTime("2023-07-19 01:00:03").getTime(); + private Long cycle = 5 * 1000L; // 5分钟 + + @Override + public void execute(JobExecutionContext context) throws JobExecutionException { + long currentTime = System.currentTimeMillis(); + + for (startTime = (startTime % cycle == 0 ? startTime : startTime + (cycle - startTime % cycle)); startTime < currentTime - cycle; startTime = startTime + cycle) { + System.out.println(DateUtil.format(new Date(currentTime), "yyyy-MM-dd HH:mm:ss:SSS") + "-------" + DateUtil.format(new Date(startTime), "yyyy-MM-dd HH:mm:ss:SSS")); + } + + } + + public Long getStartTime() { + return startTime; + } + + public void setStartTime(Long startTime) { + this.startTime = startTime; + } + + public Long getCycle() { + return cycle; + } + + public void setCycle(Long cycle) { + this.cycle = cycle; + } + } + +} diff --git a/rocketmq-streams-channel-kafka/pom.xml b/rocketmq-streams-channel-kafka/pom.xml index 89bd645c..8323f2d6 100644 --- a/rocketmq-streams-channel-kafka/pom.xml +++ b/rocketmq-streams-channel-kafka/pom.xml @@ -1,23 +1,17 @@ - rocketmq-streams org.apache.rocketmq - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT 4.0.0 rocketmq-streams-channel-kafka ROCKETMQ STREAMS :: channel-kafka - - 8 - 8 - - - org.apache.kafka diff --git a/rocketmq-streams-channel-kafka/src/main/java/org/apache/rocketmq/streams/kafka/KafkaChannelBuilder.java b/rocketmq-streams-channel-kafka/src/main/java/org/apache/rocketmq/streams/kafka/KafkaChannelBuilder.java index 489bcff1..75415bdf 100644 --- a/rocketmq-streams-channel-kafka/src/main/java/org/apache/rocketmq/streams/kafka/KafkaChannelBuilder.java +++ b/rocketmq-streams-channel-kafka/src/main/java/org/apache/rocketmq/streams/kafka/KafkaChannelBuilder.java @@ -71,14 +71,17 @@ protected JSONObject createFormatProperty(Properties properties) { } formatProperties.put(key, properties.getProperty(key)); } - IChannelBuilder.formatPropertiesName(formatProperties, properties, "groupName", "group.id"); - IChannelBuilder.formatPropertiesName(formatProperties, properties, "groupName", "consumerGroup"); - IChannelBuilder.formatPropertiesName(formatProperties, properties, "endpoint", "bootstrap.servers"); - IChannelBuilder.formatPropertiesName(formatProperties, properties, "maxThread", "thread.max.count"); - - IChannelBuilder.formatPropertiesName(formatProperties, properties, "groupName", "consumergroup"); - IChannelBuilder.formatPropertiesName(formatProperties, properties, "groupName", "groupname"); - IChannelBuilder.formatPropertiesName(formatProperties, properties, "maxThread", "maxthread"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "bootstrapServers", "bootstrap.servers"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "bootstrapServers", "bootstrap_servers"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "topic", "topic"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "groupName", "group_name"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "userName", "user_name"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "password", "password"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "sessionTimeout", "session_timeout"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "maxPollRecords", "max_poll_records"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "maxthread", "max_thread"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "maxRequestSize", " max_request_size"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "autoOffsetReset", "auto_offset_reset"); return formatProperties; } diff --git a/rocketmq-streams-channel-kafka/src/main/java/org/apache/rocketmq/streams/kafka/KafkaSplit.java b/rocketmq-streams-channel-kafka/src/main/java/org/apache/rocketmq/streams/kafka/KafkaSplit.java index 20fa3960..be845407 100644 --- a/rocketmq-streams-channel-kafka/src/main/java/org/apache/rocketmq/streams/kafka/KafkaSplit.java +++ b/rocketmq-streams-channel-kafka/src/main/java/org/apache/rocketmq/streams/kafka/KafkaSplit.java @@ -26,12 +26,16 @@ public class KafkaSplit extends BasedConfigurable implements ISplit kafkaProducer; - private volatile transient boolean stop = false; - protected String topic; @ENVDependence private String bootstrapServers; + private String maxRequestSize; + private String userName; + + private String password; private int sessionTimeout = 30000; public KafkaSink() { @@ -55,6 +58,10 @@ public KafkaSink(String bootstrapServers, String topic) { this.topic = topic; } + public static int getMaxPollRecords() { + return 100; + } + @Override protected boolean initConfigurable() { Properties props = new Properties(); props.put("bootstrap.servers", bootstrapServers); @@ -67,6 +74,16 @@ public KafkaSink(String bootstrapServers, String topic) { props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); props.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); props.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); + + String maxRequestSizeTmp="4194304"; + if(StringUtils.isNotEmpty(this.maxRequestSize)){ + maxRequestSizeTmp=this.maxRequestSize; + } + props.put("max.request.size",maxRequestSizeTmp); + + if (this.userName != null && this.password != null) { + props.put("sasl.jaas.config", "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"" + userName + "\" password=\"" + password + "\";"); + } this.kafkaProducer = new KafkaProducer<>(props); return super.initConfigurable(); } @@ -86,8 +103,8 @@ protected void destroyProducer() { try { kafkaProducer.close(); } catch (Throwable t) { - if (LOG.isWarnEnabled()) { - LOG.warn(t.getMessage(), t); + if (LOGGER.isWarnEnabled()) { + LOGGER.warn(t.getMessage(), t); } } } @@ -115,7 +132,7 @@ protected void destroyProducer() { NewTopic newTopic = new NewTopic(topic, splitNum, (short) 1); adminClient.createTopics(Collections.singletonList(newTopic)); - LOG.info("创建主题成功:" + topic); + LOGGER.info("创建主题成功:" + topic); } catch (Exception ex) { ex.printStackTrace(); } finally { @@ -141,17 +158,17 @@ protected void destroyProducer() { protected boolean putMessage2Mq(IMessage fieldName2Value) { try { - LOG.info(String.format("topic=%s, record=%s", topic, fieldName2Value.getMessageValue().toString())); + // LOGGER.info(String.format("topic=%s, record=%s", topic, fieldName2Value.getMessageValue().toString())); ProducerRecord records = new ProducerRecord<>(topic, fieldName2Value.getMessageValue().toString()); kafkaProducer.send(records, (recordMetadata, e) -> { if (e != null) { - //LOG.error("send kafka message error!topic=" + topic, e); + LOGGER.error("send kafka message error!topic=" + topic, e); } else { //LOG.info(String.format("send success topic=%s, record=%s", topic, jsonObject.toJSONString())); } }); } catch (Exception e) { - LOG.error("send message error:" + fieldName2Value.getMessageValue().toString(), e); + LOGGER.error("send message error:" + fieldName2Value.getMessageValue().toString(), e); return false; } return true; @@ -161,10 +178,6 @@ protected boolean putMessage2Mq(IMessage fieldName2Value) { return getSplitList().size(); } - public static int getMaxPollRecords() { - return 100; - } - public boolean isStop() { return stop; } @@ -197,4 +210,27 @@ public void setTopic(String topic) { this.topic = topic; } + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } + + public String getMaxRequestSize() { + return maxRequestSize; + } + + public void setMaxRequestSize(String maxRequestSize) { + this.maxRequestSize = maxRequestSize; + } } diff --git a/rocketmq-streams-channel-kafka/src/main/java/org/apache/rocketmq/streams/kafka/source/KafkaSource.java b/rocketmq-streams-channel-kafka/src/main/java/org/apache/rocketmq/streams/kafka/source/KafkaSource.java index a062a0f7..00e2aa92 100644 --- a/rocketmq-streams-channel-kafka/src/main/java/org/apache/rocketmq/streams/kafka/source/KafkaSource.java +++ b/rocketmq-streams-channel-kafka/src/main/java/org/apache/rocketmq/streams/kafka/source/KafkaSource.java @@ -16,8 +16,8 @@ */ package org.apache.rocketmq.streams.kafka.source; -import com.alibaba.fastjson.JSONObject; import com.google.common.collect.Lists; +import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -25,29 +25,36 @@ import java.util.Properties; import java.util.Set; import java.util.concurrent.ExecutorService; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; -import org.apache.rocketmq.streams.common.channel.source.AbstractSupportShuffleSource; -import org.apache.rocketmq.streams.common.context.Message; +import org.apache.rocketmq.streams.common.channel.source.AbstractPushSource; +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.threadpool.ThreadPoolFactory; +import org.apache.rocketmq.streams.kafka.KafkaSplit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class KafkaSource extends AbstractSupportShuffleSource { +public class KafkaSource extends AbstractPushSource { - private static final Log LOG = LogFactory.getLog(KafkaSource.class); + private static final Logger LOGGER = LoggerFactory.getLogger(KafkaSource.class); private transient Properties props; private String bootstrapServers; + + private String userName; + + private String password; + private String autoOffsetReset = "earliest"; + private transient KafkaConsumer consumer; private int maxPollRecords = 100; private volatile transient boolean stop = false; + private volatile transient boolean isFinished = false; private int sessionTimeout = 30000; + private transient ExecutorService executorService; public KafkaSource() { } @@ -73,9 +80,14 @@ public KafkaSource(String bootstrapServers, String topic, String groupName) { props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); props.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); props.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); - props.put("auto.offset.reset", "earliest"); + props.put("auto.offset.reset", autoOffsetReset); props.put("key.serializer.encoding", getEncoding()); props.put("value.serializer.encoding", getEncoding()); + props.put("security.protocol", "SASL_PLAINTEXT"); + props.put("sasl.mechanism", "PLAIN"); + if (this.userName != null && this.password != null) { + props.put("sasl.jaas.config", "org.apache.kafka.common.security.plain.PlainLoginModule required serviceName=\"" + this.getNameSpace() + "_" + this.getName() + "\" username=\"" + userName + "\" password=\"" + password + "\";"); + } this.props = props; return super.initConfigurable(); } @@ -88,57 +100,24 @@ public KafkaSource(String bootstrapServers, String topic, String groupName) { this.consumer.subscribe(Lists.newArrayList(topic)); } WorkerFunc workerFunc = new WorkerFunc(); - ExecutorService executorService = new ThreadPoolExecutor(getMaxThread(), getMaxThread(), 0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue(1000)); + if (executorService == null) { + executorService = ThreadPoolFactory.createFixedThreadPool(getMaxThread(), KafkaSource.class.getName() + "-" + getName()); + } executorService.execute(workerFunc); } catch (Exception e) { - setInitSuccess(false); - LOG.error(e.getMessage(), e); - destroy(); + LOGGER.error(e.getMessage(), e); + try { + destroy(); + } catch (Exception ex) { + throw new RuntimeException(ex); + } throw new RuntimeException(" start kafka channel error " + topic); } return true; } - protected class WorkerFunc implements Runnable { - - @Override public void run() { - long lastUpgrade = System.currentTimeMillis(); - List oldPartitionInfos = consumer.partitionsFor(topic); - while (!stop) { - try { - ConsumerRecords records = consumer.poll(1000); - List newPartitionInfos = consumer.partitionsFor(topic); - messageQueueChanged(oldPartitionInfos, newPartitionInfos); - Set queueIds = new HashSet<>(); - for (ConsumerRecord record : records) { - try { - queueIds.add(record.partition() + ""); - Map parameters = new HashMap<>(); - if (getHeaderFieldNames() != null) { - parameters.put("messageKey", record.key()); - parameters.put("topic", record.topic()); - parameters.put("partition", record.partition()); - parameters.put("offset", record.offset()); - parameters.put("timestamp", record.timestamp()); - } - JSONObject msg = create(record.value(), parameters); - Message message = createMessage(msg, record.partition() + "", record.offset() + "", false); - message.getHeader().setOffsetIsLong(true); - executeMessage(message); - } catch (Exception e) { - LOG.error(e.getMessage(), e); - } - } - if ((System.currentTimeMillis() - lastUpgrade) > checkpointTime) { - sendCheckpoint(queueIds); - consumer.commitAsync(); - lastUpgrade = System.currentTimeMillis(); - } - } catch (Exception e) { - LOG.error(e.getMessage(), e); - } - } - } + @Override protected boolean hasListenerSplitChanged() { + return true; } public void messageQueueChanged(List old, List newPartitions) { @@ -167,30 +146,46 @@ public void messageQueueChanged(List old, List new } - @Override public boolean supportNewSplitFind() { - return true; - } - - @Override public boolean supportRemoveSplitFind() { - return true; - } + @Override public List> fetchAllSplits() { + KafkaConsumer consumer = new KafkaConsumer<>(props); + consumer.subscribe(Lists.newArrayList(topic)); + try { + List partitionInfos = consumer.partitionsFor(topic); + List> splits = new ArrayList<>(); + for (PartitionInfo partitionInfo : partitionInfos) { + splits.add(new KafkaSplit(partitionInfo)); + } + return splits; - @Override public boolean supportOffsetRest() { - return false; - } + } finally { + consumer.close(); + } - @Override protected boolean isNotDataSplit(String queueId) { - return false; } protected void destroyConsumer() { + this.stop = true; + + while (!isFinished) { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } if (this.consumer != null) { this.consumer.close(); + + } + this.stop = false; + this.isFinished = false; + if (this.executorService != null) { + this.executorService.shutdown(); + this.executorService = null; } } - @Override public void destroy() { - super.destroy(); + @Override public void destroySource() { stop = true; destroyConsumer(); } @@ -199,6 +194,10 @@ public int getMaxPollRecords() { return maxPollRecords; } + public void setMaxPollRecords(int maxPollRecords) { + this.maxPollRecords = maxPollRecords; + } + public boolean isStop() { return stop; } @@ -215,6 +214,22 @@ public void setBootstrapServers(String bootstrapServers) { this.bootstrapServers = bootstrapServers; } + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } + @Override public void setMaxFetchLogGroupSize(int size) { super.setMaxFetchLogGroupSize(size); maxPollRecords = size; @@ -235,4 +250,52 @@ public KafkaConsumer getConsumer() { public void setConsumer(KafkaConsumer consumer) { this.consumer = consumer; } + + protected class WorkerFunc implements Runnable { + + @Override public void run() { + long lastUpgrade = System.currentTimeMillis(); + List oldPartitionInfos = consumer.partitionsFor(topic); + while (!stop) { + try { + ConsumerRecords records = consumer.poll(1000); + List newPartitionInfos = consumer.partitionsFor(topic); + messageQueueChanged(oldPartitionInfos, newPartitionInfos); + Set queueIds = new HashSet<>(); + for (ConsumerRecord record : records) { + try { + queueIds.add(record.partition() + ""); + Map parameters = new HashMap<>(); + if (getHeaderFieldNames() != null) { + parameters.put("messageKey", record.key()); + parameters.put("topic", record.topic()); + parameters.put("partition", record.partition()); + parameters.put("offset", record.offset()); + parameters.put("timestamp", record.timestamp()); + } + doReceiveMessage(record.value(), false, record.partition() + "", record.offset() + "", parameters); + } catch (Exception e) { + LOGGER.error(e.getMessage(), e); + } + } + if ((System.currentTimeMillis() - lastUpgrade) > checkpointTime) { + sendCheckpoint(queueIds); + consumer.commitAsync(); + lastUpgrade = System.currentTimeMillis(); + } + } catch (Exception e) { + LOGGER.error(e.getMessage(), e); + } + } + isFinished = true; + } + } + + public String getAutoOffsetReset() { + return autoOffsetReset; + } + + public void setAutoOffsetReset(String autoOffsetReset) { + this.autoOffsetReset = autoOffsetReset; + } } diff --git a/rocketmq-streams-channel-kafka/src/test/java/org/apache/rocketmq/streams/kafka/KafkaChannelTest.java b/rocketmq-streams-channel-kafka/src/test/java/org/apache/rocketmq/streams/kafka/KafkaChannelTest.java index 73e9745b..4d244c36 100644 --- a/rocketmq-streams-channel-kafka/src/test/java/org/apache/rocketmq/streams/kafka/KafkaChannelTest.java +++ b/rocketmq-streams-channel-kafka/src/test/java/org/apache/rocketmq/streams/kafka/KafkaChannelTest.java @@ -17,7 +17,6 @@ package org.apache.rocketmq.streams.kafka; 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.context.Message; @@ -29,10 +28,11 @@ * 本机搭建kafka服务端参考:https://www.cnblogs.com/BlueSkyyj/p/11425998.html **/ public class KafkaChannelTest { - private static final String END_POINT = "47.94.238.133:31013"; - private static final String TOPIC = "real_time_kafka_topic"; + //private static final String END_POINT = "47.94.238.133:31013"; + private static final String END_POINT = "ckafka-zarvdwxe.ap-guangzhou.ckafka.tencentcloudmq.com:6001"; + private static final String TOPIC = "cfw_alert"; //数据任务处理后topic:es_index_test,实时策略消息topic:real_time_kafka_topic - private static final String GROUP_NAME = "test-090"; + private static final String GROUP_NAME = "cfw"; @Test public void testKafkaReceive() throws InterruptedException { @@ -84,7 +84,7 @@ private ISink createSink() { kafkaChannel.setTopic(TOPIC); kafkaChannel.setBootstrapServers(END_POINT); kafkaChannel.setNameSpace("com.aliyun.dipper.test"); - kafkaChannel.setConfigureName("kafka_channel"); + kafkaChannel.setName("kafka_channel"); kafkaChannel.init(); return kafkaChannel; } @@ -97,7 +97,7 @@ private ISource createSource() { kafkaChannel.setGroupName(GROUP_NAME); kafkaChannel.setMaxThread(1); kafkaChannel.setNameSpace("com.aliyun.dipper.test"); - kafkaChannel.setConfigureName("kafka_channel"); + kafkaChannel.setName("kafka_channel"); kafkaChannel.init(); return kafkaChannel; } diff --git a/rocketmq-streams-channel-kafka/src/test/resources/log4j.xml b/rocketmq-streams-channel-kafka/src/test/resources/log4j.xml old mode 100755 new mode 100644 diff --git a/rocketmq-streams-checkpoint/pom.xml b/rocketmq-streams-channel-metaq/pom.xml similarity index 53% rename from rocketmq-streams-checkpoint/pom.xml rename to rocketmq-streams-channel-metaq/pom.xml index 312de599..19dc73e5 100644 --- a/rocketmq-streams-checkpoint/pom.xml +++ b/rocketmq-streams-channel-metaq/pom.xml @@ -15,52 +15,57 @@ See the License for the specific language governing permissions and limitations under the License. --> - rocketmq-streams org.apache.rocketmq - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT 4.0.0 - rocketmq-streams-checkpoint - ROCKETMQ STREAMS :: checkpoint + rocketmq-streams-channel-metaq + ROCKETMQ STREAMS :: channel-metaq jar - - - 8 - 8 - - org.apache.rocketmq - rocketmq-streams-commons + com.taobao.metaq.final + metaq-client - com.google.auto.service - auto-service + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core - - org.apache.rocketmq - rocketmq-streams-db-operator + com.alibaba.rocketmq + rocketmq-tools - com.google.auto.service - auto-service + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + 4.0.2 + + + com.alibaba.rocketmq + rocketmq-common + 4.0.2 - - com.google.auto.service - auto-service - true + org.apache.rocketmq + rocketmq-streams-serviceloader - - \ No newline at end of file + diff --git a/rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/MetaqChannelBuilder.java b/rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/MetaqChannelBuilder.java new file mode 100644 index 00000000..bead6b83 --- /dev/null +++ b/rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/MetaqChannelBuilder.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.metaq; + +import com.alibaba.fastjson.JSONObject; +import com.google.auto.service.AutoService; +import java.util.Properties; +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.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; +import org.apache.rocketmq.streams.metaq.sink.MetaqSink; +import org.apache.rocketmq.streams.metaq.source.MetaqSource; + +@AutoService(IChannelBuilder.class) +@ServiceName(value = MetaqChannelBuilder.TYPE, aliasName = "MetaqSource") +public class MetaqChannelBuilder extends AbstractSupportShuffleChannelBuilder { + public static final String TYPE = "metaq"; + + /** + * @param namespace + * @param name + * @param properties + * @return + */ + @Override + public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) { + + MetaqSource metaqChannel = (MetaqSource) ConfigurableUtil.create(MetaqSource.class.getName(), namespace, name, createFormatProperty(properties), null); + return metaqChannel; + } + + @Override + public String getType() { + return TYPE; + } + + @Override + public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { + MetaqSink metaqChannel = (MetaqSink) ConfigurableUtil.create(MetaqSink.class.getName(), namespace, name, createFormatProperty(properties), null); + return metaqChannel; + } + + /** + * 创建标准的属性文件 + * + * @param properties + * @return + */ + @Override + 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, "tags", "tag"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "groupName", "group"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "maxThread", "thread.max.count"); + return formatProperties; + } + + @Override + public ISink createBySource(ISource pipelineSource) { + MetaqSource metaqSource = (MetaqSource) pipelineSource; + String topic = metaqSource.getTopic(); + MetaqSink sink = new MetaqSink(topic, metaqSource.getTags()); + return sink; + } +} diff --git a/rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/MetaqOffset.java b/rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/MetaqOffset.java new file mode 100644 index 00000000..975d1120 --- /dev/null +++ b/rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/MetaqOffset.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.metaq; + +import com.alibaba.rocketmq.client.consumer.store.OffsetStore; +import com.alibaba.rocketmq.client.consumer.store.ReadOffsetType; +import com.alibaba.rocketmq.client.exception.MQBrokerException; +import com.alibaba.rocketmq.client.exception.MQClientException; +import com.alibaba.rocketmq.common.message.MessageQueue; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.rocketmq.streams.common.utils.ReflectUtil; +import org.apache.rocketmq.streams.metaq.debug.DebugWriter; +import org.apache.rocketmq.streams.metaq.queue.MetaqMessageQueue; +import org.apache.rocketmq.streams.metaq.source.MetaqSource; + +public class MetaqOffset implements OffsetStore { + protected OffsetStore offsetStore; + protected MetaqSource metaqSource; + protected transient volatile Long time = null; + protected transient Long time1 = null; + + public MetaqOffset(OffsetStore offsetStore, MetaqSource metaqSource) { + this.offsetStore = offsetStore; + this.metaqSource = metaqSource; + } + + @Override + public void load() throws MQClientException { + offsetStore.load(); + } + + @Override + public void updateOffset(MessageQueue mq, long offset, boolean increaseOnly) { + offsetStore.updateOffset(mq, offset, increaseOnly); + } + + @Override + public long readOffset(MessageQueue mq, ReadOffsetType type) { + return offsetStore.readOffset(mq, type); + } + + @Override + public void persistAll(Set mqs) { + if (mqs.size() == 0) { + return; + } + Set queueIds = new HashSet<>(); + for (MessageQueue mq : mqs) { + MetaqMessageQueue metaqMessageQueue = new MetaqMessageQueue(mq); + queueIds.add(metaqMessageQueue.getQueueId()); + } + //ConcurrentHashMap offsetTable=ReflectUtil.getDeclaredField(offsetStore,"offsetTable"); + //metaqSource.fileOffset.save(offsetTable); + metaqSource.sendCheckpoint(queueIds); + if (DebugWriter.isOpenDebug()) { + ConcurrentMap offsetTable = ReflectUtil.getDeclaredField(this.offsetStore, "offsetTable"); + DebugWriter.getInstance(metaqSource.getTopic()).writeSaveOffset(offsetTable); + } + offsetStore.persistAll(mqs); + } + + @Override + public void persist(MessageQueue mq) { + MetaqMessageQueue metaqMessageQueue = new MetaqMessageQueue(mq); + metaqSource.sendCheckpoint(metaqMessageQueue.getQueueId()); + if (DebugWriter.isOpenDebug()) { + ConcurrentMap offsetTable = ReflectUtil.getDeclaredField(this.offsetStore, "offsetTable"); + DebugWriter.getInstance(metaqMessageQueue.getTopic()).writeSaveOffset(mq, offsetTable.get(mq)); + } + offsetStore.persist(mq); + } + + @Override + public void removeOffset(MessageQueue mq) { + Set splitIds = new HashSet<>(); + splitIds.add(new MetaqMessageQueue(mq).getQueueId()); + metaqSource.removeSplit(splitIds); + offsetStore.removeOffset(mq); + } + + @Override + public Map cloneOffsetTable(String topic) { + return offsetStore.cloneOffsetTable(topic); + } + + @Override + public void updateConsumeOffsetToBroker(MessageQueue mq, long offset, boolean isOneway) + throws MQBrokerException, InterruptedException, MQClientException, com.alibaba.rocketmq.remoting.exception.RemotingException { + offsetStore.updateConsumeOffsetToBroker(mq, offset, isOneway); + } +} diff --git a/rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/debug/DebugWriter.java b/rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/debug/DebugWriter.java new file mode 100644 index 00000000..71923c7b --- /dev/null +++ b/rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/debug/DebugWriter.java @@ -0,0 +1,114 @@ +package org.apache.rocketmq.streams.metaq.debug; + +import com.alibaba.fastjson.JSONObject; +import com.alibaba.rocketmq.common.message.MessageQueue; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.common.utils.FileUtil; +import org.apache.rocketmq.streams.metaq.queue.MetaqMessageQueue; + +public class DebugWriter { + + protected static Map debugWriterMap = new HashMap<>(); + protected String dir = "/tmp/rocketmq-streams/mq"; + protected String topic; + + public DebugWriter(String topic) { + this.topic = topic; + this.dir = this.dir + "/" + topic.replaceAll("\\.", "_"); + } + + public DebugWriter(String topic, String dir) { + this.topic = topic; + this.dir = dir; + } + + public static DebugWriter getInstance(String topic) { + DebugWriter debugWriter = debugWriterMap.get(topic); + if (debugWriter == null) { + debugWriter = new DebugWriter(topic); + debugWriterMap.put(topic, debugWriter); + } + return debugWriter; + } + + public static boolean isOpenDebug() { + return ComponentCreator.getPropertyBooleanValue("metq.debug.switch"); + } + + /** + * write offset 2 file + * + * @param offsets + */ + public void writeSaveOffset(Map offsets) { + String path = dir + "/offsets/offset.txt"; + if (offsets == null || offsets.size() == 0) { + return; + } + Iterator> it = offsets.entrySet().iterator(); + List rows = new ArrayList<>(); + while (it.hasNext()) { + Map.Entry entry = it.next(); + String queueId = new MetaqMessageQueue(entry.getKey()).getQueueId(); + if (queueId.indexOf("%") != -1) { + continue; + } + JSONObject msg = new JSONObject(); + Long offset = entry.getValue().get(); + msg.put(queueId, offset); + msg.put("saveTime", DateUtil.getCurrentTimeString()); + msg.put("queueId", queueId); + msg.put("saveCheckPoint", true); + rows.add(msg.toJSONString()); + } + FileUtil.write(path, rows, true); + } + + public void writeSaveOffset(MessageQueue messageQueue, AtomicLong offset) { + Map offsets = new HashMap<>(); + offsets.put(messageQueue, offset); + writeSaveOffset(offsets); + } + + public void receiveFirstData(String queueId, Long offset) { + String path = dir + "/offsets/offset.txt"; + Map offsets = load(); + Long saveOffset = offsets.get(queueId); + JSONObject msg = new JSONObject(); + msg.put(queueId, offset); + msg.put("receiver", true); + msg.put("save_offset", saveOffset); + msg.put("current_time", DateUtil.getCurrentTimeString()); + List rows = new ArrayList<>(); + rows.add(msg.toJSONString()); + FileUtil.write(path, rows, true); + System.out.println("queueId is " + queueId + "current offset " + offset + "====" + saveOffset); + } + + /** + * load offsets + * + * @return + */ + public Map load() { + String path = dir + "/offsets/offset.txt"; + List lines = FileUtil.loadFileLine(path); + Map offsets = new HashMap<>(); + for (String line : lines) { + JSONObject row = JSONObject.parseObject(line); + if (row.getBoolean("saveCheckPoint") == null || !row.getBoolean("saveCheckPoint")) { + continue; + } + String queueId = row.getString("queueId"); + offsets.put(queueId, row.getLong(queueId)); + } + return offsets; + } +} diff --git a/rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/queue/MetaqMessageQueue.java b/rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/queue/MetaqMessageQueue.java new file mode 100644 index 00000000..abbe9b60 --- /dev/null +++ b/rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/queue/MetaqMessageQueue.java @@ -0,0 +1,131 @@ +/* + * 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.metaq.queue; + +import com.alibaba.fastjson.JSONObject; +import com.alibaba.rocketmq.common.message.MessageQueue; +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; + +public class MetaqMessageQueue extends BasedConfigurable implements ISplit { + protected transient MessageQueue queue; + protected String brokeName; + protected String topic; + protected int mqQueueId; + + public MetaqMessageQueue(String topic, String brokeName, int mqQueueId) { + this.topic = topic; + this.brokeName = brokeName; + this.mqQueueId = mqQueueId; + this.queue = new MessageQueue(topic, brokeName, mqQueueId); + } + + public MetaqMessageQueue(MessageQueue queue) { + this.queue = queue; + this.brokeName = queue.getBrokerName(); + this.topic = queue.getTopic(); + this.mqQueueId = queue.getQueueId(); + } + + public MetaqMessageQueue(org.apache.rocketmq.common.message.MessageQueue queue) { + this(queue.getTopic(), queue.getBrokerName(), queue.getQueueId()); + } + + public MetaqMessageQueue() { + + } + + public static String getQueueId(MessageQueue queue) { + + return (MapKeyUtil.createKeyBySign("_", queue.getTopic(), queue.getBrokerName(), getSplitNumberStr(queue.getQueueId()) + "")); + } + +// public org.apache.rocketmq.common.message.MessageQueue getRocketMQQueue(){ +// return new org.apache.rocketmq.common.message.MessageQueue(topic,brokeName,mqQueueId); +// } + + public static String getQueueId(org.apache.rocketmq.common.message.MessageQueue queue) { + + return (MapKeyUtil.createKeyBySign("_", queue.getTopic(), queue.getBrokerName(), getSplitNumberStr(queue.getQueueId()) + "")); + } + + /** + * 获取分片的字符串格式,需要3位对齐 + * + * @param splitNumber + * @return + */ + private static String getSplitNumberStr(int splitNumber) { + int len = (splitNumber + "").length(); + if (len == 3) { + return splitNumber + ""; + } + String splitNumerStr = splitNumber + ""; + while (len < 3) { + splitNumerStr = "0" + splitNumerStr; + len = splitNumerStr.length(); + } + return splitNumerStr; + } + + @Override + protected void getJsonObject(JSONObject jsonObject) { + super.getJsonObject(jsonObject); + queue = new MessageQueue(topic, brokeName, mqQueueId); + } + + @Override + public MessageQueue getQueue() { + return queue; + } + + @Override + public int compareTo(MetaqMessageQueue o) { + return queue.compareTo(o.queue); + } + + @Override + public String getQueueId() { + return getQueueId(this.queue); + } + + public String getBrokeName() { + return brokeName; + } + + public void setBrokeName(String brokeName) { + this.brokeName = brokeName; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public int getMqQueueId() { + return mqQueueId; + } + + public void setMqQueueId(int mqQueueId) { + this.mqQueueId = mqQueueId; + } + +} \ No newline at end of file diff --git a/rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/sink/MetaqSink.java b/rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/sink/MetaqSink.java new file mode 100644 index 00000000..c948cd58 --- /dev/null +++ b/rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/sink/MetaqSink.java @@ -0,0 +1,276 @@ +/* + * 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.metaq.sink; + +import com.alibaba.rocketmq.common.MixAll; +import com.alibaba.rocketmq.common.message.Message; +import com.alibaba.rocketmq.common.message.MessageQueue; +import com.alibaba.rocketmq.common.protocol.body.TopicList; +import com.alibaba.rocketmq.tools.admin.DefaultMQAdminExt; +import com.taobao.metaq.client.MetaProducer; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +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.annotation.ENVDependence; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.metaq.queue.MetaqMessageQueue; + +public class MetaqSink extends AbstractSupportShuffleSink { + + protected static final Log LOG = LogFactory.getLog(MetaqSink.class); + private static final String PREFIX = "dipper.upgrade.channel.metaq.envkey"; + @ENVDependence + protected String tags = "*"; // 每个消息队列都会有一个线程 + protected transient MetaProducer producer; + protected String topic; + + public MetaqSink() { + } + + public MetaqSink(String topic, String tags) { + setTopic(topic); + setTags(tags); + } + + public MetaqSink(String topic) { + setTopic(topic); + } + + public static void main(String[] args) throws InterruptedException { + String topic = "TOPIC_DIPPER_SYSTEM_MSG_9"; + MetaqSink metaqSink = new MetaqSink(topic); + metaqSink.setSplitNum(5); + metaqSink.init(); + System.out.println(metaqSink.getSplitNum()); + // System.out.println(metaqSink.getSplitList()); + // MetaqSource metaqSource=new MetaqSource(topic,"fds"); + // metaqSource.init(); + // metaqSource.start(new IStreamOperator() { + // @Override + // public Object doMessage(IMessage message, AbstractContext context) { + // System.out.println(message.getHeader().getQueueId()); + // return null; + // } + // }); + Thread.sleep(100000000); + //DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt(); + //defaultMQAdminExt.setVipChannelEnabled(false); + //defaultMQAdminExt.setInstanceName(Long.toString(System.currentTimeMillis())); + // + //try { + // defaultMQAdminExt.start(); + // + // TopicRouteData topicRouteData = defaultMQAdminExt.examineTopicRouteInfo(topic); + // String json = topicRouteData.toJson(true); + // System.out.printf("%s%n", json); + //} catch (Exception e) { + // e.printStackTrace(); + //} finally { + // defaultMQAdminExt.shutdown(); + //} + } + + public void destroyProduce() { + if (producer != null) { + try { + producer.shutdown(); + producer = null; + } catch (Throwable t) { + if (LOG.isWarnEnabled()) { + LOG.warn(t.getMessage(), t); + } + } + } + } + + @Override + public void destroy() { + super.destroy(); + destroyProduce(); + } + + public String getTags() { + return tags; + } + + public void setTags(String tags) { + this.tags = tags; + } + + @Override + protected boolean batchInsert(List messages) { + if (messages == null) { + return true; + } + if (StringUtil.isEmpty(topic)) { + if (LOG.isErrorEnabled()) { + LOG.error("topic is blank"); + } + return false; + } + initProducer(); + + try { + Map> msgsByQueueId = new HashMap<>();// group by queueId, if the message not contains queue info ,the set default string as default queueId + Map messageQueueMap = new HashMap<>();//if has queue id in message, save the map for queueid 2 messagequeeue + String defaultQueueId = "";//message is not contains queue ,use default + for (IMessage msg : messages) { + ISplit channelQueue = getSplit(msg); + String queueId = defaultQueueId; + if (channelQueue != null) { + queueId = channelQueue.getQueueId(); + MetaqMessageQueue metaqMessageQueue = (MetaqMessageQueue) channelQueue; + messageQueueMap.put(queueId, metaqMessageQueue.getQueue()); + } + List messageList = msgsByQueueId.get(queueId); + if (messageList == null) { + messageList = new ArrayList<>(); + msgsByQueueId.put(queueId, messageList); + } + messageList.add(new Message(topic, tags, null, msg.getMessageBody().toJSONString().getBytes("UTF-8"))); + } + List messageList = msgsByQueueId.get(defaultQueueId); + if (messageList != null) { + for (Message message : messageList) { + producer.sendOneway(message); + } + messageQueueMap.remove(defaultQueueId); + } + if (messageQueueMap.size() <= 0) { + return true; + } + for (String queueId : msgsByQueueId.keySet()) { + messageList = msgsByQueueId.get(queueId); + for (Message message : messageList) { + MessageQueue queue = messageQueueMap.get(queueId); + producer.send(message, queue); + } + + } + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException("batch insert error ", e); + } + + return true; + } + + protected void initProducer() { + if (producer == null) { + synchronized (this) { + if (producer == null) { + destroy(); + + producer = new MetaProducer(UUID.randomUUID() + "producer"); + producer.setSendMessageWithVIPChannel(false);//add by 明亮 + producer.setInstanceName(UUID.randomUUID().toString()); + try { + producer.start(); + } catch (Exception e) { + throw new RuntimeException("创建队列失败," + topic + ",msg=" + e.getMessage(), e); + } + } + } + + } + + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + @Override + public String getShuffleTopicFieldName() { + return "topic"; + } + + @Override + protected void createTopicIfNotExist(int splitNum) { + + DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt(10000); + defaultMQAdminExt.setVipChannelEnabled(false); + defaultMQAdminExt.setInstanceName(Long.toString(System.currentTimeMillis())); + try { + + defaultMQAdminExt.start(); + TopicList topicList = defaultMQAdminExt.fetchAllTopicList(); + for (String topic : topicList.getTopicList()) { + if (topic.equals(this.topic)) { + return; + } + } + defaultMQAdminExt.createTopic(MixAll.DEFAULT_TOPIC, topic, splitNum, 1); + + } catch (Exception e) { + throw new RuntimeException("create topic error " + topic, e); + } finally { + defaultMQAdminExt.shutdown(); + } + } + + @Override + public List> getSplitList() { + initProducer(); + List> messageQueues = new ArrayList<>(); + try { + + List metaqQueueSet = producer.fetchPublishMessageQueues(topic); + List> queueList = new ArrayList<>(); + for (MessageQueue queue : metaqQueueSet) { + MetaqMessageQueue metaqMessageQueue = new MetaqMessageQueue(queue); + queueList.add(metaqMessageQueue); + + } + queueList.sort((Comparator) Comparable::compareTo); + messageQueues = queueList; + } catch (Exception e) { + throw new RuntimeException(e); + } + + return messageQueues; + } + + @Override + public int getSplitNum() { + List> splits = getSplitList(); + if (splits == null || splits.size() == 0) { + return 0; + } + Set splitNames = new HashSet<>(); + for (ISplit split : splits) { + MessageQueue messageQueue = (MessageQueue) split.getQueue(); + splitNames.add(messageQueue.getQueueId()); + } + return splitNames.size(); + } + +} diff --git a/rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/source/MetaqSource.java b/rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/source/MetaqSource.java new file mode 100644 index 00000000..11330b91 --- /dev/null +++ b/rocketmq-streams-channel-metaq/src/main/java/org/apache/rocketmq/streams/metaq/source/MetaqSource.java @@ -0,0 +1,328 @@ +/* + * 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.metaq.source; + +import com.alibaba.fastjson.JSONObject; +import com.alibaba.rocketmq.client.consumer.listener.ConsumeOrderlyStatus; +import com.alibaba.rocketmq.client.consumer.listener.MessageListenerOrderly; +import com.alibaba.rocketmq.client.consumer.store.RemoteBrokerOffsetStore; +import com.alibaba.rocketmq.client.exception.MQClientException; +import com.alibaba.rocketmq.client.impl.MQClientManager; +import com.alibaba.rocketmq.client.impl.consumer.DefaultMQPushConsumerImpl; +import com.alibaba.rocketmq.client.impl.factory.MQClientInstance; +import com.alibaba.rocketmq.common.consumer.ConsumeFromWhere; +import com.alibaba.rocketmq.common.message.MessageExt; +import com.alibaba.rocketmq.common.message.MessageQueue; +import com.alibaba.rocketmq.common.protocol.body.Connection; +import com.taobao.metaq.client.MetaPushConsumer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.rocketmq.common.protocol.body.ConsumerConnection; +import org.apache.rocketmq.common.protocol.body.ConsumerRunningInfo; +import org.apache.rocketmq.streams.common.channel.source.AbstractPushSource; +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.AbstractContext; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; +import org.apache.rocketmq.streams.common.utils.RuntimeUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.metaq.MetaqOffset; +import org.apache.rocketmq.streams.metaq.debug.DebugWriter; +import org.apache.rocketmq.streams.metaq.queue.MetaqMessageQueue; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; + +public class MetaqSource extends AbstractPushSource { + + protected static final Log LOG = LogFactory.getLog(MetaqSource.class); + @ENVDependence + protected String tags = "*"; // 每个消息队列都会有一个线程 + protected Long pullIntervalMs; + protected transient ConsumeFromWhere consumeFromWhere;//默认从哪里消费,不会被持久化。不设置默认从尾部消费 + protected transient String consumerOffset;//从哪里开始消费 + protected transient List> messageQueues;//topic对应的queue全集 + private transient MetaPushConsumer consumer = null; + + public MetaqSource() { + } + + public MetaqSource(String topic, String tags, String groupName) { + setTopic(topic); + setTags(tags); + setGroupName(groupName); + } + + public MetaqSource(String topic, String groupName) { + setTopic(topic); + setGroupName(groupName); + } + + public static void main(String[] args) throws InterruptedException { + MetaqSource metaqSource = new MetaqSource("TOPIC_DIPPER_SYSTEM_MSG_6", "fdsdf"); + metaqSource.init(); + metaqSource.start(new IStreamOperator() { + @Override public Object doMessage(IMessage message, AbstractContext context) { + // System.out.println(message.getMessageBody()); + return null; + } + }); + System.out.println(metaqSource.fetchAllSplits().size()); + while (true) { + Map>> map = metaqSource.getWorkingSplitsGroupByInstances(); + List> ownerSplits = map.get(RuntimeUtil.getDipperInstanceId()); + int count = 0; + if (ownerSplits != null) { + count = ownerSplits.size(); + } + int sum = 0; + for (List> splits : map.values()) { + sum += splits.size(); + } + System.out.println(count + " " + sum); + Thread.sleep(1000); + } + } + + @Override + protected boolean startSource() { + try { + destroyConsumer(); + consumer = startConsumer(); + return true; + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException("start metaq channel error " + topic, e); + } + } + + protected MetaPushConsumer startConsumer() { + try { + MetaPushConsumer consumer = new MetaPushConsumer(groupName); + consumer.getMetaPushConsumerImpl().setVipChannelEnabled(false);//add by 明亮 + if (pullIntervalMs != null) { + consumer.setPullInterval(pullIntervalMs); + } + consumer.setInstanceName(RuntimeUtil.getDipperInstanceId()); + consumer.setConsumeThreadMax(maxThread); + consumer.setConsumeThreadMin(maxThread); + consumer.subscribe(topic, tags); + consumer.setPersistConsumerOffsetInterval((int) this.checkpointTime); + consumer.setConsumeMessageBatchMaxSize(maxFetchLogGroupSize); + if (consumeFromWhere != null) { + consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_TIMESTAMP); + if (consumerOffset != null) { + consumer.setConsumeTimestamp(consumerOffset); + } + } + Map isFirstDataForQueue = new HashMap<>(); + // consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_TIMESTAMP); + //consumer.setConsumeTimestamp("20210307094606"); + consumer.registerMessageListener((MessageListenerOrderly) (msgs, context) -> { + try { + Thread.currentThread().setName(Thread.currentThread().getName()); + for (MessageExt msg : msgs) { + JSONObject jsonObject = create(msg.getBody(), msg.getProperties()); + String queueId = MetaqMessageQueue.getQueueId(context.getMessageQueue()); + String offset = msg.getQueueOffset() + ""; + org.apache.rocketmq.streams.common.context.Message message = createMessage(jsonObject, queueId, offset, false); + message.getHeader().setOffsetIsLong(true); + if (DebugWriter.isOpenDebug()) { + Boolean isFirstData = isFirstDataForQueue.get(queueId); + if (isFirstData == null) { + synchronized (this) { + isFirstData = isFirstDataForQueue.get(queueId); + if (isFirstData == null) { + isFirstDataForQueue.put(queueId, true); + } + DebugWriter.getInstance(getTopic()).receiveFirstData(queueId, msg.getQueueOffset()); + } + } + } + executeMessage(message); + } + } catch (Exception e) { + LOG.error("消费metaq报错:" + e, e); + } + + return ConsumeOrderlyStatus.SUCCESS;// 返回消费成功 + }); + //设置offset的存储器,对原有的实现做包装,在提交offset前,发送系统消息 + setOffsetStore(consumer); + consumer.start(); + + //consumer.getOffsetStore(); + return consumer; + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException("start metaq channel error " + topic, e); + } + } + + @Override public Map>> getWorkingSplitsGroupByInstances() { + DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt(); + defaultMQAdminExt.setVipChannelEnabled(false); + defaultMQAdminExt.setAdminExtGroup(UUID.randomUUID().toString()); + defaultMQAdminExt.setInstanceName(this.consumer.getInstanceName()); + try { + defaultMQAdminExt.start(); + Map queue2Instances = getMessageQueueAllocationResult(defaultMQAdminExt, this.groupName); + Map>> instanceOwnerQueues = new HashMap<>(); + for (org.apache.rocketmq.common.message.MessageQueue messageQueue : queue2Instances.keySet()) { + MetaqMessageQueue metaqMessageQueue = new MetaqMessageQueue(new MessageQueue(messageQueue.getTopic(), messageQueue.getBrokerName(), messageQueue.getQueueId())); + if (isNotDataSplit(metaqMessageQueue.getQueueId())) { + continue; + } + String instanceName = queue2Instances.get(messageQueue); + List> splits = instanceOwnerQueues.get(instanceName); + if (splits == null) { + splits = new ArrayList<>(); + instanceOwnerQueues.put(instanceName, splits); + } + splits.add(metaqMessageQueue); + } + return instanceOwnerQueues; + + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + defaultMQAdminExt.shutdown(); + } + } + + protected boolean isNotDataSplit(String queueId) { + return queueId.toUpperCase().startsWith("%RETRY%"); + } + + /** + * 设置offset存储,包装原有的RemoteBrokerOffsetStore,在保存offset前发送系统消息 + * + * @param consumer + */ + protected void setOffsetStore(MetaPushConsumer consumer) { + DefaultMQPushConsumerImpl defaultMQPushConsumer = consumer.getMetaPushConsumerImpl().getDefaultMQPushConsumerImpl(); + MQClientInstance mQClientFactory = MQClientManager.getInstance().getAndCreateMQClientInstance(defaultMQPushConsumer.getDefaultMQPushConsumer(), null); + RemoteBrokerOffsetStore offsetStore = new RemoteBrokerOffsetStore(mQClientFactory, groupName); + consumer.setOffsetStore(new MetaqOffset(offsetStore, this));//每个一分钟运行一次 + } + + @Override protected boolean hasListenerSplitChanged() { + return true; + } + + @Override + public List> fetchAllSplits() { + try { + if (messageQueues == null || messageQueues.size() == 0) { + Set metaqQueueSet = consumer.fetchSubscribeMessageQueues(this.topic); + List> queueList = new ArrayList<>(); + for (MessageQueue queue : metaqQueueSet) { + MetaqMessageQueue metaqMessageQueue = new MetaqMessageQueue(queue); + if (isNotDataSplit(metaqMessageQueue.getQueueId())) { + continue; + } + + queueList.add(metaqMessageQueue); + + } + messageQueues = queueList; + } + return messageQueues; + } catch (MQClientException e) { + e.printStackTrace(); + throw new RuntimeException("get all splits error ", e); + } + } + + protected Map getMessageQueueAllocationResult(DefaultMQAdminExt defaultMQAdminExt, String groupName) { + HashMap results = new HashMap(); + + try { + ConsumerConnection consumerConnection = defaultMQAdminExt.examineConsumerConnectionInfo(groupName); + Iterator var5 = consumerConnection.getConnectionSet().iterator(); + + while (var5.hasNext()) { + Connection connection = (Connection) var5.next(); + String clientId = connection.getClientId(); + ConsumerRunningInfo consumerRunningInfo = defaultMQAdminExt.getConsumerRunningInfo(groupName, clientId, false); + Iterator var9 = consumerRunningInfo.getMqTable().keySet().iterator(); + + while (var9.hasNext()) { + org.apache.rocketmq.common.message.MessageQueue messageQueue = (org.apache.rocketmq.common.message.MessageQueue) var9.next(); + results.put(messageQueue, clientId.split("@")[1]); + } + } + } catch (Exception var11) { + ; + } + + return results; + } + + public void destroyConsumer() { + List oldConsumers = new ArrayList<>(); + if (consumer != null) { + oldConsumers.add(consumer); + } + + try { + for (MetaPushConsumer consumer : oldConsumers) { + consumer.shutdown(); + } + + } catch (Throwable t) { + if (LOG.isWarnEnabled()) { + LOG.warn(t.getMessage(), t); + } + } + + } + + @Override + public void destroySource() { + destroyConsumer(); + } + + public String getTags() { + return tags; + } + + public void setTags(String tags) { + this.tags = tags; + } + + public void setConsumerWhere(String offsetTime) { + if (StringUtil.isNotEmpty(offsetTime)) { + this.consumeFromWhere = ConsumeFromWhere.CONSUME_FROM_TIMESTAMP; + this.consumerOffset = offsetTime; + } + } + + public Long getPullIntervalMs() { + return pullIntervalMs; + } + + public void setPullIntervalMs(Long pullIntervalMs) { + this.pullIntervalMs = pullIntervalMs; + } +} diff --git a/rocketmq-streams-channel-metaq/src/test/java/org/apache/rocketmq/streams/AbstractChannelTest.java b/rocketmq-streams-channel-metaq/src/test/java/org/apache/rocketmq/streams/AbstractChannelTest.java new file mode 100644 index 00000000..59bf4931 --- /dev/null +++ b/rocketmq-streams-channel-metaq/src/test/java/org/apache/rocketmq/streams/AbstractChannelTest.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; + +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.context.Message; +import org.junit.Test; + +public abstract class AbstractChannelTest { + + @Test + public void testChannel() throws InterruptedException { + ISource channel = createSource(); + channel.setGroupName("CID_XXX_TEST"); + channel.setMaxThread(1); + channel.start((message, context) -> { + //System.out.println(message.getMessageBody().getString(IChannel.OFFSET)+"-"+message.getMessageBody() + // .getString(IChannel.QUEUE_ID)+"-"+message.getMessageBody().getString(IChannel.IS_BATCH)+"-"+Thread + // .currentThread().getId()); + System.out.println(message.getMessageBody()); + return message; + }); + while (true) { + Thread.sleep(1000L); + } + } + + protected abstract ISource createSource(); + + protected abstract ISink createSink(); + + @Test + public void testOutput() { + ISink channel = createSink(); + JSONObject message = new JSONObject(); + message.put("name", "chris"); + message.put("age", 18); + channel.batchAdd(new Message(message)); + channel.flush(); + } +} diff --git a/rocketmq-streams-channel-mqtt/pom.xml b/rocketmq-streams-channel-mqtt/pom.xml index abf10b72..4c498a73 100644 --- a/rocketmq-streams-channel-mqtt/pom.xml +++ b/rocketmq-streams-channel-mqtt/pom.xml @@ -1,22 +1,17 @@ - rocketmq-streams org.apache.rocketmq - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT 4.0.0 rocketmq-streams-channel-mqtt ROCKETMQ STREAMS :: channel-mqtt - - 8 - 8 - - org.apache.rocketmq @@ -30,6 +25,10 @@ org.eclipse.paho org.eclipse.paho.client.mqttv3 + + org.slf4j + slf4j-api + \ No newline at end of file diff --git a/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/MqttChannelBuilder.java b/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/MqttChannelBuilder.java index 4c20c7af..15d6a7d8 100644 --- a/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/MqttChannelBuilder.java +++ b/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/MqttChannelBuilder.java @@ -20,6 +20,7 @@ import java.util.Properties; 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.MemorySink; 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; @@ -29,7 +30,7 @@ import org.apache.rocketmq.streams.mqtt.source.PahoSource; @AutoService(IChannelBuilder.class) -@ServiceName(value = MqttChannelBuilder.TYPE, aliasName = "MqttSource") +@ServiceName(value = MqttChannelBuilder.TYPE, aliasName = "PahoSource") public class MqttChannelBuilder extends AbstractSupportShuffleChannelBuilder { public static final String TYPE = "mqtt"; @@ -50,11 +51,6 @@ public ISink createSink(String namespace, String name, Properties properties, Me @Override public ISink createBySource(ISource pipelineSource) { - PahoSource source = (PahoSource) pipelineSource; - if (source.getUsername() != null && source.getPassword() != null) { - return new PahoSink(source.getUrl(), source.getClientId(), source.getTopic(), source.getUsername(), source.getPassword()); - } else { - return new PahoSink(source.getUrl(), source.getClientId(), source.getTopic()); - } + return new MemorySink(); } } diff --git a/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/factory/MessageProcess.java b/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/factory/MessageProcess.java new file mode 100644 index 00000000..bb5d5ebf --- /dev/null +++ b/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/factory/MessageProcess.java @@ -0,0 +1,19 @@ +package org.apache.rocketmq.streams.mqtt.factory; + +import org.eclipse.paho.client.mqttv3.MqttMessage; + +/** + * @author fgm + * @date 2023/5/16 + * @description MQTT消息回调 + */ +public interface MessageProcess { + + /** + * 处理消息 + * + * @param topic + * @param message + */ + void process(String topic, MqttMessage message); +} diff --git a/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/factory/MqttClientFactory.java b/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/factory/MqttClientFactory.java new file mode 100644 index 00000000..acfdb0a7 --- /dev/null +++ b/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/factory/MqttClientFactory.java @@ -0,0 +1,211 @@ +package org.apache.rocketmq.streams.mqtt.factory; + +import com.google.common.collect.Maps; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantLock; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.commons.lang3.exception.ExceptionUtils; +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken; +import org.eclipse.paho.client.mqttv3.MqttCallbackExtended; +import org.eclipse.paho.client.mqttv3.MqttClient; +import org.eclipse.paho.client.mqttv3.MqttConnectOptions; +import org.eclipse.paho.client.mqttv3.MqttException; +import org.eclipse.paho.client.mqttv3.MqttMessage; +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @author fgm + * @date 2023/5/7 + * @description MQTT客户端工厂 + */ +public class MqttClientFactory { + private static final Logger LOGGER = LoggerFactory.getLogger(MqttClientFactory.class); + private static final ScheduledExecutorService CONNECT_EXECUTOR = Executors.newSingleThreadScheduledExecutor(r -> { + Thread t = new Thread(r, "MQTT-Connect-Task"); + t.setDaemon(true); + return t; + }); + private static final ExecutorService SUBSCRIBE_EXECUTOR = Executors.newSingleThreadExecutor(r -> { + Thread t = new Thread(r, "MQTT-Subscribe-Task"); + t.setDaemon(true); + return t; + }); + private static Map CLIENT_MAP = Maps.newConcurrentMap(); + private static ReentrantLock REENTRANT_LOCK = new ReentrantLock(); + + static { + CONNECT_EXECUTOR.scheduleWithFixedDelay(() -> heartbeat(), 0, 30, TimeUnit.SECONDS); + } + + /** + * 连接心跳 + */ + private static void heartbeat() { + try { + for (Map.Entry entry : CLIENT_MAP.entrySet()) { + connect(entry.getValue()); + } + } catch (Exception ex) { + LOGGER.error("MQTT-Heartbeat error,ex:{}", ExceptionUtils.getStackTrace(ex)); + } + } + + /** + * 连接 + * + * @param connection + */ + public static void connect(MqttConnection connection) { + try { + MqttClient mqttClient = connection.getMqttClient(); + MqttConnectOptions options = connection.getOptions(); + if (null == mqttClient) { + LOGGER.info("MQTT Connecting error,mqttClient is null,broker:{}", connection.getUrl()); + return; + } + if (null == options) { + LOGGER.info("MQTT Connecting error,options is null,broker:{}", connection.getUrl()); + return; + } + if (mqttClient.isConnected()) { + return; + } + LOGGER.info("MQTT Connecting,broker:{}", connection.getUrl()); + mqttClient.connect(options); + LOGGER.info("MQTT Connected,broker:{}", connection.getUrl()); + } catch (MqttException ex) { + LOGGER.error("MQTT Connecting error,clientKey:{},code:{},msg:{}", connection.getClientKey(), ex.getReasonCode(), ex.getMessage()); + } catch (Exception ex) { + LOGGER.error("MQTT Connecting error,clientKey:{},ex:{}", connection.getClientKey(), ExceptionUtils.getStackTrace(ex)); + } + } + + public synchronized static void removeClient(String clientKey) { + MqttConnection hasConnection = CLIENT_MAP.get(clientKey); + if (hasConnection != null) { + int count = hasConnection.getReferenceCount() - 1; + if (count <= 0) { + CLIENT_MAP.remove(clientKey); + MqttClient client = hasConnection.getMqttClient(); + try { + if (client != null && client.isConnected()) { + client.disconnect(); + client.close(); + } + } catch (MqttException e) { + throw new RuntimeException("PahoSink close error", e); + } + } else { + hasConnection.setReferenceCount(count); + } + } + } + + public synchronized static MqttClient getClient(MqttConnection connection) { + String clientKey = connection.getClientKey(); + MqttConnection connected = CLIENT_MAP.get(clientKey); + if (connected != null) { + connected.setReferenceCount(connected.getReferenceCount() + 1); + MqttClient mqttClient = connected.getMqttClient(); + return mqttClient; + } + try { + boolean success = REENTRANT_LOCK.tryLock(5000, TimeUnit.SECONDS); + if (!success) { + LOGGER.info("MQTT getClient timeout,clientId:{},broker:{}", connection.getClientId(), connection.getUrl()); + return null; + } else { + //已经存在 + connected = CLIENT_MAP.get(clientKey); + if (connected != null) { + connected.setReferenceCount(connected.getReferenceCount() + 1); + MqttClient mqttClient = connected.getMqttClient(); + return mqttClient; + } + } + String uniqueId = connection.getClientId() + "_" + RandomStringUtils.randomAlphanumeric(6); + MqttClient mqttClient = new MqttClient(connection.getUrl(), uniqueId, new MemoryPersistence()); + //最多等待1000ms + mqttClient.setTimeToWait(1000); + //设置回调 + mqttClient.setCallback(new MqttClientCallback(connection)); + connection.setMqttClient(mqttClient); + //触发连接 + if (connection.isAutoConnect()) { + connect(connection); + } + CLIENT_MAP.put(connection.getClientKey(), connection); + return mqttClient; + } catch (MqttException ex) { + LOGGER.info("MQTT getClient error,clientId:{},broker:{},code:{},msg:{}", connection.getClientId(), connection.getUrl(), ex.getReasonCode(), ex.getMessage()); + } catch (Exception ex) { + LOGGER.info("MQTT getClient error,clientId:{},broker:{},ex:{}", connection.getClientId(), connection.getUrl(), ExceptionUtils.getStackTrace(ex)); + } finally { + REENTRANT_LOCK.unlock(); + } + return null; + } + + /** + * MQTT 回调 + */ + public static class MqttClientCallback implements MqttCallbackExtended { + private MqttConnection clientInfo; + + public MqttClientCallback(MqttConnection clientInfo) { + this.clientInfo = clientInfo; + } + + /** + * 客户端连接成功后就需要尽快订阅需要的Topic。 + */ + @Override + public void connectComplete(boolean reconnect, String serverURI) { + String topic = clientInfo.getTopic(); + LOGGER.info("MQTT Subscribe topic:{},broker:{}", topic, serverURI); + SUBSCRIBE_EXECUTOR.submit(() -> { + try { + clientInfo.getMqttClient().subscribe(topic); + } catch (MqttException ex) { + LOGGER.error("MQTT Subscribe error,clientKey:{},topic:{},code:{},message:{}", clientInfo.getClientKey(), topic, ex.getReasonCode(), ex.getMessage()); + } + }); + } + + /** + * 连接丢失 + * + * @param throwable + */ + @Override + public void connectionLost(Throwable throwable) { + connect(clientInfo); + } + + @Override + public void messageArrived(String topic, MqttMessage message) { + MessageProcess messageProcess = clientInfo.getMessageProcess(); + if (messageProcess == null) { + return; + } + try { + messageProcess.process(topic, message); + } catch (Exception ex) { + LOGGER.error("MQTT process message error,topic:{},ex:{}", topic, ExceptionUtils.getStackTrace(ex)); + } + } + + @Override + public void deliveryComplete(IMqttDeliveryToken iMqttDeliveryToken) { + + } + + } + +} diff --git a/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/factory/MqttConnection.java b/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/factory/MqttConnection.java new file mode 100644 index 00000000..e5fee6bd --- /dev/null +++ b/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/factory/MqttConnection.java @@ -0,0 +1,122 @@ +package org.apache.rocketmq.streams.mqtt.factory; + +import org.eclipse.paho.client.mqttv3.MqttClient; +import org.eclipse.paho.client.mqttv3.MqttConnectOptions; + +/** + * @author fgm + * @date 2023/5/7 + * @description Mqtt连接信息 + */ +public class MqttConnection { + + protected int referenceCount = 0; + //输入属性 + private String url; + private String topic; + private String username; + private String password; + private String clientId; + private MqttConnectOptions options; + //是否自动连接 + private boolean autoConnect; + //消息处理 + private MessageProcess messageProcess; + //生成属性 + private String clientKey; + private MqttClient mqttClient; + + public MqttConnection(String url, String topic, String username, String password, String clientId, MqttConnectOptions options) { + this.url = url; + this.topic = topic; + this.username = username; + this.password = password; + this.clientId = clientId; + this.options = options; + this.clientKey = this.url + "#" + this.topic + "#" + this.username + "#" + this.password + "#" + this.clientId; + } + + public String getClientKey() { + return clientKey; + } + + public int getReferenceCount() { + return referenceCount; + } + + public void setReferenceCount(int referenceCount) { + this.referenceCount = referenceCount; + } + + public String getUrl() { + return url; + } + + public void setUrl(String url) { + this.url = url; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public String getUsername() { + return username; + } + + public void setUsername(String username) { + this.username = username; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } + + public String getClientId() { + return clientId; + } + + public void setClientId(String clientId) { + this.clientId = clientId; + } + + public MqttConnectOptions getOptions() { + return options; + } + + public void setOptions(MqttConnectOptions options) { + this.options = options; + } + + public MqttClient getMqttClient() { + return mqttClient; + } + + public void setMqttClient(MqttClient mqttClient) { + this.mqttClient = mqttClient; + } + + public boolean isAutoConnect() { + return autoConnect; + } + + public void setAutoConnect(boolean autoConnect) { + this.autoConnect = autoConnect; + } + + public MessageProcess getMessageProcess() { + return messageProcess; + } + + public void setMessageProcess(MessageProcess messageProcess) { + this.messageProcess = messageProcess; + } +} diff --git a/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/sink/PahoSink.java b/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/sink/PahoSink.java index c266a46b..3e982c34 100644 --- a/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/sink/PahoSink.java +++ b/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/sink/PahoSink.java @@ -20,22 +20,28 @@ import java.util.List; import org.apache.rocketmq.streams.common.channel.sink.AbstractSink; import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.mqtt.factory.MqttClientFactory; +import org.apache.rocketmq.streams.mqtt.factory.MqttConnection; import org.eclipse.paho.client.mqttv3.MqttClient; import org.eclipse.paho.client.mqttv3.MqttConnectOptions; import org.eclipse.paho.client.mqttv3.MqttException; import org.eclipse.paho.client.mqttv3.MqttMessage; -import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class PahoSink extends AbstractSink { + private static final Logger LOGGER = LoggerFactory.getLogger(PahoSink.class); + protected String clientKey; private String broker; private String clientId; private String topic; private int qos; private String username; private String password; - - private transient MqttClient client; + private Boolean automaticReconnect; + //不需要序列化的属性 + private transient volatile MqttClient client; public PahoSink() { } @@ -45,36 +51,51 @@ public PahoSink(String broker, String clientId, String topic) { } public PahoSink(String broker, String clientId, String topic, String username, String password) { - this(broker, clientId, topic, null, null, 2); + this(broker, clientId, topic, null, null, true, 2); + } + + public PahoSink(String broker, String clientId, String topic, String username, String password, + Boolean automaticReconnect) { + this(broker, clientId, topic, null, null, true, 2); } - public PahoSink(String broker, String clientId, String topic, String username, String password, int qos) { + public PahoSink(String broker, String clientId, String topic, String username, String password, + Boolean automaticReconnect, int qos) { this.broker = broker; this.clientId = clientId; this.topic = topic; this.username = username; this.password = password; + this.automaticReconnect = automaticReconnect; this.qos = qos; } + @Override + public boolean initConfigurable() { + super.initConfigurable(); + //建立连接 + MqttConnection connection = new MqttConnection(this.broker, this.topic, this.username, this.password, this.clientId, getOptions()); + connection.setAutoConnect(true); + this.clientKey = connection.getClientKey(); + MqttClient client = MqttClientFactory.getClient(connection); + if (null == client) { + return false; + } + this.client = client; + return true; + } + @Override protected boolean batchInsert(List messages) { try { if (this.client == null) { - this.client = new MqttClient(broker, clientId, new MemoryPersistence()); + LOGGER.error("PahoSink batchInsert discard,MQTT Client IS Empty!"); + return false; } if (!this.client.isConnected()) { - MqttConnectOptions connOpts = new MqttConnectOptions(); - connOpts.setCleanSession(true); - if (this.username != null && this.password != null) { - connOpts.setUserName(this.username); - connOpts.setPassword(this.password.toCharArray()); - } - System.out.println("Connecting to broker: " + broker); - this.client.connect(connOpts); - System.out.println("Connected"); + LOGGER.error("PahoSink batchInsert discard,MQTT Client Not Connected!"); + return false; } - for (IMessage msg : messages) { String messageString = ""; if (msg.isJsonMessage()) { @@ -88,27 +109,42 @@ protected boolean batchInsert(List messages) { } return true; } catch (MqttException e) { - System.err.println("reason " + e.getReasonCode()); - System.err.println("msg " + e.getMessage()); - System.err.println("loc " + e.getLocalizedMessage()); - System.err.println("cause " + e.getCause()); - System.err.println("exception " + e); - e.printStackTrace(); + LOGGER.error("PahoSink batchInsert error,reason:{},msg:{}", e.getReasonCode(), e.getMessage()); } return false; } + private MqttConnectOptions getOptions() { + MqttConnectOptions connOpts = new MqttConnectOptions(); + connOpts.setCleanSession(true); + if (this.username != null && this.password != null) { + connOpts.setUserName(this.username); + connOpts.setPassword(this.password.toCharArray()); + } + //默认自动重连 + if (null == this.automaticReconnect) { + connOpts.setAutomaticReconnect(true); + } else { + connOpts.setAutomaticReconnect(this.automaticReconnect); + } + return connOpts; + } + @Override public void destroy() { super.destroy(); - try { - if (this.client != null) { - this.client.disconnect(); - this.client.close(); - } - } catch (MqttException e) { - e.printStackTrace(); - } + MqttClientFactory.removeClient(clientKey); + //sink 资源共享,客户端不能关闭 +// try { +// if (this.client != null && this.client.isConnected()) { +// LOGGER.error("PahoSink destroy client:{}",this.client); +// this.client.disconnect(); +// this.client.close(); +// } +// } catch (Exception e) { +// throw new RuntimeException("Paho close error", e); +// } + } public String getBroker() { @@ -158,4 +194,4 @@ public String getPassword() { public void setPassword(String password) { this.password = password; } -} +} \ No newline at end of file diff --git a/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/source/PahoSource.java b/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/source/PahoSource.java index 11490ec3..50743f8f 100644 --- a/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/source/PahoSource.java +++ b/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/source/PahoSource.java @@ -16,25 +16,31 @@ */ package org.apache.rocketmq.streams.mqtt.source; -import com.alibaba.fastjson.JSONObject; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.concurrent.atomic.AtomicLong; +import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.rocketmq.streams.common.channel.source.AbstractSource; +import org.apache.rocketmq.streams.common.channel.split.CommonSplit; +import org.apache.rocketmq.streams.common.channel.split.ISplit; import org.apache.rocketmq.streams.common.utils.RuntimeUtil; -import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken; -import org.eclipse.paho.client.mqttv3.MqttCallback; +import org.apache.rocketmq.streams.mqtt.factory.MessageProcess; +import org.apache.rocketmq.streams.mqtt.factory.MqttClientFactory; +import org.apache.rocketmq.streams.mqtt.factory.MqttConnection; import org.eclipse.paho.client.mqttv3.MqttClient; import org.eclipse.paho.client.mqttv3.MqttConnectOptions; -import org.eclipse.paho.client.mqttv3.MqttException; import org.eclipse.paho.client.mqttv3.MqttMessage; -import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class PahoSource extends AbstractSource { private static final Logger LOGGER = LoggerFactory.getLogger(PahoSource.class); - + protected transient String clientKey; + protected transient AtomicLong offsetGenerator; private String url; private String clientId; private String topic; @@ -44,6 +50,9 @@ public class PahoSource extends AbstractSource { private Integer connectionTimeout; private Integer aliveInterval; private Boolean automaticReconnect; + private transient MqttClient client; + // 记录上次持久化 check point 的时间 + private transient long mLastCheckTime = 0; public PahoSource() { } @@ -72,134 +81,59 @@ public PahoSource(String url, String clientId, String topic, String username, St this.automaticReconnect = automaticReconnect; } - private transient MqttClient client; - protected transient AtomicLong offsetGenerator; - - @Override protected boolean startSource() { + @Override + protected boolean startSource() { try { - this.client = new MqttClient(url, clientId, new MemoryPersistence()); this.offsetGenerator = new AtomicLong(System.currentTimeMillis()); - this.client.setCallback(new MqttCallback() { - - @Override public void connectionLost(Throwable throwable) { - LOGGER.info("Reconnecting to broker: " + url); - while (true) { - MqttConnectOptions connOpts = new MqttConnectOptions(); - if (username != null && password != null) { - connOpts.setUserName(username); - connOpts.setPassword(password.toCharArray()); - } - if (cleanSession == null) { - connOpts.setCleanSession(true); - } else { - connOpts.setCleanSession(cleanSession); - } - - if (connectionTimeout == null) { - connOpts.setConnectionTimeout(10); - } else { - connOpts.setConnectionTimeout(connectionTimeout); - } - if (aliveInterval == null) { - connOpts.setKeepAliveInterval(60); - } else { - connOpts.setKeepAliveInterval(aliveInterval); - } - if (automaticReconnect == null) { - connOpts.setAutomaticReconnect(true); - } else { - connOpts.setAutomaticReconnect(automaticReconnect); - } - - try { - if (!client.isConnected()) { - client.connect(connOpts); - LOGGER.info("Reconnecting success"); - } - client.subscribe(topic); - break; - } catch (MqttException e) { - try { - LOGGER.error("Reconnecting err: " + e.getMessage()); - e.printStackTrace(); - Thread.sleep(10000); - } catch (InterruptedException ex) { - ex.printStackTrace(); - } - } - } - } - - @Override public void messageArrived(String s, MqttMessage mqttMessage) throws Exception { - JSONObject msg = create(new String(mqttMessage.getPayload(), StandardCharsets.UTF_8)); - msg.put("__topic", s); - doReceiveMessage(msg, false, RuntimeUtil.getDipperInstanceId(), offsetGenerator.incrementAndGet() + ""); - } - - @Override public void deliveryComplete(IMqttDeliveryToken token) { - LOGGER.info("deliveryComplete---------" + token.isComplete()); - } - }); - - MqttConnectOptions connOpts = new MqttConnectOptions(); - if (username != null && password != null) { - connOpts.setUserName(username); - connOpts.setPassword(password.toCharArray()); + MqttConnection connection = new MqttConnection(this.url, this.topic, this.username, this.password, this.clientId, getOptions()); + connection.setAutoConnect(true); + connection.setMessageProcess(new SourceMessageProcess()); + clientKey = connection.getClientKey(); + MqttClient client = MqttClientFactory.getClient(connection); + if (null == client) { + LOGGER.error("PahoSource startSource error,MQTT Client IS Empty!"); + return false; } - if (this.cleanSession == null) { - connOpts.setCleanSession(true); - } else { - connOpts.setCleanSession(this.cleanSession); - } - - if (this.connectionTimeout == null) { - connOpts.setConnectionTimeout(10); - } else { - connOpts.setConnectionTimeout(this.connectionTimeout); - } - if (this.aliveInterval == null) { - connOpts.setKeepAliveInterval(60); - } else { - connOpts.setKeepAliveInterval(this.aliveInterval); - } - if (this.automaticReconnect == null) { - connOpts.setAutomaticReconnect(true); - } else { - connOpts.setAutomaticReconnect(this.automaticReconnect); - } - - LOGGER.info("Connecting to broker: " + url); - if (!this.client.isConnected()) { - this.client.connect(connOpts); - LOGGER.info("Connected"); - } - this.client.subscribe(topic); + this.client = client; return true; - } catch (MqttException e) { - e.printStackTrace(); + } catch (Exception ex) { + LOGGER.error("PahoSource startSource error,broker:{},ex:{}", url, ExceptionUtils.getStackTrace(ex)); } return false; } - @Override public void destroy() { - super.destroy(); - try { - if (this.client != null && this.client.isConnected()) { - this.client.disconnect(); - this.client.close(); - } - super.destroy(); - } catch (MqttException e) { - e.printStackTrace(); + public MqttConnectOptions getOptions() { + MqttConnectOptions connOpts = new MqttConnectOptions(); + if (username != null && password != null) { + connOpts.setUserName(username); + connOpts.setPassword(password.toCharArray()); } + if (this.cleanSession == null) { + connOpts.setCleanSession(true); + } else { + connOpts.setCleanSession(this.cleanSession); + } + if (this.connectionTimeout == null) { + connOpts.setConnectionTimeout(10); + } else { + connOpts.setConnectionTimeout(this.connectionTimeout); + } + if (this.aliveInterval == null) { + connOpts.setKeepAliveInterval(60); + } else { + connOpts.setKeepAliveInterval(this.aliveInterval); + } + if (this.automaticReconnect == null) { + connOpts.setAutomaticReconnect(true); + } else { + connOpts.setAutomaticReconnect(this.automaticReconnect); + } + return connOpts; } - @Override public boolean supportRemoveSplitFind() { - return false; - } - - @Override protected boolean isNotDataSplit(String queueId) { - return false; + @Override + public void destroySource() { + MqttClientFactory.removeClient(clientKey); } public String getUrl() { @@ -218,14 +152,24 @@ public void setClientId(String clientId) { this.clientId = clientId; } - @Override public String getTopic() { + @Override + public String getTopic() { return topic; } - @Override public void setTopic(String topic) { + @Override + public void setTopic(String topic) { this.topic = topic; } + @Override + public List> fetchAllSplits() { + ISplit split = new CommonSplit("1"); + List> splits = new ArrayList<>(); + splits.add(split); + return splits; + } + public String getUsername() { return username; } @@ -274,4 +218,27 @@ public void setAutomaticReconnect(Boolean automaticReconnect) { this.automaticReconnect = automaticReconnect; } -} + /** + * 消息回调 + */ + class SourceMessageProcess implements MessageProcess { + @Override + public void process(String topic, MqttMessage message) { + String msg = new String(message.getPayload(), StandardCharsets.UTF_8); + Map additionValues = new HashMap<>(); + additionValues.put("__topic", topic); + String queueId = offsetGenerator.incrementAndGet() + ""; + doReceiveMessage(msg, false, RuntimeUtil.getDipperInstanceId(), queueId, additionValues); + + long curTime = System.currentTimeMillis(); + //每200ms调用一次 + // 每隔 60 秒,写一次 check point 到服务端,如果 60 秒内,worker crash, + // 新启动的 worker 会从上一个 checkpoint 其消费数据,有可能有重复数据 + if (curTime - mLastCheckTime > getCheckpointTime()) { + mLastCheckTime = curTime; + sendCheckpoint(String.valueOf(queueId)); + } + } + } + +} \ No newline at end of file diff --git a/rocketmq-streams-channel-openapi/.gitignore b/rocketmq-streams-channel-openapi/.gitignore new file mode 100644 index 00000000..b83d2226 --- /dev/null +++ b/rocketmq-streams-channel-openapi/.gitignore @@ -0,0 +1 @@ +/target/ diff --git a/rocketmq-streams-channel-openapi/pom.xml b/rocketmq-streams-channel-openapi/pom.xml new file mode 100755 index 00000000..b83780c8 --- /dev/null +++ b/rocketmq-streams-channel-openapi/pom.xml @@ -0,0 +1,49 @@ + + + + 4.0.0 + + rocketmq-streams + org.apache.rocketmq + 2.0.1-SNAPSHOT + + rocketmq-streams-channel-openapi + ROCKETMQ STREAMS :: channel-openapi + jar + + + com.aliyun + alibabacloud-sas20181203 + 1.1.29 + + + com.github.tomakehurst + wiremock-standalone + + + + + org.apache.rocketmq + rocketmq-streams-connectors + + + org.apache.rocketmq + rocketmq-streams-sts + + + org.apache.rocketmq + rocketmq-streams-dim + + + com.aliyun + aliyun-java-sdk-core + + + org.apache.httpcomponents + httpclient + + + + diff --git a/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/OpenAPIChannelBuilder.java b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/OpenAPIChannelBuilder.java new file mode 100644 index 00000000..083a3476 --- /dev/null +++ b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/OpenAPIChannelBuilder.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.openapi; + +import com.alibaba.fastjson.JSONObject; +import com.google.auto.service.AutoService; +import java.util.Properties; +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.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 = OpenAPIChannelBuilder.TYPE, aliasName = "OpenAPISource") +public class OpenAPIChannelBuilder extends AbstractSupportShuffleChannelBuilder { + public static final String TYPE = "openapi"; + + @Override public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) { + return (OpenAPISource) ConfigurableUtil.create(OpenAPISource.class.getName(), namespace, name, createFormatProperty(properties), null); + } + + @Override public String getType() { + return TYPE; + } + + @Override public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { + throw new RuntimeException("can not support this method"); + } + + 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.get(key)); + } + + IChannelBuilder.formatPropertiesName(formatProperties, properties, "regionId", "region"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "accessKeyId", "ak"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "accessSecret", "sk"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "productName", "product"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "defaultParameterStr", "query"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "actionVersion", "version"); + + return formatProperties; + } + + @Override public ISink createBySource(ISource pipelineSource) { + return null; + } +} diff --git a/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/OpenAPIDim.java b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/OpenAPIDim.java new file mode 100644 index 00000000..7e800e8e --- /dev/null +++ b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/OpenAPIDim.java @@ -0,0 +1,302 @@ +/* + * 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.openapi; + +import com.alibaba.fastjson.JSONArray; +import com.alibaba.fastjson.JSONObject; +import com.aliyuncs.profile.DefaultProfile; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; +import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; +import org.apache.rocketmq.streams.common.interfaces.IDim; +import org.apache.rocketmq.streams.common.utils.CollectionUtil; +import org.apache.rocketmq.streams.common.utils.SQLUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.openapi.executor.OpenAPIExecutor; +import org.apache.rocketmq.streams.sts.StsIdentity; +import org.apache.rocketmq.streams.sts.StsService; + +public class OpenAPIDim extends BasedConfigurable implements IDim { + + protected String regionId; + @ENVDependence + protected String accessKeyId; + @ENVDependence + protected String accessSecret; + protected String domain; + protected String productName; + protected String action; + protected String actionVersion; + protected String defaultParameterStr;//默认参数,格式是key;value,key:value + protected transient Map defaultParameters; + + protected String dataArrayFieldName;//返回的数据列表的字段名 + protected transient OpenAPIExecutor openAPIExecutor; + @ENVDependence private boolean isSts; + private String aliuid; + private String stsRoleArn; + private String stsAssumeRoleFor; + private String stsSessionPrefix = "openapi"; + private int stsExpireSeconds = 86400; + private String ramEndpoint = "sts-inner.aliyuncs.com"; + private transient StsService stsService; + + public OpenAPIDim() { + setType(TYPE); + } + + @Override public void startLoadDimData() { + if (StringUtil.isNotEmpty(this.defaultParameterStr)) { + defaultParameters = new HashMap<>(); + String[] values = this.defaultParameterStr.split(","); + for (String value : values) { + String[] kv = value.split(":"); + defaultParameters.put(kv[0], kv[1]); + } + } + if (StringUtil.isEmpty(stsRoleArn)) { + stsRoleArn = "acs:ram::" + aliuid + ":role/"; + } + + stsAssumeRoleFor = aliuid; + boolean b = initSts(); + this.openAPIExecutor = new OpenAPIExecutor(regionId, accessKeyId, accessSecret, domain, productName, action, actionVersion); + } + + @Override public List> matchExpression(String expressionStr, JSONObject msg, boolean needAll, + String script) { + Map parameters = new HashMap<>(); + if (defaultParameters != null) { + for (String key : defaultParameters.keySet()) { + Object value = SQLUtil.parseIbatisSQL(msg, defaultParameters.get(key)); + parameters.put(key, value); + } + } + DefaultProfile profile = null; + if (isSts) { + try { + StsIdentity stsIdentity = stsService.getStsIdentity(); + String localAccessId = stsIdentity.getAccessKeyId(); + String localAccessKey = stsIdentity.getAccessKeySecret(); + String localToken = stsIdentity.getSecurityToken(); + profile = DefaultProfile.getProfile(regionId, localAccessId, localAccessKey, localToken); + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException("sts execute error", e); + } + + } + JSONObject jsonObject = openAPIExecutor.invokeBySDK(parameters, profile); + if (StringUtil.isEmpty(dataArrayFieldName)) { + List> list = new ArrayList<>(); + list.add(jsonObject); + return list; + } else { + JSONArray data = jsonObject.getJSONArray(dataArrayFieldName); + if (CollectionUtil.isEmpty(data)) { + return null; + } + List> msgs = new ArrayList<>(); + jsonObject.remove(dataArrayFieldName); + + for (int i = 0; i < data.size(); i++) { + JSONObject message = new JSONObject(); + message.putAll(jsonObject); + message.putAll(data.getJSONObject(i)); + msgs.add(message); + } + return msgs; + } + } + + @Override public Map matchExpression(String expressionStr, JSONObject msg) { + List> rows = matchExpression(expressionStr, msg, true, null); + if (rows != null && rows.size() > 0) { + return rows.get(0); + } + return null; + } + + @Override + public List> matchExpression(String msgFieldName, String dimFieldName, JSONObject msg) { + return matchExpression("(" + msgFieldName + "," + dimFieldName + ")", msg, true, null); + } + + @Override public List getIndexs() { + return new ArrayList<>(); + } + + @Override public String addIndex(String... indexs) { + return null; + } + + private boolean initSts() { + if (isSts) { + if (StringUtil.isEmpty(stsRoleArn) || StringUtil.isEmpty(stsAssumeRoleFor)) { + return false; + } + stsService = new StsService(); + stsService.setAccessId(accessKeyId); + stsService.setAccessKey(accessSecret); + stsService.setRamEndPoint(ramEndpoint); + stsService.setStsExpireSeconds(stsExpireSeconds); + stsService.setStsSessionPrefix(stsSessionPrefix); + stsService.setRoleArn(stsRoleArn); + stsService.setStsAssumeRoleFor(stsAssumeRoleFor); + } + return true; + } + + public String getRegionId() { + return regionId; + } + + public void setRegionId(String regionId) { + this.regionId = regionId; + } + + public String getAccessKeyId() { + return accessKeyId; + } + + public void setAccessKeyId(String accessKeyId) { + this.accessKeyId = accessKeyId; + } + + public String getAccessSecret() { + return accessSecret; + } + + public void setAccessSecret(String accessSecret) { + this.accessSecret = accessSecret; + } + + public String getDomain() { + return domain; + } + + public void setDomain(String domain) { + this.domain = domain; + } + + public String getProductName() { + return productName; + } + + public void setProductName(String productName) { + this.productName = productName; + } + + public String getAction() { + return action; + } + + public void setAction(String action) { + this.action = action; + } + + public String getActionVersion() { + return actionVersion; + } + + public void setActionVersion(String actionVersion) { + this.actionVersion = actionVersion; + } + + public String getDefaultParameterStr() { + return defaultParameterStr; + } + + public void setDefaultParameterStr(String defaultParameterStr) { + this.defaultParameterStr = defaultParameterStr; + } + + public Map getDefaultParameters() { + return defaultParameters; + } + + public void setDefaultParameters(Map defaultParameters) { + this.defaultParameters = defaultParameters; + } + + public String getDataArrayFieldName() { + return dataArrayFieldName; + } + + public void setDataArrayFieldName(String dataArrayFieldName) { + this.dataArrayFieldName = dataArrayFieldName; + } + + public boolean isSts() { + return isSts; + } + + public void setSts(boolean sts) { + isSts = sts; + } + + public String getAliuid() { + return aliuid; + } + + public void setAliuid(String aliuid) { + this.aliuid = aliuid; + } + + public String getStsRoleArn() { + return stsRoleArn; + } + + public void setStsRoleArn(String stsRoleArn) { + this.stsRoleArn = stsRoleArn; + } + + public String getStsAssumeRoleFor() { + return stsAssumeRoleFor; + } + + public void setStsAssumeRoleFor(String stsAssumeRoleFor) { + this.stsAssumeRoleFor = stsAssumeRoleFor; + } + + public String getStsSessionPrefix() { + return stsSessionPrefix; + } + + public void setStsSessionPrefix(String stsSessionPrefix) { + this.stsSessionPrefix = stsSessionPrefix; + } + + public int getStsExpireSeconds() { + return stsExpireSeconds; + } + + public void setStsExpireSeconds(int stsExpireSeconds) { + this.stsExpireSeconds = stsExpireSeconds; + } + + public String getRamEndpoint() { + return ramEndpoint; + } + + public void setRamEndpoint(String ramEndpoint) { + this.ramEndpoint = ramEndpoint; + } +} diff --git a/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/OpenAPISQLParser.java b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/OpenAPISQLParser.java new file mode 100644 index 00000000..20c8144f --- /dev/null +++ b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/OpenAPISQLParser.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.openapi; + +import com.alibaba.fastjson.JSONObject; +import com.google.auto.service.AutoService; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.Properties; +import org.apache.rocketmq.streams.common.interfaces.IDim; +import org.apache.rocketmq.streams.common.metadata.MetaData; +import org.apache.rocketmq.streams.common.model.ServiceName; +import org.apache.rocketmq.streams.common.utils.ConfigurableUtil; +import org.apache.rocketmq.streams.dim.builder.IDimSQLParser; + +@AutoService(IDimSQLParser.class) +@ServiceName(value = OpenAPISQLParser.TYPE, aliasName = "openapi") +public class OpenAPISQLParser implements IDimSQLParser { + public static final String TYPE = "openapi"; + + protected static Map formatNames = new HashMap<>(); + + static { + formatNames.put("region", "regionId"); + formatNames.put("ak", "accessKeyId"); + formatNames.put("sk", "accessSecret"); + formatNames.put("product", "productName"); + formatNames.put("query", "defaultParameterStr"); + formatNames.put("version", "actionVersion"); + } + + @Override public IDim parseDim(String namespace, Properties properties, MetaData metaData) { + JSONObject paras = new JSONObject(); + Iterator it = properties.keySet().iterator(); + while (it.hasNext()) { + String key = (String) it.next(); + Object value = properties.get(key); + if (formatNames.containsKey(key)) { + key = formatNames.get(key); + } + if ("type".equals(key)) { + continue; + } + paras.put(key, value); + } + OpenAPIDim dim = (OpenAPIDim) ConfigurableUtil.create(OpenAPIDim.class.getName(), namespace, null, paras, null); + return dim; + } +} diff --git a/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/OpenAPISource.java b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/OpenAPISource.java new file mode 100644 index 00000000..8a7802b6 --- /dev/null +++ b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/OpenAPISource.java @@ -0,0 +1,500 @@ +/* + * 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.openapi; + +import com.alibaba.fastjson.JSONArray; +import com.alibaba.fastjson.JSONObject; +import com.aliyuncs.profile.DefaultProfile; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +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.MessageOffset; +import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.common.utils.IdUtil; +import org.apache.rocketmq.streams.common.utils.ReflectUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.connectors.model.PullMessage; +import org.apache.rocketmq.streams.connectors.reader.AbstractSplitReader; +import org.apache.rocketmq.streams.connectors.reader.ISplitReader; +import org.apache.rocketmq.streams.connectors.source.AbstractPullSource; +import org.apache.rocketmq.streams.openapi.executor.OpenAPIExecutor; +import org.apache.rocketmq.streams.openapi.splits.OpenAPISplit; +import org.apache.rocketmq.streams.sts.StsIdentity; +import org.apache.rocketmq.streams.sts.StsService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class OpenAPISource extends AbstractPullSource { + + private static final Logger LOGGER = LoggerFactory.getLogger(OpenAPISource.class); + + protected String regionId; + @ENVDependence + protected String accessKeyId; + @ENVDependence + protected String accessSecret; + protected String domain; + protected String productName; + protected String action; + protected String actionVersion; + protected String defaultParameterStr;//默认参数,格式是key;value,key:value + + protected transient Map defaultParameters; + + protected String startTimeFieldName = "StartTime";//开始时间的字段名 + protected String endTimeFieldName = "EndTime";//结束时间的字段名 + protected String dataArrayFieldName;//返回的数据列表的字段名 + protected String pageNumberFieldName = "PageNumber";//取哪页数据 + protected String pageSizeFieldName = "PageSize";//每页数据最大值 + protected String totalCountField = "TotalCount";//总条数的字段名,支持嵌套 + + protected int maxPageSize = 10;//最大一页数据数量 + protected boolean isTimeStamp = true; + protected boolean isSecondTimeStamp = false; + + protected long dateAddOfHistoryData = 60;//如果有历史数据,则历史数据每次查询一个小时的数据 + + protected String initFirstTime = DateUtil.getCurrentTimeString(); + + protected Long pollingMinute = 10L; + protected int splitCount = 1;//在一个周期内的并发性 + + @ENVDependence private boolean isSts; + private String aliUid; + private String stsRoleArn; + private String stsAssumeRoleFor; + private String stsSessionPrefix = "openapi"; + + private int stsExpireSeconds = 86400; + + private String ramEndpoint = "sts-inner.aliyuncs.com"; + + private transient StsService stsService; + + @Override protected boolean initConfigurable() { + if (StringUtil.isNotEmpty(this.defaultParameterStr)) { + defaultParameters = new HashMap<>(); + String[] values = this.defaultParameterStr.split(","); + for (String value : values) { + String[] kv = value.split(":"); + defaultParameters.put(kv[0], kv[1]); + } + } + if (StringUtil.isEmpty(stsRoleArn)) { + stsRoleArn = "acs:ram::" + aliUid + ":role/"; + } + if (StringUtil.isEmpty(aliUid) && StringUtil.isNotEmpty(stsRoleArn)) { + String tmp = stsRoleArn.replace("acs:ram::", ""); + int index = tmp.lastIndexOf(":"); + aliUid = tmp.substring(0, index); + } + stsAssumeRoleFor = aliUid; + boolean b = initSts(); + return b && super.initConfigurable(); + } + + @Override public List> fetchAllSplits() { + List> splits = new ArrayList<>(); + long baseDataAdd = pollingMinute / splitCount; + long remainder = pollingMinute % splitCount; + Date date = DateUtil.getWindowBeginTime(DateUtil.parseTime(initFirstTime).getTime(), pollingMinute); + for (int i = 0; i < splitCount; i++) { + long dataAdd = baseDataAdd + (remainder > 0 ? 1 : 0); + remainder--; + Date endDate = DateUtil.addMinute(date, (int) dataAdd); + OpenAPISplit openAPISplit = new OpenAPISplit(String.valueOf(i), date.getTime(), dataAdd, pollingMinute); + splits.add(openAPISplit); + date = endDate; + } + return splits; + } + + @Override protected ISplitReader createSplitReader(ISplit split) { + return new AbstractSplitReader(split) { + + private long startTime; + private long dateAdd; + private long pollingMinute; + private int pageNum = 1; + private transient OpenAPIExecutor openAPIExecutor; + + @Override public void open() { + OpenAPISplit openAPISplit = (OpenAPISplit) split; + this.startTime = openAPISplit.getInitStartTime(); + this.dateAdd = openAPISplit.getDateAdd(); + this.pollingMinute = openAPISplit.getPollingMinute(); + this.openAPIExecutor = new OpenAPIExecutor(regionId, accessKeyId, accessSecret, domain, productName, action, actionVersion); + } + + @Override public boolean next() { + return startTime + pollingMinute * 60 * 1000 < System.currentTimeMillis(); + } + + @Override public Iterator> getMessage() { + Map parameters = new HashMap<>(); + if (StringUtil.isNotEmpty(startTimeFieldName)) { + parameters.put(startTimeFieldName, createTime(startTime)); + } + if (StringUtil.isNotEmpty(endTimeFieldName)) { + parameters.put(endTimeFieldName, createTime(startTime + dateAdd * 60 * 1000)); + } + if (StringUtil.isNotEmpty(pageNumberFieldName)) { + parameters.put(pageNumberFieldName, pageNum); + } + if (StringUtil.isNotEmpty(pageSizeFieldName)) { + parameters.put(pageSizeFieldName, maxPageSize); + } + + if (defaultParameters != null) { + parameters.putAll(defaultParameters); + } + DefaultProfile profile = null; + if (isSts) { + try { + StsIdentity stsIdentity = stsService.getStsIdentity(); + String localAccessId = stsIdentity.getAccessKeyId(); + String localAccessKey = stsIdentity.getAccessKeySecret(); + String localToken = stsIdentity.getSecurityToken(); + profile = DefaultProfile.getProfile(regionId, localAccessId, localAccessKey, localToken); + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException("sts execute error", e); + } + + } + Object queryStartTime = parameters.get(startTimeFieldName); + Object queryEndTime = parameters.get(endTimeFieldName); + JSONObject msg = openAPIExecutor.invokeBySDK(parameters, profile); + JSONArray data = msg.getJSONArray(dataArrayFieldName); + if (data == null || data.size() == 0) { + startTime = startTime + pollingMinute * 60 * 1000; + pageNum = 1; + return null; + } else if (StringUtil.isNotEmpty(totalCountField)) { + Integer totalCount = ReflectUtil.getBeanFieldOrJsonValue(msg, totalCountField); + if (totalCount == null) { + throw new RuntimeException("expect get totalCount from result msg by " + totalCountField + ", real is not find"); + } + long currentCount = (long) (pageNum) * maxPageSize; + if (currentCount >= totalCount) { + startTime = startTime + pollingMinute * 60 * 1000; + pageNum = 1; + } else { + pageNum++; + } + } else if (data.size() < maxPageSize) { + startTime = startTime + pollingMinute * 60 * 1000; + pageNum = 1; + } else { + pageNum++; + + } + List> messages = new ArrayList<>(); + msg.remove(dataArrayFieldName); + LOGGER.info("[{}][{}] Execute_Openapi_Query_Result_Size({})_Query_Time_Gap({}-{})_Query_Page({})", IdUtil.instanceId(), getName(), data.size(), DateUtil.format(new Date(startTime)), DateUtil.format(new Date(startTime + dateAdd * 60 * 1000)), pageNum); + for (int i = 0; i < data.size(); i++) { + JSONObject message = new JSONObject(); + message.putAll(msg); + message.putAll(data.getJSONObject(i)); + message.put(startTimeFieldName, queryStartTime); + message.put(endTimeFieldName, queryEndTime); + + PullMessage pullMessage = new PullMessage<>(); + pullMessage.setMessage(message); + MessageOffset messageOffset = new MessageOffset(startTime + ";" + (10000 + pageNum)); + messageOffset.addLayerOffset(i); + pullMessage.setMessageOffset(messageOffset); + messages.add(pullMessage); + } + return messages.iterator(); + } + + @Override public long getDelay() { + return System.currentTimeMillis() - startTime; + } + + @Override public long getFetchedDelay() { + return 0; + } + + @Override public void seek(String cursor) { + if (cursor == null) { + this.cursor = this.startTime + ";" + (pageNum + 10000); + return; + } + this.cursor = cursor; + String[] values = cursor.split(";"); + this.startTime = Long.parseLong(values[0]); + this.pageNum = Integer.parseInt(values[1]) - 10000; + } + + @Override public String getCursor() { + return this.startTime + ";" + (pageNum + 10000); + } + }; + } + + protected Object createTime(long time) { + if (isTimeStamp || isSecondTimeStamp) { + if (isSecondTimeStamp) { + return time / 1000; + } + return time; + } else { + return DateUtil.longToString(time); + } + + } + + private boolean initSts() { + if (isSts) { + if (StringUtil.isEmpty(stsRoleArn) || StringUtil.isEmpty(stsAssumeRoleFor)) { + return false; + } + stsService = new StsService(); + stsService.setAccessId(accessKeyId); + stsService.setAccessKey(accessSecret); + stsService.setRamEndPoint(ramEndpoint); + stsService.setStsExpireSeconds(stsExpireSeconds); + stsService.setStsSessionPrefix(stsSessionPrefix); + stsService.setRoleArn(stsRoleArn); + stsService.setStsAssumeRoleFor(stsAssumeRoleFor); + } + return true; + } + + public String getDefaultParameterStr() { + return defaultParameterStr; + } + + public void setDefaultParameterStr(String defaultParameterStr) { + this.defaultParameterStr = defaultParameterStr; + } + + public String getRegionId() { + return regionId; + } + + public void setRegionId(String regionId) { + this.regionId = regionId; + } + + public String getAccessKeyId() { + return accessKeyId; + } + + public void setAccessKeyId(String accessKeyId) { + this.accessKeyId = accessKeyId; + } + + public String getAccessSecret() { + return accessSecret; + } + + public void setAccessSecret(String accessSecret) { + this.accessSecret = accessSecret; + } + + public String getDomain() { + return domain; + } + + public void setDomain(String domain) { + this.domain = domain; + } + + public String getProductName() { + return productName; + } + + public void setProductName(String productName) { + this.productName = productName; + } + + public String getAction() { + return action; + } + + public void setAction(String action) { + this.action = action; + } + + public String getStartTimeFieldName() { + return startTimeFieldName; + } + + public void setStartTimeFieldName(String startTimeFieldName) { + this.startTimeFieldName = startTimeFieldName; + } + + public String getEndTimeFieldName() { + return endTimeFieldName; + } + + public void setEndTimeFieldName(String endTimeFieldName) { + this.endTimeFieldName = endTimeFieldName; + } + + public String getDataArrayFieldName() { + return dataArrayFieldName; + } + + public void setDataArrayFieldName(String dataArrayFieldName) { + this.dataArrayFieldName = dataArrayFieldName; + } + + public String getPageNumberFieldName() { + return pageNumberFieldName; + } + + public void setPageNumberFieldName(String pageNumberFieldName) { + this.pageNumberFieldName = pageNumberFieldName; + } + + public String getPageSizeFieldName() { + return pageSizeFieldName; + } + + public void setPageSizeFieldName(String pageSizeFieldName) { + this.pageSizeFieldName = pageSizeFieldName; + } + + public int getMaxPageSize() { + return maxPageSize; + } + + public void setMaxPageSize(int maxPageSize) { + this.maxPageSize = maxPageSize; + } + + public String getInitFirstTime() { + return initFirstTime; + } + + public void setInitFirstTime(String initFirstTime) { + this.initFirstTime = initFirstTime; + } + + public boolean isTimeStamp() { + return isTimeStamp; + } + + public void setTimeStamp(boolean timeStamp) { + isTimeStamp = timeStamp; + } + + public Long getPollingMinute() { + return pollingMinute; + } + + public void setPollingMinute(Long pollingMinute) { + this.pollingMinute = pollingMinute; + } + + public int getSplitCount() { + return splitCount; + } + + public void setSplitCount(int splitCount) { + this.splitCount = splitCount; + } + + public boolean isSecondTimeStamp() { + return isSecondTimeStamp; + } + + public void setSecondTimeStamp(boolean secondTimeStamp) { + isSecondTimeStamp = secondTimeStamp; + } + + public String getTotalCountField() { + return totalCountField; + } + + public void setTotalCountField(String totalCountField) { + this.totalCountField = totalCountField; + } + + public long getDateAddOfHistoryData() { + return dateAddOfHistoryData; + } + + public void setDateAddOfHistoryData(long dateAddOfHistoryData) { + this.dateAddOfHistoryData = dateAddOfHistoryData; + } + + public boolean isSts() { + return isSts; + } + + public void setSts(boolean sts) { + isSts = sts; + } + + public String getStsRoleArn() { + return stsRoleArn; + } + + public void setStsRoleArn(String stsRoleArn) { + this.stsRoleArn = stsRoleArn; + } + + public String getStsAssumeRoleFor() { + return stsAssumeRoleFor; + } + + public void setStsAssumeRoleFor(String stsAssumeRoleFor) { + this.stsAssumeRoleFor = stsAssumeRoleFor; + } + + public String getStsSessionPrefix() { + return stsSessionPrefix; + } + + public void setStsSessionPrefix(String stsSessionPrefix) { + this.stsSessionPrefix = stsSessionPrefix; + } + + public int getStsExpireSeconds() { + return stsExpireSeconds; + } + + public void setStsExpireSeconds(int stsExpireSeconds) { + this.stsExpireSeconds = stsExpireSeconds; + } + + public String getRamEndpoint() { + return ramEndpoint; + } + + public void setRamEndpoint(String ramEndpoint) { + this.ramEndpoint = ramEndpoint; + } + + public String getAliUid() { + return aliUid; + } + + public void setAliUid(String aliUid) { + this.aliUid = aliUid; + } +} diff --git a/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/AbstractOpenAPIExecutor.java b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/AbstractOpenAPIExecutor.java new file mode 100644 index 00000000..5a694199 --- /dev/null +++ b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/AbstractOpenAPIExecutor.java @@ -0,0 +1,210 @@ +/* + * 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.openapi.executor; + +import com.alibaba.fastjson.JSONObject; +import com.aliyuncs.http.FormatType; +import com.aliyuncs.http.MethodType; +import com.aliyuncs.http.ProtocolType; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; +import org.apache.rocketmq.streams.common.utils.SQLUtil; + +public abstract class AbstractOpenAPIExecutor { + public static final String JSON_FORMAT = "json"; + public static final String XML_FORMAT = "xml"; + private static final String PREFIX = "dipper.upgrade.channel.openapi.envkey"; + /** + * 需要用户输入的参数 + */ + protected String productName;//业务名称 + //调用业务接口的名称方法参照《OpenAPI变量的获取和例子项目的使用》中2.1.2节的方法获取 + protected String action; + // protected JSONObject query = new JSONObject();//业务参数 + //具体要调用openapi 接口 的版本 sas的版本如下 其他服务参考 对应服务的openapi文档 + protected String version = "2018-01-16"; + /** + * 如果作为channel,需要配置的调度参数 + */ + protected Long pollingTime = 10 * 60L;//轮询时间,单位是秒 + protected String cron;//详细的定时配置 + protected boolean startNow = true;//启动立刻执行一次 + /** + * 需要预先配置的参数 + */ + @ENVDependence + protected String accessKeyId; + @ENVDependence + protected String accessSecret; + //String domain = "sasprivate.cn-qingdao-env12-d01.env12.shuguang.com";//获取方法参照《OpenAPI变量的获取和例子项目的使用》中1.2节的方法获取 + @ENVDependence + protected String domain; + /** + * 默认不需要改变,特殊情况,可以覆盖的参数 + */ + protected String method = "GET"; //默认是get请求 + protected String format = JSON_FORMAT;//返回响应的格式 取值为JSON|XML 默认是XML + protected String protocol = ProtocolType.HTTPS.toString(); + protected int maxRetryCounts = 3;//最大重试次数 + protected transient Map value2Enum = new HashMap<>(); + + public AbstractOpenAPIExecutor() { + value2Enum.put(ProtocolType.HTTP.toString(), ProtocolType.HTTP); + value2Enum.put(ProtocolType.HTTPS.toString(), ProtocolType.HTTPS); + + value2Enum.put(MethodType.GET.toString(), MethodType.GET); + value2Enum.put(MethodType.POST.toString(), MethodType.POST); + + value2Enum.put(JSON_FORMAT, FormatType.JSON); + value2Enum.put(XML_FORMAT, FormatType.XML); + } + + protected JSONObject appBusinessParameters(JSONObject query) { + JSONObject paras = new JSONObject(); + if (query == null || query.size() == 0) { + return paras; + } + + Iterator> it = query.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry entry = it.next(); + String key = entry.getKey(); + Object value = entry.getValue(); + value = SQLUtil.parseIbatisSQL(query, value.toString(), true); + value = value.toString().replaceAll("'", ""); + paras.put(key, value); + } + return paras; + } + + protected JSONObject doInvoke(JSONObject query) { + JSONObject paras = appBusinessParameters(query); + JSONObject jsonObject = invoke(paras); + return jsonObject; + } + + protected abstract JSONObject invoke(JSONObject paras); + + public String getCron() { + return cron; + } + + public void setCron(String cron) { + this.cron = cron; + } + + public boolean isStartNow() { + return startNow; + } + + public void setStartNow(boolean startNow) { + this.startNow = startNow; + } + + public String getProductName() { + return productName; + } + + public void setProductName(String productName) { + this.productName = productName; + } + + public String getProtocol() { + return protocol; + } + + public void setProtocol(String protocol) { + this.protocol = protocol; + } + + public String getAccessKeyId() { + return accessKeyId; + } + + public void setAccessKeyId(String accessKeyId) { + this.accessKeyId = accessKeyId; + } + + public String getAccessSecret() { + return accessSecret; + } + + public void setAccessSecret(String accessSecret) { + this.accessSecret = accessSecret; + } + + public String getDomain() { + return domain; + } + + public void setDomain(String domain) { + this.domain = domain; + } + + public String getMethod() { + return method; + } + + public void setMethod(String method) { + this.method = method; + } + + public String getFormat() { + return format; + } + + public void setFormat(String format) { + this.format = format; + } + + public String getAction() { + return action; + } + + public void setAction(String action) { + this.action = action; + } + + public int getMaxRetryCounts() { + return maxRetryCounts; + } + + public void setMaxRetryCounts(int maxRetryCounts) { + this.maxRetryCounts = maxRetryCounts; + } + + public Long getPollingTime() { + return pollingTime; + } + + public void setPollingTime(Long pollingTime) { + this.pollingTime = pollingTime; + } + + protected void destroyCunsumer() { + } + + public String getVersion() { + return version; + } + + public void setVersion(String version) { + this.version = version; + } +} diff --git a/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/GenericAcsResponse.java b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/GenericAcsResponse.java new file mode 100644 index 00000000..be0e99ae --- /dev/null +++ b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/GenericAcsResponse.java @@ -0,0 +1,55 @@ +/* + * 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.openapi.executor; + +import com.aliyuncs.AcsResponse; +import com.aliyuncs.exceptions.ClientException; +import com.aliyuncs.exceptions.ServerException; +import com.aliyuncs.http.HttpResponse; +import com.aliyuncs.transform.UnmarshallerContext; +import java.io.UnsupportedEncodingException; + +public class GenericAcsResponse extends AcsResponse { + + private String content; + + public GenericAcsResponse() { + } + + @Override + public GenericAcsResponse getInstance(UnmarshallerContext context) throws ClientException, ServerException { + try { + HttpResponse httpResponse = context.getHttpResponse(); + if (httpResponse.getEncoding() == null) { + this.content = new String(httpResponse.getHttpContent()); + } else { + this.content = new String(httpResponse.getHttpContent(), httpResponse.getEncoding()); + } + return this; + } catch (UnsupportedEncodingException e) { + throw new ClientException("SDK.UnsupportedEncoding", "Can not parse response due to un supported encoding."); + } + } + + public String getContent() { + return this.content; + } + + public void setContent(String content) { + this.content = content; + } +} \ No newline at end of file diff --git a/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/GenericRpcAcsRequest.java b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/GenericRpcAcsRequest.java new file mode 100644 index 00000000..fb3760e6 --- /dev/null +++ b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/GenericRpcAcsRequest.java @@ -0,0 +1,53 @@ +/* + * 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.openapi.executor; + +import com.aliyuncs.RpcAcsRequest; +import com.aliyuncs.http.FormatType; + +public class GenericRpcAcsRequest extends RpcAcsRequest { + + public GenericRpcAcsRequest(String product) { + super(product); + this.setAcceptFormat(FormatType.JSON); + } + + public GenericRpcAcsRequest(String product, String version) { + super(product); + this.setAcceptFormat(FormatType.JSON); + } + + public GenericRpcAcsRequest(String product, String version, String action) { + super(product, version, action); + this.setAcceptFormat(FormatType.JSON); + } + + public GenericRpcAcsRequest(String product, String version, String action, String locationProduct) { + super(product, version, action, locationProduct); + this.setAcceptFormat(FormatType.JSON); + } + + public GenericRpcAcsRequest(String product, String version, String action, String locationProduct, String endpointType) { + super(product, version, action, locationProduct, endpointType); + this.setAcceptFormat(FormatType.JSON); + } + + @Override + public Class getResponseClass() { + return GenericAcsResponse.class; + } +} \ No newline at end of file diff --git a/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/HttpOpenAPIExecutor.java b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/HttpOpenAPIExecutor.java new file mode 100644 index 00000000..2ebef8b5 --- /dev/null +++ b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/HttpOpenAPIExecutor.java @@ -0,0 +1,128 @@ +/* + * 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.openapi.executor; + +import com.alibaba.fastjson.JSONObject; +import java.io.IOException; +import java.security.KeyManagementException; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.util.Iterator; +import java.util.Map; +import java.util.TreeMap; +import java.util.UUID; +import javax.net.ssl.SSLContext; +import org.apache.http.HttpEntity; +import org.apache.http.HttpHost; +import org.apache.http.client.config.RequestConfig; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.conn.ssl.SSLConnectionSocketFactory; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.ssl.SSLContextBuilder; +import org.apache.http.util.EntityUtils; +import org.apache.rocketmq.streams.openapi.utils.ParamUtil; + +public class HttpOpenAPIExecutor extends AbstractOpenAPIExecutor { + protected String proxyHttp;//主要用于日常环境测试,正式场景不需要 + protected int port = 0;//代理端口,主要用于测试 + protected transient String signatureMethod = "HMAC-SHA1"; + //唯一随机数,用于防止网络重放攻击。用户在不同请求间要使用不同的随机数值 + protected transient String signatureNonce = UUID.randomUUID().toString().replace("-", "").toUpperCase() + "fddsfsddsffsdfds"; + //签名算法版本 固定值 + protected transient String signatureVersion = "1.0"; + + public String createURL(JSONObject businessParameters) { + TreeMap paraMap = new TreeMap<>(); + paraMap.putAll(this.appBusinessParameters(businessParameters)); + //公共参数 + paraMap.put("Format", format); + paraMap.put("AccessKeyId", accessKeyId); + paraMap.put("Action", action); + paraMap.put("SignatureMethod", signatureMethod); + paraMap.put("SignatureNonce", signatureNonce); + paraMap.put("Version", version); + paraMap.put("SignatureVersion", signatureVersion); + Iterator> it = paraMap.entrySet().iterator(); + String url = ParamUtil.getUrl(domain, accessSecret, method, paraMap); + return url; + } + + @Override + protected JSONObject doInvoke(JSONObject businessParameters) { + JSONObject paras = appBusinessParameters(businessParameters); + return invoke(paras); + } + + @Override + protected JSONObject invoke(JSONObject businessParameters) { + RequestConfig defaultConfig = RequestConfig.custom().build(); + if (!"".equals(proxyHttp) && port != 0) { + HttpHost proxy = new HttpHost(proxyHttp, port, "http"); + defaultConfig = RequestConfig.custom().setProxy(proxy).build(); + } + + SSLContext sslContext = null; + CloseableHttpResponse response = null; + try { + sslContext = SSLContextBuilder.create().useProtocol(SSLConnectionSocketFactory.SSL).loadTrustMaterial((x, + y) -> true).build(); + } catch (KeyManagementException | NoSuchAlgorithmException | KeyStoreException e) { + e.printStackTrace(); + } + CloseableHttpClient httpClient = HttpClients.custom().setDefaultRequestConfig(defaultConfig).setSSLContext(sslContext).setSSLHostnameVerifier((x, y) -> true).build(); + String url = createURL(businessParameters); + HttpGet get = new HttpGet(url); + String result = ""; + try { + response = httpClient.execute(get); + HttpEntity entity = response.getEntity(); + result = EntityUtils.toString(entity, "utf-8"); + + } catch (IOException e) { + throw new RuntimeException("send open api http request error " + url); + + } finally { + if (response != null) { + try { + response.close(); + } catch (IOException e) { + e.printStackTrace(); + } + } + } + return JSONObject.parseObject(result); + + } + + public String getProxyHttp() { + return proxyHttp; + } + + public void setProxyHttp(String proxyHttp) { + this.proxyHttp = proxyHttp; + } + + public int getPort() { + return port; + } + + public void setPort(int port) { + this.port = port; + } +} diff --git a/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/OpenAPIExecutor.java b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/OpenAPIExecutor.java new file mode 100644 index 00000000..4e61d9ea --- /dev/null +++ b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/executor/OpenAPIExecutor.java @@ -0,0 +1,97 @@ +/* + * 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.openapi.executor; + +import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson.JSONObject; +import com.aliyuncs.DefaultAcsClient; +import com.aliyuncs.IAcsClient; +import com.aliyuncs.http.FormatType; +import com.aliyuncs.profile.DefaultProfile; +import java.util.Map; +import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; + +public class OpenAPIExecutor extends HttpOpenAPIExecutor { + @ENVDependence + protected String regionId = "global:region"; + protected boolean userHttp = false; + + public OpenAPIExecutor(String regionId, String accessKeyId, String accessSecret, String domain, String productName, String action, String version) { + this.setRegionId(regionId); + this.setAccessKeyId(accessKeyId); + this.setAccessSecret(accessSecret); + this.setAction(action); + this.setDomain(domain); + this.setProductName(productName); + this.setVersion(version); + } + + @Override + protected JSONObject invoke(JSONObject businessParameters) { + if (userHttp) { + return super.invoke(businessParameters); + } else { + return invokeBySDK(businessParameters, null); + } + } + + public JSONObject invokeBySDK(Map paras, DefaultProfile profile) { + try { + if (profile == null) { + profile = DefaultProfile.getProfile(this.regionId, this.accessKeyId, this.accessSecret); + } + DefaultProfile.addEndpoint(regionId, regionId, productName, domain); + IAcsClient client = new DefaultAcsClient(profile); + GenericRpcAcsRequest request = new GenericRpcAcsRequest(productName, version, action); + request.setAcceptFormat((FormatType) value2Enum.get(format)); + request.setConnectTimeout(80000); + request.setReadTimeout(80000); + if (paras != null && paras.size() > 0) { + for (Map.Entry entry : paras.entrySet()) { + request.putQueryParameter(entry.getKey(), entry.getValue()); + } + } + if (domain == null) { + throw new RuntimeException("product not exist"); + } + if (domain.contains("[")) { + domain = domain.replaceAll("\\[[^]]+]", regionId); + } + GenericAcsResponse response = client.getAcsResponse(request, false, maxRetryCounts); + return JSON.parseObject(response.getContent()); + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException("invoke pop service error " + productName + ":" + action, e); + } + } + + public String getRegionId() { + return regionId; + } + + public void setRegionId(String regionId) { + this.regionId = regionId; + } + + public boolean isUserHttp() { + return userHttp; + } + + public void setUserHttp(boolean userHttp) { + this.userHttp = userHttp; + } +} diff --git a/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/splits/OpenAPISplit.java b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/splits/OpenAPISplit.java new file mode 100644 index 00000000..ac8ed93e --- /dev/null +++ b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/splits/OpenAPISplit.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.openapi.splits; + +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; + +public class OpenAPISplit extends BasedConfigurable implements ISplit { + protected String queueId; + protected Long initStartTime; + protected long dateAdd; + protected Long pollingMinute; + + public OpenAPISplit(String queueId, Long initStartTime, long dateAdd, Long pollingMinute) { + this.queueId = queueId; + this.initStartTime = initStartTime; + this.dateAdd = dateAdd; + this.pollingMinute = pollingMinute; + } + + public OpenAPISplit() { + } + + @Override + public String getQueueId() { + return queueId; + } + + public void setQueueId(String queueId) { + this.queueId = queueId; + } + + @Override + public String getQueue() { + return queueId; + } + + @Override + public int compareTo(OpenAPISplit o) { + return queueId.compareTo(o.queueId); + } + + public Long getInitStartTime() { + return initStartTime; + } + + public void setInitStartTime(Long initStartTime) { + this.initStartTime = initStartTime; + } + + public long getDateAdd() { + return dateAdd; + } + + public void setDateAdd(long dateAdd) { + this.dateAdd = dateAdd; + } + + public Long getPollingMinute() { + return pollingMinute; + } + + public void setPollingMinute(Long pollingMinute) { + this.pollingMinute = pollingMinute; + } +} diff --git a/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/utils/ParamUtil.java b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/utils/ParamUtil.java new file mode 100644 index 00000000..760dce38 --- /dev/null +++ b/rocketmq-streams-channel-openapi/src/main/java/org/apache/rocketmq/streams/openapi/utils/ParamUtil.java @@ -0,0 +1,153 @@ +/* + * 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.openapi.utils; + +import com.aliyuncs.auth.AcsURLEncoder; +import java.io.UnsupportedEncodingException; +import java.net.URLEncoder; +import java.util.Calendar; +import java.util.SortedMap; +import java.util.UUID; +import java.util.stream.Collectors; +import javax.crypto.Mac; +import javax.crypto.spec.SecretKeySpec; +import org.apache.rocketmq.streams.common.utils.Base64Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ParamUtil { + private static final String ENCODING = "UTF-8"; + private static final String MAC_NAME = "HmacSHA1"; + private static final Logger LOGGER = LoggerFactory.getLogger(ParamUtil.class); + + public static String getUrl(String domain, String sk, String HTTPMethod, SortedMap params) { + //本地时间 与 服务器时间不能超过15分钟 + params.put("Timestamp", getUTCTimeStr()); + String signature = null; + StringBuffer url = null; + String SignatureNonce = null; + try { + //获取signature + do { + SignatureNonce = UUID.randomUUID().toString().replace("-", "").toUpperCase(); + params.put("SignatureNonce", SignatureNonce); + url = new StringBuffer(); + url.append("https://").append(domain).append("/?"); + url.append(allParams(params)); + signature = hamcsha1(composeStringToSign(HTTPMethod, params).getBytes(ENCODING), sk.getBytes(ENCODING)); + } + while (signature.contains("+")); + } catch (UnsupportedEncodingException e) { + LOGGER.error(" signature is:" + signature); + } + //url的拼接 + url.append("&").append("Signature").append("=").append(signature); + return url.toString(); + } + + public static String hamcsha1(byte[] data, byte[] key) { + try { + SecretKeySpec signingKey = new SecretKeySpec(key, "HmacSHA1"); + Mac mac = Mac.getInstance("HmacSHA1"); + mac.init(signingKey); + return Base64Utils.encode(mac.doFinal(data)); + } catch (Exception e) { + LOGGER.error("hamc sha1 error", e); + } + return null; + } + + private static String composeStringToSign(String method, SortedMap queryParameters) { + try { + String canonicalQueryString = queryParameters.entrySet().stream().map(entry -> percentEncode(entry.getKey()) + "=" + percentEncode(entry.getValue().toString())).collect(Collectors.joining("&")); + + LOGGER.info(canonicalQueryString); + return method.toUpperCase() + "&" + AcsURLEncoder.percentEncode("/") + "&" + AcsURLEncoder.percentEncode(canonicalQueryString); + } catch (UnsupportedEncodingException e) { + LOGGER.error(" composeStringToSign error " + e); + return null; + } + + } + + private static String allParams(SortedMap queryParameters) { + return queryParameters.entrySet().stream().map(entry -> entry.getKey() + "=" + entry.getValue()).collect(Collectors.joining("&")); + } + + public static String percentEncode(String value) { + try { + return value != null ? URLEncoder.encode(value, ENCODING).replace("+", "%20").replace("*", "%2A").replace("%7E", "~") : null; + } catch (UnsupportedEncodingException e) { + LOGGER.error(" percentEncode error " + e); + return null; + } + + } + + private static String getUTCTimeStr() { + StringBuffer UTCTimeBuffer = new StringBuffer(); + // 1、取得本地时间: + Calendar cal = Calendar.getInstance(); + // 2、取得时间偏移量: + int zoneOffset = cal.get(java.util.Calendar.ZONE_OFFSET); + // 3、取得夏令时差: + int dstOffset = cal.get(java.util.Calendar.DST_OFFSET); + // 4、从本地时间里扣除这些差量,即可以取得UTC时间: + cal.add(java.util.Calendar.MILLISECOND, -(zoneOffset + dstOffset)); + int year = cal.get(Calendar.YEAR); + int month = cal.get(Calendar.MONTH) + 1; + int day = cal.get(Calendar.DAY_OF_MONTH); + int hour = cal.get(Calendar.HOUR_OF_DAY); + int minute = cal.get(Calendar.MINUTE); + int sec = cal.get(Calendar.SECOND); + StringBuffer appendMonth = new StringBuffer(); + if (month < 10) { + appendMonth.append("0").append(month); + } else { + appendMonth.append(month); + } + StringBuffer appendDay = new StringBuffer(); + if (day < 10) { + appendDay.append("0").append(day); + } else { + appendDay.append(day); + } + StringBuffer appendHour = new StringBuffer(); + if (hour < 10) { + appendHour.append("0").append(hour); + } else { + appendHour.append(hour); + } + StringBuffer appendMinute = new StringBuffer(); + if (minute < 10) { + appendMinute.append("0").append(minute); + } else { + appendMinute.append(minute); + } + StringBuffer appendSec = new StringBuffer(); + if (sec < 10) { + appendSec.append("0").append(sec); + } else { + appendSec.append(sec); + } + + UTCTimeBuffer.append(year).append("-").append(appendMonth).append("-").append(appendDay); + UTCTimeBuffer.append("T").append(appendHour).append(":").append(appendMinute).append(":").append(appendSec).append("Z"); + + return UTCTimeBuffer.toString(); + } +} diff --git a/rocketmq-streams-channel-openapi/src/test/java/org/apache/rocketmq/streams/openapi/OpenAPIExecutorTest.java b/rocketmq-streams-channel-openapi/src/test/java/org/apache/rocketmq/streams/openapi/OpenAPIExecutorTest.java new file mode 100644 index 00000000..64028483 --- /dev/null +++ b/rocketmq-streams-channel-openapi/src/test/java/org/apache/rocketmq/streams/openapi/OpenAPIExecutorTest.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.openapi; + +import org.junit.Test; + +public class OpenAPIExecutorTest { + + @Test + public void testExecutor() { +// OpenAPIExecutor openAPIExecutor=new OpenAPIExecutor("cn-hangzhou","","","tds.aliyuncs.com","Sas","DescribeAlarmEventList","2018-12-03"); +// Map paras=new HashMap<>(); +// paras.put("CurrentPage",1); +// paras.put("PageSize",10); +// paras.put("From","sas"); +// JSONObject msg= openAPIExecutor.invokeBySDK(paras); +// System.out.println(JsonableUtil.formatJson(msg)); + System.out.println(System.currentTimeMillis()); + } +} diff --git a/rocketmq-streams-channel-openapi/src/test/java/org/apache/rocketmq/streams/openapi/RuleEngineCenterTest.java b/rocketmq-streams-channel-openapi/src/test/java/org/apache/rocketmq/streams/openapi/RuleEngineCenterTest.java new file mode 100644 index 00000000..fe2d2342 --- /dev/null +++ b/rocketmq-streams-channel-openapi/src/test/java/org/apache/rocketmq/streams/openapi/RuleEngineCenterTest.java @@ -0,0 +1,67 @@ +/* + * 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.openapi; + +import org.junit.Test; + +public class RuleEngineCenterTest { + + @Test + public void testRuleEngine() { +// Op channel = new HTTPOpenAPIChannel(); +// channel.setAccessKeyId(""); +// channel.setAccessSecret("&"); +// channel.setAction(""); +// channel.setDomain(""); +// channel.setProductName("Sas"); +// channel.setVersion("2017-07-13"); +// Map businessParameters = new HashMap<>(); +// businessParameters.put("PublishVersionCode", 0); +// businessParameters.put("PrivateCloudVersion", "V3"); +// businessParameters.put("PublishVersion", "0"); +// businessParameters.put("AliUid", 1816086363469125L); +// channel.setParameters(businessParameters); +// channel.setProxyHttp("100.127.2.62"); +// channel.setPort(8888); +// channel.init(); +// String proxy = ENVUtile.getENVParameter("http_proxy"); +// System.out.println("http_proxy=" + proxy); +// +// System.setProperty("http.proxyHost", "100.127.2.62"); +// +// System.setProperty("http.proxyPort", "8888"); +// +// System.setProperty("https.proxyHost", "100.127.2.62"); +// +// System.setProperty("https.proxyPort", "8888"); +// channel.start(new IStreamOperator() { +// @Override +// public Object doMessage(IMessage message, AbstractContext context) { +// System.out.println(message.getMessageBody()); +// return null; +// } +// }); +// while (true) { +// try { +// Thread.sleep(1000); +// } catch (InterruptedException e) { +// e.printStackTrace(); +// } +// } + } + +} diff --git a/rocketmq-streams-channel-rocketmq/pom.xml b/rocketmq-streams-channel-rocketmq/pom.xml index 3b12c212..28679a19 100644 --- a/rocketmq-streams-channel-rocketmq/pom.xml +++ b/rocketmq-streams-channel-rocketmq/pom.xml @@ -15,13 +15,13 @@ See the License for the specific language governing permissions and limitations under the License. --> - rocketmq-streams org.apache.rocketmq - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT 4.0.0 @@ -37,35 +37,6 @@ org.apache.rocketmq rocketmq-streams-serviceloader - - - org.apache.rocketmq - rocketmq-client - - - ch.qos.logback - logback-classic - - - ch.qos.logback - logback-core - - - - - org.apache.rocketmq - rocketmq-tools - - - org.apache.rocketmq - rocketmq-acl - - - junit - junit - test - - \ No newline at end of file diff --git a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/RocketMQChannelBuilder.java b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/RocketMQChannelBuilder.java index 682888aa..98af1538 100644 --- a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/RocketMQChannelBuilder.java +++ b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/RocketMQChannelBuilder.java @@ -27,11 +27,11 @@ import org.apache.rocketmq.streams.common.metadata.MetaData; import org.apache.rocketmq.streams.common.model.ServiceName; import org.apache.rocketmq.streams.common.utils.ConfigurableUtil; -import org.apache.rocketmq.streams.sink.RocketMQSink; -import org.apache.rocketmq.streams.source.RocketMQSource; +import org.apache.rocketmq.streams.rocketmq.sink.RocketMQSink; +import org.apache.rocketmq.streams.rocketmq.source.RocketMQSource; @AutoService(IChannelBuilder.class) -@ServiceName(value = RocketMQChannelBuilder.TYPE, aliasName = "RocketMQSource") +@ServiceName(value = RocketMQChannelBuilder.TYPE, aliasName = "RocketMQSource,metaq") public class RocketMQChannelBuilder extends AbstractSupportShuffleChannelBuilder { public static final String TYPE = "rocketmq"; @@ -40,6 +40,7 @@ public ISource createSource(String namespace, String name, Properties properties return (RocketMQSource) ConfigurableUtil.create(RocketMQSource.class.getName(), namespace, name, createFormatProperty(properties), null); } + @Override protected JSONObject createFormatProperty(Properties properties) { JSONObject formatProperties = new JSONObject(); for (Object object : properties.keySet()) { @@ -50,29 +51,16 @@ protected JSONObject createFormatProperty(Properties properties) { formatProperties.put(key, properties.get(key)); } IChannelBuilder.formatPropertiesName(formatProperties, properties, "topic", "topic"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "tags", "tags"); IChannelBuilder.formatPropertiesName(formatProperties, properties, "tags", "tag"); IChannelBuilder.formatPropertiesName(formatProperties, properties, "maxThread", "thread.max.count"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "maxThread", "maxthread"); IChannelBuilder.formatPropertiesName(formatProperties, properties, "pullIntervalMs", "pullIntervalMs"); IChannelBuilder.formatPropertiesName(formatProperties, properties, "offsetTime", "offsetTime"); IChannelBuilder.formatPropertiesName(formatProperties, properties, "namesrvAddr", "namesrvAddr"); IChannelBuilder.formatPropertiesName(formatProperties, properties, "groupName", "producerGroup"); IChannelBuilder.formatPropertiesName(formatProperties, properties, "groupName", "consumerGroup"); - IChannelBuilder.formatPropertiesName(formatProperties, properties, "maxThread", "maxthread"); - IChannelBuilder.formatPropertiesName(formatProperties, properties, "pullIntervalMs", "pullintervalms"); - IChannelBuilder.formatPropertiesName(formatProperties, properties, "offsetTime", "offsettime"); - IChannelBuilder.formatPropertiesName(formatProperties, properties, "namesrvAddr", "namesrvaddr"); - IChannelBuilder.formatPropertiesName(formatProperties, properties, "groupName", "producergroup"); - IChannelBuilder.formatPropertiesName(formatProperties, properties, "groupName", "consumergroup"); - if (properties.getProperty("group") != null) { - String group = properties.getProperty("group"); - if (group.startsWith("GID_")) { - formatProperties.put("groupName", group); - } else { - formatProperties.put("groupName", "GID_" + group); - } - } - return formatProperties; } diff --git a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/RocketMQOffset.java b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/RocketMQOffset.java index 66507445..3d660a06 100644 --- a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/RocketMQOffset.java +++ b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/RocketMQOffset.java @@ -29,17 +29,17 @@ import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.remoting.exception.RemotingException; -import org.apache.rocketmq.streams.common.channel.source.AbstractSupportShuffleSource; +import org.apache.rocketmq.streams.common.channel.source.AbstractPushSource; import org.apache.rocketmq.streams.common.utils.ReflectUtil; -import org.apache.rocketmq.streams.debug.DebugWriter; -import org.apache.rocketmq.streams.queue.RocketMQMessageQueue; +import org.apache.rocketmq.streams.rocketmq.debug.DebugWriter; +import org.apache.rocketmq.streams.rocketmq.queue.RocketMQMessageQueue; public class RocketMQOffset implements OffsetStore { protected OffsetStore offsetStore; - protected AbstractSupportShuffleSource source; + protected AbstractPushSource source; private AtomicBoolean starting; - public RocketMQOffset(OffsetStore offsetStore, AbstractSupportShuffleSource source) { + public RocketMQOffset(OffsetStore offsetStore, AbstractPushSource source) { this.offsetStore = offsetStore; this.source = source; this.starting = new AtomicBoolean(true); diff --git a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/debug/DebugWriter.java b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/rocketmq/debug/DebugWriter.java similarity index 97% rename from rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/debug/DebugWriter.java rename to rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/rocketmq/debug/DebugWriter.java index 1c622a81..41610e13 100644 --- a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/debug/DebugWriter.java +++ b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/rocketmq/debug/DebugWriter.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.rocketmq.streams.debug; +package org.apache.rocketmq.streams.rocketmq.debug; import com.alibaba.fastjson.JSONObject; import java.util.ArrayList; @@ -27,16 +27,22 @@ import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.common.utils.DateUtil; import org.apache.rocketmq.streams.common.utils.FileUtil; -import org.apache.rocketmq.streams.queue.RocketMQMessageQueue; +import org.apache.rocketmq.streams.rocketmq.queue.RocketMQMessageQueue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class DebugWriter { private static final Logger LOGGER = LoggerFactory.getLogger(DebugWriter.class); - - protected String dir = "/tmp/rocksmq-streams/mq"; protected static Map debugWriterMap = new HashMap<>(); + protected String dir = "/tmp/rocksmq-streams/mq"; + + public DebugWriter() { + } + + public DebugWriter(String dir) { + this.dir = dir; + } public static DebugWriter getInstance(String topic) { DebugWriter debugWriter = debugWriterMap.get(topic); @@ -51,13 +57,6 @@ public static boolean isOpenDebug() { return false; } - public DebugWriter() { - } - - public DebugWriter(String dir) { - this.dir = dir; - } - /** * write offset 2 file * diff --git a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/queue/RocketMQMessageQueue.java b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/rocketmq/queue/RocketMQMessageQueue.java similarity index 74% rename from rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/queue/RocketMQMessageQueue.java rename to rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/rocketmq/queue/RocketMQMessageQueue.java index add29d7e..3cd0dfcb 100644 --- a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/queue/RocketMQMessageQueue.java +++ b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/rocketmq/queue/RocketMQMessageQueue.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.queue; +package org.apache.rocketmq.streams.rocketmq.queue; import com.alibaba.fastjson.JSONObject; import org.apache.rocketmq.common.message.MessageQueue; @@ -28,23 +28,49 @@ public class RocketMQMessageQueue extends BasedConfigurable implements ISplit 1) { + return MapKeyUtil.createKeyBySign("_", topic[1], queue.getBrokerName(), getSplitNumerStr(queue.getQueueId()) + ""); + } + return MapKeyUtil.createKeyBySign("_", queue.getTopic(), queue.getBrokerName(), getSplitNumerStr(queue.getQueueId()) + ""); } - public RocketMQMessageQueue() { + /** + * 获取分片的字符串格式,需要3位对齐 + * + * @param splitNumer + * @return + */ + private static String getSplitNumerStr(int splitNumer) { + int len = (splitNumer + "").length(); + if (len == 3) { + return splitNumer + ""; + } + String splitNumerStr = splitNumer + ""; + while (len < 3) { + splitNumerStr = "0" + splitNumerStr; + len = splitNumerStr.length(); + } + return splitNumerStr; + } + @Override + protected void getJsonObject(JSONObject jsonObject) { + super.getJsonObject(jsonObject); + queue = new MessageQueue(topic, brokeName, mqQueueId); } @Override @@ -57,40 +83,11 @@ public int compareTo(RocketMQMessageQueue o) { return queue.compareTo(o.queue); } - - @Override public String getQueueId() { return getQueueId(this.queue); } - - public static String getQueueId(MessageQueue queue){ - - String[] topic = queue.getTopic().split("%"); - if (topic.length > 1) { - return MapKeyUtil.createKeyBySign("_",topic[1],queue.getBrokerName(),getSplitNumerStr(queue.getQueueId())+""); - } - return MapKeyUtil.createKeyBySign("_",queue.getTopic(),queue.getBrokerName(),getSplitNumerStr(queue.getQueueId())+""); - } - /** - * 获取分片的字符串格式,需要3位对齐 - * @param splitNumer - * @return - */ - private static String getSplitNumerStr(int splitNumer){ - int len=(splitNumer+"").length(); - if(len==3){ - return splitNumer+""; - } - String splitNumerStr=splitNumer+""; - while (len<3){ - splitNumerStr="0"+splitNumerStr; - len=splitNumerStr.length(); - } - return splitNumerStr; - } - public String getBrokeName() { return brokeName; } @@ -115,5 +112,4 @@ public void setMqQueueId(int mqQueueId) { this.mqQueueId = mqQueueId; } - } diff --git a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/sink/RocketMQSink.java b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/rocketmq/sink/RocketMQSink.java similarity index 71% rename from rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/sink/RocketMQSink.java rename to rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/rocketmq/sink/RocketMQSink.java index d825603a..3d8eead6 100644 --- a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/sink/RocketMQSink.java +++ b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/rocketmq/sink/RocketMQSink.java @@ -15,54 +15,45 @@ * limitations under the License. */ -package org.apache.rocketmq.streams.sink; +package org.apache.rocketmq.streams.rocketmq.sink; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Comparator; -import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; -import org.apache.rocketmq.client.exception.MQBrokerException; -import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.client.producer.DefaultMQProducer; -import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.message.Message; import org.apache.rocketmq.common.message.MessageQueue; -import org.apache.rocketmq.common.protocol.RequestCode; -import org.apache.rocketmq.common.protocol.ResponseCode; -import org.apache.rocketmq.common.protocol.header.CreateTopicRequestHeader; -import org.apache.rocketmq.remoting.exception.RemotingException; -import org.apache.rocketmq.remoting.protocol.RemotingCommand; 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.annotation.ENVDependence; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.utils.StringUtil; -import org.apache.rocketmq.streams.queue.RocketMQMessageQueue; +import org.apache.rocketmq.streams.rocketmq.queue.RocketMQMessageQueue; import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; import org.apache.rocketmq.tools.command.CommandUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class RocketMQSink extends AbstractSupportShuffleSink { - private static final Log LOG = LogFactory.getLog(RocketMQSink.class); - @ENVDependence - private String tags = "*"; + private static final Logger LOGGER = LoggerFactory.getLogger(RocketMQSink.class); private String topic; + @ENVDependence private String tags = "*"; + private String groupName; private String clusterName = "DefaultCluster"; private boolean order = false; - private transient List consumers = new ArrayList<>(); - private transient DefaultMQProducer producer; + private List consumers = new ArrayList<>(); + private DefaultMQProducer producer; private Long pullIntervalMs; private String namesrvAddr; @@ -87,20 +78,18 @@ public RocketMQSink(String namesrvAddr, String topic, String groupName, String c this.groupName = groupName; } - @Override - protected boolean initConfigurable() { + @Override protected boolean initConfigurable() { super.initConfigurable(); return true; } - @Override - protected boolean batchInsert(List messages) { + @Override protected boolean batchInsert(List messages) { if (messages == null) { return true; } if (StringUtil.isEmpty(topic)) { - if (LOG.isErrorEnabled()) { - LOG.error("topic is blank"); + if (LOGGER.isErrorEnabled()) { + LOGGER.error("topic is blank"); } return false; } @@ -151,20 +140,14 @@ protected void initProducer() { if (producer == null) { synchronized (this) { if (producer == null) { - destroy(); - producer = new DefaultMQProducer(groupName + "producer", true, null); try { - //please not use the code,the name srv addr may be empty in jmenv -// if (this.namesrvAddr == null || "".equals(this.namesrvAddr)) { -// throw new RuntimeException("namesrvAddr can not be null."); -// } - + destroy(); + producer = new DefaultMQProducer(groupName, true, null); if (StringUtil.isNotEmpty(this.namesrvAddr)) { producer.setNamesrvAddr(this.namesrvAddr); } producer.start(); } catch (Exception e) { - setInitSuccess(false); throw new RuntimeException("create producer failed," + topic + ",msg=" + e.getMessage(), e); } } @@ -179,28 +162,26 @@ public void destroyProduce() { producer.shutdown(); producer = null; } catch (Throwable t) { - if (LOG.isWarnEnabled()) { - LOG.warn(t.getMessage(), t); + if (LOGGER.isWarnEnabled()) { + LOGGER.warn(t.getMessage(), t); } + throw new RuntimeException("producer close error", t); } } } - @Override - public void destroy() { + @Override public void destroy() { super.destroy(); destroyProduce(); } - @Override - public String getShuffleTopicFieldName() { + @Override public String getShuffleTopicFieldName() { return "topic"; } - @Override - protected void createTopicIfNotExist(int splitNum) { + @Override protected void createTopicIfNotExist(int splitNum) { if (StringUtil.isEmpty(topic)) { - LOG.error("Topic should be empty"); + LOGGER.error("Topic should be empty"); throw new RuntimeException("Topic should be empty"); } DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt(); @@ -214,58 +195,50 @@ protected void createTopicIfNotExist(int splitNum) { try { defaultMQAdminExt.start(); - Set masterSet = - CommandUtil.fetchMasterAddrByClusterName(defaultMQAdminExt, clusterName); + Set masterSet = CommandUtil.fetchMasterAddrByClusterName(defaultMQAdminExt, clusterName); for (String master : masterSet) { defaultMQAdminExt.createAndUpdateTopicConfig(master, topicConfig); - LOG.info("Create topic to success: " + master); + LOGGER.info("Create topic to success: " + master); } if (this.order) { - Set brokerNameSet = - CommandUtil.fetchBrokerNameByClusterName(defaultMQAdminExt, clusterName); + Set brokerNameSet = CommandUtil.fetchBrokerNameByClusterName(defaultMQAdminExt, clusterName); StringBuilder orderConf = new StringBuilder(); String splitor = ""; for (String s : brokerNameSet) { - orderConf.append(splitor).append(s).append(":") - .append(topicConfig.getWriteQueueNums()); + orderConf.append(splitor).append(s).append(":").append(topicConfig.getWriteQueueNums()); splitor = ";"; } - defaultMQAdminExt.createOrUpdateOrderConf(topicConfig.getTopicName(), - orderConf.toString(), true); - System.out.printf("set cluster orderConf. isOrder=%s, orderConf=[%s]", order, orderConf); + defaultMQAdminExt.createOrUpdateOrderConf(topicConfig.getTopicName(), orderConf.toString(), true); + LOGGER.info("set cluster orderConf. isOrder={}, orderConf=[{}]", order, orderConf); } } catch (Exception e) { - LOG.error("Create topic error", e); + LOGGER.error("Create topic error", e); throw new RuntimeException("Create topic error " + topic, e); } finally { defaultMQAdminExt.shutdown(); } } - @Override - public List> getSplitList() { + @Override public List> getSplitList() { initProducer(); - List> messageQueues = new ArrayList<>(); - List metaqQueueSet = new ArrayList<>(); + List> messageQueues; + List messageQueueSet; try { + try { + messageQueueSet = producer.fetchPublishMessageQueues(topic); + } catch (Exception e) { + producer.send(new Message(topic, "test", "test".getBytes(StandardCharsets.UTF_8))); + messageQueueSet = producer.fetchPublishMessageQueues(topic); + } + List> queueList = new ArrayList<>(); + for (MessageQueue queue : messageQueueSet) { + RocketMQMessageQueue rocketMQMessageQueue = new RocketMQMessageQueue(queue); + queueList.add(rocketMQMessageQueue); - if (messageQueues == null || messageQueues.size() == 0) { - try { - metaqQueueSet = producer.fetchPublishMessageQueues(topic); - }catch (Exception e) { - producer.send(new Message(topic, "test", "test".getBytes(StandardCharsets.UTF_8))); - metaqQueueSet = producer.fetchPublishMessageQueues(topic); - } - List> queueList = new ArrayList<>(); - for (MessageQueue queue : metaqQueueSet) { - RocketMQMessageQueue rocketMQMessageQueue = new RocketMQMessageQueue(queue); - queueList.add(rocketMQMessageQueue); - - } - queueList.sort((Comparator) Comparable::compareTo); - messageQueues = queueList; } + queueList.sort((Comparator) Comparable::compareTo); + messageQueues = queueList; } catch (Exception e) { throw new RuntimeException(e); } @@ -273,8 +246,7 @@ protected void createTopicIfNotExist(int splitNum) { return messageQueues; } - @Override - public int getSplitNum() { + @Override public int getSplitNum() { List> splits = getSplitList(); if (splits == null || splits.size() == 0) { return 0; diff --git a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/source/RocketMQSource.java b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/rocketmq/source/RocketMQSource.java similarity index 58% rename from rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/source/RocketMQSource.java rename to rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/rocketmq/source/RocketMQSource.java index 641a464d..0ece98ea 100644 --- a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/source/RocketMQSource.java +++ b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/rocketmq/source/RocketMQSource.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.rocketmq.streams.source; +package org.apache.rocketmq.streams.rocketmq.source; import com.alibaba.fastjson.JSONObject; import java.util.ArrayList; @@ -28,11 +28,12 @@ import java.util.UUID; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicLong; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.client.consumer.AllocateMessageQueueStrategy; import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; +import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext; +import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus; import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyStatus; +import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently; import org.apache.rocketmq.client.consumer.listener.MessageListenerOrderly; import org.apache.rocketmq.client.consumer.rebalance.AllocateMessageQueueAveragely; import org.apache.rocketmq.client.consumer.rebalance.AllocateMessageQueueByConfig; @@ -41,6 +42,7 @@ import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.client.impl.MQClientManager; import org.apache.rocketmq.client.impl.consumer.DefaultMQPushConsumerImpl; +import org.apache.rocketmq.client.impl.consumer.RebalancePushImpl; import org.apache.rocketmq.client.impl.factory.MQClientInstance; import org.apache.rocketmq.common.consumer.ConsumeFromWhere; import org.apache.rocketmq.common.message.MessageExt; @@ -51,24 +53,26 @@ import org.apache.rocketmq.common.protocol.body.ConsumerRunningInfo; import org.apache.rocketmq.common.protocol.heartbeat.MessageModel; import org.apache.rocketmq.remoting.exception.RemotingException; -import org.apache.rocketmq.streams.common.channel.source.AbstractSupportShuffleSource; +import org.apache.rocketmq.streams.common.channel.source.AbstractPushSource; import org.apache.rocketmq.streams.common.channel.split.ISplit; import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; import org.apache.rocketmq.streams.common.utils.ReflectUtil; -import org.apache.rocketmq.streams.debug.DebugWriter; -import org.apache.rocketmq.streams.queue.RocketMQMessageQueue; +import org.apache.rocketmq.streams.common.utils.RuntimeUtil; +import org.apache.rocketmq.streams.rocketmq.debug.DebugWriter; +import org.apache.rocketmq.streams.rocketmq.queue.RocketMQMessageQueue; import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class RocketMQSource extends AbstractSupportShuffleSource { +public class RocketMQSource extends AbstractPushSource { - protected static final Log LOG = LogFactory.getLog(RocketMQSource.class); + private static final Logger LOGGER = LoggerFactory.getLogger(RocketMQSource.class); private static final String STRATEGY_AVERAGE = "average"; private static final String STRATEGY_MACHINE = "machine"; private static final String STRATEGY_CONFIG = "config"; - @ENVDependence - protected String tags = "*"; + @ENVDependence protected String tags = "*"; /** * 消息队列命名空间接入点 @@ -79,11 +83,16 @@ public class RocketMQSource extends AbstractSupportShuffleSource { protected String strategyName; + protected String instanceName; + + protected boolean isMessageListenerConcurrently = false;//当有窗口时,不建议用concurrent,会有丢数据的情况 + protected transient DefaultMQPushConsumer consumer; protected transient ConsumeFromWhere consumeFromWhere;//默认从哪里消费,不会被持久化。不设置默认从尾部消费 protected transient String consumerOffset;//从哪里开始消费 public RocketMQSource() { + this.instanceName = RuntimeUtil.getDipperInstanceId(); } public RocketMQSource(String topic, String tags, String groupName, String namesrvAddr) { @@ -96,120 +105,124 @@ public RocketMQSource(String topic, String tags, String groupName, String namesr this.groupName = groupName; this.namesrvAddr = namesrvAddr; this.strategyName = strategyName; + this.instanceName = RuntimeUtil.getDipperInstanceId(); } - @Override - protected boolean initConfigurable() { + @Override protected boolean initConfigurable() { return super.initConfigurable(); } - @Override - protected boolean startSource() { + @Override protected boolean startSource() { try { - destroyConsumer(); - consumer = startConsumer(); + if (this.consumer == null) { + this.consumer = new DefaultMQPushConsumer(groupName); + DefaultMQPushConsumerImpl impl = ReflectUtil.getDeclaredField(this.consumer, "defaultMQPushConsumerImpl"); + + ReflectUtil.setBeanFieldValue(impl, "rebalanceImpl", new RebalancePushImpl(impl) { + + @Override public boolean doRebalance(boolean isOrder) { + super.doRebalance(isOrder); + List cid = super.mQClientFactory.findConsumerIdList(topic, groupName); + doDispatch(topic, cid); + return true; + } + }); + + this.consumer.setInstanceName(this.instanceName); + if (pullIntervalMs != null) { + this.consumer.setPullInterval(pullIntervalMs); + } + AllocateMessageQueueStrategy defaultStrategy = new AllocateMessageQueueAveragely(); + if (STRATEGY_AVERAGE.equalsIgnoreCase(this.strategyName)) { + this.consumer.setAllocateMessageQueueStrategy(defaultStrategy); + } else if (STRATEGY_MACHINE.equalsIgnoreCase(this.strategyName)) { + //consumer.setAllocateMessageQueueStrategy(new AllocateMessageQueueByMachine(defaultStrategy)); + } else if (STRATEGY_CONFIG.equalsIgnoreCase(this.strategyName)) { + this.consumer.setAllocateMessageQueueStrategy(new AllocateMessageQueueByConfig()); + } + + this.consumer.setPersistConsumerOffsetInterval((int) this.checkpointTime); + this.consumer.setConsumeMessageBatchMaxSize(maxFetchLogGroupSize); + this.consumer.setNamesrvAddr(this.namesrvAddr); + if (consumeFromWhere != null) { + this.consumer.setConsumeFromWhere(consumeFromWhere); + if (consumerOffset != null) { + this.consumer.setConsumeTimestamp(consumerOffset); + } + } else { + this.consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET); + } + //consumer.setCommitOffsetWithPullRequestEnable(false); + this.consumer.subscribe(topic, tags); + if (isMessageListenerConcurrently) { + this.consumer.registerMessageListener(new MessageListenerConcurrently() { + @Override public ConsumeConcurrentlyStatus consumeMessage(List msgs, + ConsumeConcurrentlyContext context) { + processMessage(msgs, context.getMessageQueue()); + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; + } + }); + } else { + this.consumer.registerMessageListener((MessageListenerOrderly) (msgs, context) -> { + processMessage(msgs, context.getMessageQueue()); + return ConsumeOrderlyStatus.SUCCESS;// 返回消费成功 + }); + } + + setOffsetStore(this.consumer); + } + this.consumer.start(); return true; } catch (Exception e) { - setInitSuccess(false); e.printStackTrace(); throw new RuntimeException("start rocketmq channel error " + topic, e); } } - protected DefaultMQPushConsumer startConsumer() { + protected void processMessage(List msgs, MessageQueue messageQueue) { try { - DefaultMQPushConsumer consumer = new DefaultMQPushConsumer(groupName); - if (pullIntervalMs != null) { - consumer.setPullInterval(pullIntervalMs); - } - AllocateMessageQueueStrategy defaultStrategy = new AllocateMessageQueueAveragely(); - if (STRATEGY_AVERAGE.equalsIgnoreCase(this.strategyName)) { - consumer.setAllocateMessageQueueStrategy(defaultStrategy); - } else if (STRATEGY_MACHINE.equalsIgnoreCase(this.strategyName)) { - //consumer.setAllocateMessageQueueStrategy(new AllocateMessageQueueByMachine(defaultStrategy)); - } else if (STRATEGY_CONFIG.equalsIgnoreCase(this.strategyName)) { - consumer.setAllocateMessageQueueStrategy(new AllocateMessageQueueByConfig()); - } - - consumer.setPersistConsumerOffsetInterval((int) this.checkpointTime); - consumer.setConsumeMessageBatchMaxSize(maxFetchLogGroupSize); - consumer.setNamesrvAddr(this.namesrvAddr); - if (consumeFromWhere != null) { - consumer.setConsumeFromWhere(consumeFromWhere); - if (consumerOffset != null) { - consumer.setConsumeTimestamp(consumerOffset); - } - } - Map isFirstDataForQueue = new HashMap<>(); - //consumer.setCommitOffsetWithPullRequestEnable(false); - consumer.subscribe(topic, tags); - consumer.registerMessageListener((MessageListenerOrderly) (msgs, context) -> { - try { - int i = 0; - for (MessageExt msg : msgs) { - - JSONObject jsonObject = create(msg.getBody(), msg.getProperties()); - - String queueId = RocketMQMessageQueue.getQueueId(context.getMessageQueue()); - String offset = msg.getQueueOffset() + ""; - org.apache.rocketmq.streams.common.context.Message message = createMessage(jsonObject, queueId, offset, false); - message.getHeader().setOffsetIsLong(true); - - if (DebugWriter.isOpenDebug()) { - Boolean isFirstData = isFirstDataForQueue.get(queueId); - if (isFirstData == null) { - synchronized (this) { - isFirstData = isFirstDataForQueue.get(queueId); - if (isFirstData == null) { - isFirstDataForQueue.put(queueId, true); - } - DebugWriter.getInstance(getTopic()).receiveFirstData(queueId, msg.getQueueOffset()); - } - } - } + int i = 0; + for (MessageExt msg : msgs) { - if (i == msgs.size() - 1) { - message.getHeader().setNeedFlush(true); - } - executeMessage(message); - i++; - } - } catch (Exception e) { + JSONObject jsonObject = create(msg.getBody(), msg.getProperties()); - LOG.error("consume message from rocketmq error " + e, e); - e.printStackTrace(); + String queueId = RocketMQMessageQueue.getQueueId(messageQueue); + String offset = msg.getQueueOffset() + ""; + org.apache.rocketmq.streams.common.context.Message message = createMessage(jsonObject, queueId, offset, false); + message.getHeader().setOffsetIsLong(true); + if (i == msgs.size() - 1) { + message.getHeader().setNeedFlush(true); } - - return ConsumeOrderlyStatus.SUCCESS;// 返回消费成功 - }); - - setOffsetStore(consumer); - consumer.start(); - - return consumer; + executeMessage(message); + i++; + } } catch (Exception e) { - setInitSuccess(false); + LOGGER.error("consume message from rocketmq error " + e, e); e.printStackTrace(); - throw new RuntimeException("start metaq channel error " + topic, e); } - } - @Override - public List getAllSplits() { - try { - List messageQueues = new ArrayList<>(); - if (messageQueues == null || messageQueues.size() == 0) { - Set metaqQueueSet = consumer.fetchSubscribeMessageQueues(this.topic); - for (MessageQueue queue : metaqQueueSet) { - RocketMQMessageQueue metaqMessageQueue = new RocketMQMessageQueue(queue); - if (isNotDataSplit(metaqMessageQueue.getQueueId())) { - continue; - } + } - messageQueues.add(metaqMessageQueue); + /** + * 用于任务分发的接口, 默认source不会使用该方法 + * + * @param topic 用于分发的topic + * @param consumerIdList 消费者ID列表 + */ + protected void doDispatch(String topic, List consumerIdList) { + } + @Override public List> fetchAllSplits() { + try { + List> messageQueues = new ArrayList<>(); + Set metaqQueueSet = consumer.fetchSubscribeMessageQueues(this.topic); + for (MessageQueue queue : metaqQueueSet) { + RocketMQMessageQueue metaqMessageQueue = new RocketMQMessageQueue(queue); + if (isNotDataSplit(metaqMessageQueue.getQueueId())) { + continue; } + messageQueues.add(metaqMessageQueue); } return messageQueues; } catch (MQClientException e) { @@ -218,8 +231,7 @@ public List getAllSplits() { } } - @Override - public Map> getWorkingSplitsGroupByInstances() { + @Override public Map>> getWorkingSplitsGroupByInstances() { DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt(); defaultMQAdminExt.setVipChannelEnabled(false); defaultMQAdminExt.setAdminExtGroup(UUID.randomUUID().toString()); @@ -227,18 +239,14 @@ public Map> getWorkingSplitsGroupByInstances() { try { defaultMQAdminExt.start(); Map queue2Instances = getMessageQueueAllocationResult(defaultMQAdminExt, this.groupName); - Map> instanceOwnerQueues = new HashMap<>(); + Map>> instanceOwnerQueues = new HashMap<>(); for (MessageQueue messageQueue : queue2Instances.keySet()) { RocketMQMessageQueue metaqMessageQueue = new RocketMQMessageQueue(new MessageQueue(messageQueue.getTopic(), messageQueue.getBrokerName(), messageQueue.getQueueId())); if (isNotDataSplit(metaqMessageQueue.getQueueId())) { continue; } String instanceName = queue2Instances.get(messageQueue); - List splits = instanceOwnerQueues.get(instanceName); - if (splits == null) { - splits = new ArrayList<>(); - instanceOwnerQueues.put(instanceName, splits); - } + List> splits = instanceOwnerQueues.computeIfAbsent(instanceName, k -> new ArrayList<>()); splits.add(metaqMessageQueue); } return instanceOwnerQueues; @@ -250,9 +258,8 @@ public Map> getWorkingSplitsGroupByInstances() { } } - protected Map getMessageQueueAllocationResult(DefaultMQAdminExt defaultMQAdminExt, - String groupName) { - HashMap results = new HashMap(); + protected Map getMessageQueueAllocationResult(DefaultMQAdminExt defaultMQAdminExt, String groupName) { + Map results = new HashMap<>(); try { ConsumerConnection consumerConnection = defaultMQAdminExt.examineConsumerConnectionInfo(groupName); @@ -270,7 +277,7 @@ protected Map getMessageQueueAllocationResult(DefaultMQAdm } } } catch (Exception var11) { - ; + var11.printStackTrace(); } return results; @@ -289,17 +296,14 @@ protected void setOffsetStore(DefaultMQPushConsumer consumer) { MQClientInstance mQClientFactory = MQClientManager.getInstance().getOrCreateMQClientInstance(defaultMQPushConsumer.getDefaultMQPushConsumer()); RemoteBrokerOffsetStore offsetStore = new RemoteBrokerOffsetStore(mQClientFactory, NamespaceUtil.wrapNamespace(consumer.getNamespace(), consumer.getConsumerGroup())) { - @Override - public void removeOffset(MessageQueue mq) { + @Override public void removeOffset(MessageQueue mq) { Set splitIds = new HashSet<>(); splitIds.add(new RocketMQMessageQueue(mq).getQueueId()); removeSplit(splitIds); super.removeOffset(mq); } - @Override - public void updateConsumeOffsetToBroker(MessageQueue mq, long offset, - boolean isOneway) throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + @Override public void updateConsumeOffsetToBroker(MessageQueue mq, long offset, boolean isOneway) throws RemotingException, MQBrokerException, InterruptedException, MQClientException { sendCheckpoint(new RocketMQMessageQueue(mq).getQueueId()); if (DebugWriter.isOpenDebug()) { ConcurrentMap offsetTable = ReflectUtil.getDeclaredField(this, "offsetTable"); @@ -312,48 +316,23 @@ public void updateConsumeOffsetToBroker(MessageQueue mq, long offset, consumer.setOffsetStore(offsetStore);//每个一分钟运行一次 } - @Override protected boolean isNotDataSplit(String queueId) { return queueId.toUpperCase().startsWith("RETRY") || queueId.toUpperCase().startsWith("%RETRY%"); } - @Override - public boolean supportNewSplitFind() { - return true; + @Override public void destroySource() { + if (this.consumer != null) { + this.consumer.shutdown(); + this.consumer = null; + } } - @Override - public boolean supportRemoveSplitFind() { + @Override protected boolean hasListenerSplitChanged() { return true; } - @Override - public boolean supportOffsetRest() { - return false; - } - - public void destroyConsumer() { - List oldConsumers = new ArrayList<>(); - if (consumer != null) { - oldConsumers.add(consumer); - } - try { - for (DefaultMQPushConsumer consumer : oldConsumers) { - consumer.shutdown(); - } - - } catch (Throwable t) { - if (LOG.isWarnEnabled()) { - LOG.warn(t.getMessage(), t); - } - } - - } - - @Override - public void destroy() { - super.destroy(); - destroyConsumer(); + public String getInstanceName() { + return instanceName; } public String getStrategyName() { @@ -407,4 +386,12 @@ public String getConsumerOffset() { public void setConsumerOffset(String consumerOffset) { this.consumerOffset = consumerOffset; } + + public boolean isMessageListenerConcurrently() { + return isMessageListenerConcurrently; + } + + public void setMessageListenerConcurrently(boolean messageListenerConcurrently) { + isMessageListenerConcurrently = messageListenerConcurrently; + } } \ No newline at end of file diff --git a/rocketmq-streams-channel-rocketmq/src/test/java/org/apache/rocketmq/streams/RocketMQChannelTest.java b/rocketmq-streams-channel-rocketmq/src/test/java/org/apache/rocketmq/streams/RocketMQChannelTest.java index cc62a4bf..2e56927f 100644 --- a/rocketmq-streams-channel-rocketmq/src/test/java/org/apache/rocketmq/streams/RocketMQChannelTest.java +++ b/rocketmq-streams-channel-rocketmq/src/test/java/org/apache/rocketmq/streams/RocketMQChannelTest.java @@ -20,21 +20,17 @@ import com.alibaba.fastjson.JSONObject; import java.util.ArrayList; import java.util.Date; -import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.Properties; 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.source.ISource; import org.apache.rocketmq.streams.common.channel.split.ISplit; -import org.apache.rocketmq.streams.common.component.ComponentCreator; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.Message; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; import org.apache.rocketmq.streams.common.utils.DateUtil; -import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.junit.Test; public class RocketMQChannelTest extends AbstractChannelTest { @@ -65,34 +61,11 @@ public void testSink() { @Test public void testGetSplit() { - AbstractSupportShuffleSink sink = (AbstractSupportShuffleSink)createSink(); - List> splits = sink.getSplitList(); + AbstractSupportShuffleSink sink = (AbstractSupportShuffleSink) createSink(); + List> splits = sink.getSplitList(); System.out.println(splits.size()); } - // @Test - // public void testCreateChannel() throws InterruptedException { - // RocketMQChannelBuilder builder = new RocketMQChannelBuilder(); - // IChannel consumer = builder.createChannel(getWindowNameSpace(),getWindowName(), createChannelProperties(), null); - // IChannel producer = builder.createChannel(getWindowNameSpace(),getWindowName(), createChannelProperties(), null); - // consumer.start(new IMessageProcssor() { - // @Override - // public Object doMessage(IMessage message, AbstractContext context) { - // System.out.println(message.getMessageBody().toJSONString()); - // return null; - // } - // });((AbstractBatchMessageChannel)producer).getQueueList(); - // - // List queueList = ((AbstractBatchMessageChannel)producer).getQueueList(); - // for (int i=0; i<100000; i++) { - // - // producer.batchSave(createMsg(queueList.get(0))); - // } - // producer.flush(); - // - // Thread.sleep(10000000l); - // } - public List createMsg() { JSONObject obj = new JSONObject(); obj.put("test", "11111"); @@ -104,41 +77,7 @@ public List createMsg() { } protected Properties createChannelProperties() { - Properties properties = new Properties(); - Iterator> it = ComponentCreator.getProperties().entrySet().iterator(); - while (it.hasNext()) { - Map.Entry entry = it.next(); - String key = (String)entry.getKey(); - String value = (String)entry.getValue(); - if (key.startsWith(WINDOW_TASK_PROPERTY_KEY_PREFIX)) { - String channelKey = key.replace(WINDOW_TASK_PROPERTY_KEY_PREFIX, ""); - properties.put(channelKey, value); - } - - } - String dynamicProperty = properties.getProperty("dynamic.property"); - if (dynamicProperty != null) { - String namespace = this.getWindowNameSpace(); - String name = this.getWindowName(); - String startTime = this.getStartTime(); - String endTime = this.getEndTime(); - String startTimeStr = startTime.replace("-", "").replace(" ", "").replace(":", ""); - String endTimeStr = endTime.replace("-", "").replace(" ", "").replace(":", ""); - String dynamicPropertyValue = MapKeyUtil.createKeyBySign("_", namespace, name, startTimeStr + "", - endTimeStr + ""); - dynamicPropertyValue = dynamicPropertyValue.replaceAll("\\.", "_"); - String[] mutilPropertys = dynamicProperty.split(","); - String groupName = MapKeyUtil.createKeyBySign("_", namespace, name).replaceAll("\\.", "_"); - for (String properyKey : mutilPropertys) { - if (properyKey.equals("group")) { - properties.put(properyKey, groupName); - } else { - properties.put(properyKey, dynamicPropertyValue); - } - - } - } - return properties; + return new Properties(); } public String getWindowNameSpace() { @@ -160,19 +99,13 @@ public String getEndTime() { @Override protected ISource createSource() { RocketMQChannelBuilder builder = new RocketMQChannelBuilder(); - ISource rocketMQSource = builder.createSource(getWindowNameSpace(), getWindowName(), createChannelProperties(), null); - // RocketMQSource rocketMQSource = new RocketMQSource(); - // rocketMQSource.setTopic("TOPIC_DIPPER_WINDOW_STATISTICS"); - // rocketMQSource.setTags("test"); - // rocketMQSource.setAccessKey(); - return rocketMQSource; + return builder.createSource(getWindowNameSpace(), getWindowName(), createChannelProperties(), null); } @Override protected ISink createSink() { RocketMQChannelBuilder builder = new RocketMQChannelBuilder(); - ISink sink = builder.createSink(getWindowNameSpace(), getWindowName(), createChannelProperties(), null); - return sink; + return builder.createSink(getWindowNameSpace(), getWindowName(), createChannelProperties(), null); } } diff --git a/rocketmq-streams-channel-sls/pom.xml b/rocketmq-streams-channel-sls/pom.xml new file mode 100644 index 00000000..55096f98 --- /dev/null +++ b/rocketmq-streams-channel-sls/pom.xml @@ -0,0 +1,51 @@ + + + + rocketmq-streams + org.apache.rocketmq + 2.0.1-SNAPSHOT + + 4.0.0 + + rocketmq-streams-channel-sls + ROCKETMQ STREAMS :: channel-sls + + + com.aliyun.openservices + aliyun-log-producer + + + com.aliyun.openservices + aliyun-log + + + com.google.protobuf + protobuf-java + + + + com.aliyun.openservices + loghub-client-lib + + + org.apache.rocketmq + rocketmq-streams-serviceloader + + + org.apache.rocketmq + rocketmq-streams-sts + + + org.apache.rocketmq + rocketmq-streams-connectors + + + org.apache.rocketmq + rocketmq-streams-schedule + + + + + \ No newline at end of file diff --git a/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/PullSlsChannelBuilder.java b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/PullSlsChannelBuilder.java new file mode 100644 index 00000000..91b17ce5 --- /dev/null +++ b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/PullSlsChannelBuilder.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.sls; + +import com.alibaba.fastjson.JSONObject; +import com.google.auto.service.AutoService; +import java.util.Properties; +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.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; +import org.apache.rocketmq.streams.sls.sink.SLSSink; +import org.apache.rocketmq.streams.sls.source.SLSPullSource; + +@AutoService(IChannelBuilder.class) @ServiceName(value = PullSlsChannelBuilder.TYPE, aliasName = "SLSPullSource") public class PullSlsChannelBuilder extends AbstractSupportShuffleChannelBuilder { + public static final String TYPE = "sls_pull"; + + /** + * @param properties + * @return + */ + @Override public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) { + return (ISource) ConfigurableUtil.create(SLSPullSource.class.getName(), namespace, name, createFormatProperty(properties), null); + } + + @Override public String getType() { + return TYPE; + } + + @Override public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { + return (ISink) ConfigurableUtil.create(SLSSink.class.getName(), namespace, name, createFormatProperty(properties), null); + } + + /** + * 创建标准的属性文件 + * + * @param properties + * @return + */ + @Override + protected JSONObject createFormatProperty(Properties properties) { + JSONObject formatProperties = new JSONObject(); + for (Object object : properties.keySet()) { + String key = (String) object; + if ("type".equals(key)) { + continue; + } + //properties.getProperty 只能获取字符串的值 + formatProperties.put(key, properties.get(key)); + } + IChannelBuilder.formatPropertiesName(formatProperties, properties, "endPoint", "endPoint"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "logStore", "logstore"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "maxThread", "thread.max.count"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "groupName", "group"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "groupName", "consumerGroup"); + return formatProperties; + } + + @Override public ISink createBySource(ISource pipelineSource) { + SLSPullSource source = (SLSPullSource) pipelineSource; + return new SLSSink(source.getEndPoint(), source.getProject(), source.getLogStore(), source.getAccessId(), source.getAccessKey()); + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewChannelBuilder.java b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/SlsChannelBuilder.java similarity index 61% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewChannelBuilder.java rename to rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/SlsChannelBuilder.java index cb644cb6..738c82b1 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewChannelBuilder.java +++ b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/SlsChannelBuilder.java @@ -14,26 +14,30 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - -package org.apache.rocketmq.streams.common.channel.impl.view; +package org.apache.rocketmq.streams.sls; import com.alibaba.fastjson.JSONObject; import com.google.auto.service.AutoService; import java.util.Properties; +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.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; +import org.apache.rocketmq.streams.sls.sink.SLSSink; +import org.apache.rocketmq.streams.sls.source.SLSSource; -@AutoService(IChannelBuilder.class) -@ServiceName(value = ViewChannelBuilder.TYPE, aliasName = "view") -public class ViewChannelBuilder implements IChannelBuilder { - public static final String TYPE = "view"; +@AutoService(IChannelBuilder.class) @ServiceName(value = SlsChannelBuilder.TYPE, aliasName = "SLSSource") public class SlsChannelBuilder extends AbstractSupportShuffleChannelBuilder { + public static final String TYPE = "sls"; + /** + * @param properties + * @return + */ @Override public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) { - return (ViewSource) ConfigurableUtil.create(ViewSource.class.getName(), namespace, name, createFormatProperty(properties), null); + return (ISource) ConfigurableUtil.create(SLSSource.class.getName(), namespace, name, createFormatProperty(properties), null); } @Override public String getType() { @@ -41,7 +45,7 @@ public class ViewChannelBuilder implements IChannelBuilder { } @Override public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { - return (ViewSink) ConfigurableUtil.create(ViewSink.class.getName(), namespace, name, createFormatProperty(properties), null); + return (ISink) ConfigurableUtil.create(SLSSink.class.getName(), namespace, name, createFormatProperty(properties), null); } /** @@ -57,12 +61,19 @@ protected JSONObject createFormatProperty(Properties properties) { if ("type".equals(key)) { continue; } - formatProperties.put(key, properties.getProperty(key)); + //properties.getProperty 只能获取字符串的值 + formatProperties.put(key, properties.get(key)); } - IChannelBuilder.formatPropertiesName(formatProperties, properties, "viewTableName", "tableName"); - IChannelBuilder.formatPropertiesName(formatProperties, properties, "viewTableName", "viewName"); - IChannelBuilder.formatPropertiesName(formatProperties, properties, "viewTableName", "name"); -// IChannelBuilder.formatPropertiesName(formatProperties, properties, "maxThread", "thread.max.count"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "endPoint", "endPoint"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "logStore", "logstore"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "maxThread", "thread.max.count"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "groupName", "group"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "groupName", "consumerGroup"); return formatProperties; } + + @Override public ISink createBySource(ISource pipelineSource) { + SLSSource source = (SLSSource) pipelineSource; + return new SLSSink(source.getEndPoint(), source.getProject(), source.getLogStore(), source.getAccessId(), source.getAccessKey()); + } } diff --git a/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/SlsSearchBuilder.java b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/SlsSearchBuilder.java new file mode 100644 index 00000000..b88b13e1 --- /dev/null +++ b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/SlsSearchBuilder.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.sls; + +import com.alibaba.fastjson.JSONObject; +import com.google.auto.service.AutoService; +import java.util.Properties; +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.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; +import org.apache.rocketmq.streams.sls.source.SLSSearchSource; + +@AutoService(IChannelBuilder.class) +@ServiceName(value = SlsSearchBuilder.TYPE, aliasName = "SLSSearch") +public class SlsSearchBuilder extends AbstractSupportShuffleChannelBuilder { + public static final String TYPE = "sls_search"; + + /** + * @param properties + * @return + */ + @Override + public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) { + return (ISource) ConfigurableUtil.create(SLSSearchSource.class.getName(), namespace, name, createFormatProperty(properties), null); + } + + @Override + public String getType() { + return TYPE; + } + + @Override + public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { + throw new RuntimeException("can not support this method"); + } + + /** + * 创建标准的属性文件 + * + * @param properties + * @return + */ + @Override + protected JSONObject createFormatProperty(Properties properties) { + properties.remove("type"); + 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, "endPoint", "endPoint"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "logStore", "logstore"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "maxThread", "thread.max.count"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "groupName", "group"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "groupName", "consumerGroup"); + return formatProperties; + } + + @Override + public ISink createBySource(ISource pipelineSource) { + throw new RuntimeException("can not support this method"); + } +} diff --git a/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/sink/SLSSink.java b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/sink/SLSSink.java new file mode 100644 index 00000000..08454724 --- /dev/null +++ b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/sink/SLSSink.java @@ -0,0 +1,448 @@ +/* + * 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.sls.sink; + +import com.aliyun.openservices.aliyun.log.producer.LogProducer; +import com.aliyun.openservices.aliyun.log.producer.ProducerConfig; +import com.aliyun.openservices.aliyun.log.producer.ProjectConfig; +import com.aliyun.openservices.aliyun.log.producer.errors.ProducerException; +import com.aliyun.openservices.log.Client; +import com.aliyun.openservices.log.common.LogItem; +import com.aliyun.openservices.log.common.Shard; +import com.aliyun.openservices.log.exception.LogException; +import com.aliyun.openservices.loghub.client.config.LogHubConfig; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +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.annotation.ENVDependence; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.threadpool.ScheduleFactory; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.sls.source.utils.SLSUtil; +import org.apache.rocketmq.streams.sts.StsIdentity; +import org.apache.rocketmq.streams.sts.StsService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 做为metaq的一个消息队列。每次增加一个队列只需要在数据库中增加一条Channel记录即可。 记录中的字端代表了metaq队列的参数 要求必须有无参数构造函数 + */ +public class SLSSink extends AbstractSupportShuffleSink { + /** + * sls shard个数 默认8 + */ + private static final int SHARD_COUNT = 8; + private static final long serialVersionUID = 5429201258366881915L; + private static final Logger LOGGER = LoggerFactory.getLogger(SLSSink.class); + private static final int MAX_LIMIT_COUNT = 4000; + private final transient LogHubConfig.ConsumePosition cursorPosition = LogHubConfig.ConsumePosition.END_CURSOR; + private final transient ReadWriteLock lock = new ReentrantReadWriteLock(); + @ENVDependence public String project; + @ENVDependence public String logStore; + @ENVDependence protected String source; + + @ENVDependence protected String topic; + + protected String logTimeFieldName; + @ENVDependence private String endPoint; + @ENVDependence private String accessId; + @ENVDependence private String accessKey; + @ENVDependence private boolean isSts; + @ENVDependence private String stsRoleArn; + @ENVDependence private String stsAssumeRoleFor; + @ENVDependence private String stsSessionPrefix; + @ENVDependence private int stsExpireSeconds = 86400; + @ENVDependence private String ramEndpoint = "sts-inner.aliyuncs.com"; + private transient StsService stsService; + private transient Client client = null; + // worker 向服务端汇报心跳的时间间隔,单位是毫秒,建议取值 10000ms。 + private long heartBeatIntervalMillis = 10000; + // 是否按序消费 + private boolean consumeInOrder = true; + private transient boolean isFinished = false; // 如果消息被销毁,会通过这个标记停止消息的消费 + //sls数据保存周期,默认7天 + private int ttl = 7; + private transient LogProducer logProducer; + + public SLSSink() { + } + + public SLSSink(String endPoint, String project, String logStore, String accessId, String accessKey) { + this.endPoint = endPoint; + this.project = project; + this.logStore = logStore; + this.accessId = accessId; + this.accessKey = accessKey; + } + + @Override protected boolean initConfigurable() { + if (StringUtil.isEmpty(endPoint) || StringUtil.isEmpty(accessId) || StringUtil.isEmpty(accessKey) || StringUtil.isEmpty(logStore)) { + return false; + } + boolean b = initSts(); + if (!b) { + return false; + } + try { + this.client = createClient(); + } catch (ExecutionException e) { + e.printStackTrace(); + } + + if (this.logProducer == null) { + try { + ProjectConfig projectConfig = createProjectConfig(); + this.logProducer = new LogProducer(new ProducerConfig()); + this.logProducer.putProjectConfig(projectConfig); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + this.startRefreshAsync(); + return super.initConfigurable(); + } + + @Override public int getSplitNum() { + return getSplitList().size(); + } + + @Override public List> getSplitList() { + lock.readLock().lock(); + try { + List shards = client.ListShard(project, logStore).GetShards(); + List> splits = new ArrayList<>(); + for (Shard shard : shards) { + splits.add(new SLSSplit(shard)); + } + return splits; + } catch (LogException ex) { + throw new RuntimeException("Error while listing shards", ex); + } finally { + lock.readLock().unlock(); + } + } + + private boolean initSts() { + if (isSts) { + if (StringUtil.isEmpty(stsRoleArn) || StringUtil.isEmpty(stsAssumeRoleFor)) { + return false; + } + stsService = new StsService(); + stsService.setAccessId(accessId); + stsService.setAccessKey(accessKey); + stsService.setRamEndPoint(ramEndpoint); + stsService.setStsExpireSeconds(stsExpireSeconds); + stsService.setStsSessionPrefix(stsSessionPrefix); + stsService.setRoleArn(stsRoleArn); + stsService.setStsAssumeRoleFor(stsAssumeRoleFor); + } + return true; + } + + protected void startRefreshAsync() { + if (isSts) { + Runnable stsRefreshTask = () -> { + Client c = null; + ProjectConfig projectConfig = null; + try { + c = createClient(); + projectConfig = createProjectConfig(); + } catch (ExecutionException e) { + e.printStackTrace(); + } + lock.writeLock().lock(); + client = c; + lock.writeLock().unlock(); + logProducer.putProjectConfig(projectConfig); + + }; + ScheduleFactory.getInstance().execute(getNameSpace() + "-" + getName() + "-sink_sts_schedule", stsRefreshTask, stsService.getRefreshTimeSecond(), stsService.getRefreshTimeSecond(), TimeUnit.SECONDS); + } + } + + @Override public String getShuffleTopicFieldName() { + return "logStore"; + } + + @Override public void destroy() { + super.destroy(); + if (logProducer != null) { + try { + logProducer.close(); + } catch (InterruptedException | ProducerException e) { + throw new RuntimeException("Sls sink close error", e); + } finally { + logProducer = null; + } + } + ScheduleFactory.getInstance().cancel(getNameSpace() + "-" + getName() + "-sink_sts_schedule"); + } + + @Override public boolean batchInsert(List messages) { + List logItems = new ArrayList<>(); + for (IMessage message : messages) { + LogItem logItem = SLSUtil.createLogItem(message, logTimeFieldName); + logItems.add(logItem); + if (logItems.size() >= MAX_LIMIT_COUNT) { + putLogs(project, logStore, topic, source, logItems); + logItems = new ArrayList<>(); + } + } + if (logItems.size() > 0) { + putLogs(project, logStore, topic, source, logItems); + } + + return true; + } + + @Override protected void createTopicIfNotExist(int splitNum) { + lock.readLock().lock(); + try { + if (!SLSUtil.existProject(client, project)) { + SLSUtil.createProject(client, project); + } + + if (!SLSUtil.existLogstore(client, project, logStore)) { + SLSUtil.createLogstore(client, project, logStore, ttl, splitNum > 0 ? splitNum : SHARD_COUNT); + } + } finally { + lock.readLock().unlock(); + } + if (logProducer == null) { + ProjectConfig projectConfig = null; + try { + projectConfig = createProjectConfig(); + } catch (ExecutionException e) { + e.printStackTrace(); + } + logProducer = new LogProducer(new ProducerConfig()); + logProducer.putProjectConfig(projectConfig); + } + } + + protected ProjectConfig createProjectConfig() throws ExecutionException { + ProjectConfig projectConfig; + if (isSts) { + StsIdentity stsIdentity = stsService.getStsIdentity(); + projectConfig = new ProjectConfig(project, endPoint, stsIdentity.getAccessKeyId(), stsIdentity.getAccessKeySecret(), stsIdentity.getSecurityToken()); + } else { + projectConfig = new ProjectConfig(project, endPoint, accessId, accessKey); + } + return projectConfig; + } + + protected Client createClient() throws ExecutionException { + Client client; + if (isSts) { + StsIdentity stsIdentity = stsService.getStsIdentity(); + client = new Client(endPoint, stsIdentity.getAccessKeyId(), stsIdentity.getAccessKeySecret()); + client.setSecurityToken(stsIdentity.getSecurityToken()); + } else { + client = new Client(endPoint, accessId, accessKey); + } + return client; + } + + private void putLogs(String project, String logStore, String topic, String source, List logItems) { + if (project == null || logStore == null || logItems == null) { + LOGGER.error("HandleLogTask PutLogs error, project, logStore, logItems may be null."); + return; + } + if (logItems.size() > 4096) { + LOGGER.error("HandleLogTask PutLogs error, list too much or list is 0: project:" + project + "logStore:" + logStore + "logItems size:" + logItems.size()); + return; + } + if (logItems.size() == 0) { + return; + } + + try { + if (logProducer != null) { + logProducer.send(getProject(), getLogStore(), topic, source, logItems, result -> { + if (!result.isSuccessful()) { + LOGGER.error(result.getErrorMessage()); + } + }); + } else { + LOGGER.warn("LogProducer_Destroyed_Cannot_Send_Message"); + } + } catch (Exception e) { + LOGGER.error("send sls log error ", e); + } + } + + public LogHubConfig.ConsumePosition getCursorPosition() { + return cursorPosition; + } + + public String getEndPoint() { + return endPoint; + } + + public void setEndPoint(String endPoint) { + this.endPoint = endPoint; + } + + public String getProject() { + return project; + } + + public void setProject(String project) { + this.project = project; + } + + public String getLogStore() { + return logStore; + } + + public void setLogStore(String logStore) { + this.logStore = logStore; + } + + public String getAccessId() { + return accessId; + } + + public void setAccessId(String accessId) { + this.accessId = accessId; + } + + public String getAccessKey() { + return accessKey; + } + + public void setAccessKey(String accessKey) { + this.accessKey = accessKey; + } + + public String getSource() { + return source; + } + + public void setSource(String source) { + this.source = source; + } + + public long getHeartBeatIntervalMillis() { + return heartBeatIntervalMillis; + } + + public void setHeartBeatIntervalMillis(long heartBeatIntervalMillis) { + this.heartBeatIntervalMillis = heartBeatIntervalMillis; + } + + public boolean isConsumeInOrder() { + return consumeInOrder; + } + + public void setConsumeInOrder(boolean consumeInOrder) { + this.consumeInOrder = consumeInOrder; + } + + @Override public boolean isFinished() { + return isFinished; + } + + public void setFinished(boolean finished) { + isFinished = finished; + } + + public String getLogTimeFieldName() { + return logTimeFieldName; + } + + public void setLogTimeFieldName(String logTimeFieldName) { + this.logTimeFieldName = logTimeFieldName; + } + + public int getTtl() { + return ttl; + } + + public void setTtl(int ttl) { + this.ttl = ttl; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public boolean isSts() { + return isSts; + } + + public void setSts(boolean sts) { + isSts = sts; + } + + public String getStsRoleArn() { + return stsRoleArn; + } + + public void setStsRoleArn(String stsRoleArn) { + this.stsRoleArn = stsRoleArn; + } + + public String getStsAssumeRoleFor() { + return stsAssumeRoleFor; + } + + public void setStsAssumeRoleFor(String stsAssumeRoleFor) { + this.stsAssumeRoleFor = stsAssumeRoleFor; + } + + public String getStsSessionPrefix() { + return stsSessionPrefix; + } + + public void setStsSessionPrefix(String stsSessionPrefix) { + this.stsSessionPrefix = stsSessionPrefix; + } + + public int getStsExpireSeconds() { + return stsExpireSeconds; + } + + public void setStsExpireSeconds(int stsExpireSeconds) { + this.stsExpireSeconds = stsExpireSeconds; + } + + public String getRamEndpoint() { + return ramEndpoint; + } + + public void setRamEndpoint(String ramEndpoint) { + this.ramEndpoint = ramEndpoint; + } + + public StsService getStsService() { + return stsService; + } + + public void setStsService(StsService stsService) { + this.stsService = stsService; + } +} \ No newline at end of file diff --git a/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/sink/SLSSplit.java b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/sink/SLSSplit.java new file mode 100644 index 00000000..ddeb3a60 --- /dev/null +++ b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/sink/SLSSplit.java @@ -0,0 +1,122 @@ +/* + * 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.sls.sink; + +import com.alibaba.fastjson.JSONObject; +import com.aliyun.openservices.log.common.Shard; +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; +import org.apache.rocketmq.streams.common.utils.ReflectUtil; + +public class SLSSplit extends BasedConfigurable implements ISplit { + protected transient Shard shard; + protected int shardId = 0; + protected String shardStatus; + protected String inclusiveBeginKey; + protected String exclusiveEndKey; + protected String serverIp = null; + protected int createTime; + + public SLSSplit(Shard shard) { + this.shard = shard; + this.shardId = ReflectUtil.getDeclaredField(Shard.class, shard, "shardId"); + this.shardStatus = shard.getStatus(); + this.inclusiveBeginKey = shard.getInclusiveBeginKey(); + this.exclusiveEndKey = shard.getExclusiveEndKey(); + this.serverIp = shard.getServerIp(); + this.createTime = shard.getCreateTime(); + } + + @Override + public String getQueueId() { + return shardId + ""; + } + + @Override + public Shard getQueue() { + return shard; + } + + @Override + protected void getJsonObject(JSONObject jsonObject) { + super.getJsonObject(jsonObject); + Shard shard = new Shard(shardId, shardStatus, inclusiveBeginKey, exclusiveEndKey, createTime); + shard.setServerIp(serverIp); + this.shard = shard; + } + + @Override + public int compareTo(SLSSplit o) { + return shardId - o.shardId; + } + + public Shard getShard() { + return shard; + } + + public void setShard(Shard shard) { + this.shard = shard; + } + + public int getShardId() { + return shardId; + } + + public void setShardId(int shardId) { + this.shardId = shardId; + } + + public String getShardStatus() { + return shardStatus; + } + + public void setShardStatus(String shardStatus) { + this.shardStatus = shardStatus; + } + + public String getInclusiveBeginKey() { + return inclusiveBeginKey; + } + + public void setInclusiveBeginKey(String inclusiveBeginKey) { + this.inclusiveBeginKey = inclusiveBeginKey; + } + + public String getExclusiveEndKey() { + return exclusiveEndKey; + } + + public void setExclusiveEndKey(String exclusiveEndKey) { + this.exclusiveEndKey = exclusiveEndKey; + } + + public String getServerIp() { + return serverIp; + } + + public void setServerIp(String serverIp) { + this.serverIp = serverIp; + } + + public int getCreateTime() { + return createTime; + } + + public void setCreateTime(int createTime) { + this.createTime = createTime; + } +} diff --git a/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/LogHubProcessor.java b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/LogHubProcessor.java new file mode 100644 index 00000000..34ec60d4 --- /dev/null +++ b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/LogHubProcessor.java @@ -0,0 +1,181 @@ +/* + * 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.sls.source; + +import com.alibaba.fastjson.JSONObject; +import com.aliyun.openservices.log.Client; +import com.aliyun.openservices.log.common.FastLog; +import com.aliyun.openservices.log.common.FastLogContent; +import com.aliyun.openservices.log.common.FastLogGroup; +import com.aliyun.openservices.log.common.LogGroupData; +import com.aliyun.openservices.loghub.client.DefaultLogHubCheckPointTracker; +import com.aliyun.openservices.loghub.client.ILogHubCheckPointTracker; +import com.aliyun.openservices.loghub.client.LogHubClientAdapter; +import com.aliyun.openservices.loghub.client.interfaces.ILogHubProcessor; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.rocketmq.streams.common.channel.source.AbstractSource; +import org.apache.rocketmq.streams.common.context.Message; +import org.apache.rocketmq.streams.common.utils.Base64Utils; +import org.apache.rocketmq.streams.common.utils.IdUtil; +import org.apache.rocketmq.streams.common.utils.ReflectUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class LogHubProcessor implements ILogHubProcessor { + + private static final Logger LOGGER = LoggerFactory.getLogger(LogHubProcessor.class); + private final AbstractSource channel; + private int mShardId; + // 记录上次持久化 check point 的时间 + private long mLastCheckTime = 0; + private int mLastDataTime = 0; + + public LogHubProcessor(AbstractSource channel) { + this.channel = channel; + } + + @Override + public void initialize(int shardId) { + mShardId = shardId; + } + + // 消费数据的主逻辑 + @Override + public String process(List logGroups, ILogHubCheckPointTracker checkPointTracker) { + Long firstOffset = null; + int shardId = ReflectUtil.getDeclaredField(checkPointTracker, "shardID"); + String checkpoint = ReflectUtil.getDeclaredField(checkPointTracker, "cursor"); + if (logGroups.size() > 0) { + this.mShardId = shardId; + firstOffset = createOffset(checkpoint); + AtomicInteger atomicInteger = new AtomicInteger(0); + for (LogGroupData logGroup : logGroups) { + + List msgs = doMessage(logGroup, checkPointTracker); + for (JSONObject jsonObject : msgs) { + Message msg = channel.createMessage(jsonObject, String.valueOf(shardId), String.valueOf(firstOffset), false); + msg.getHeader().addLayerOffset(atomicInteger.incrementAndGet()); + //shardOffset.subOffset.add(msg.getHeader().getOffset()); + msg.getHeader().setOffsetIsLong(true); + channel.executeMessage(msg); + } + } + } + long curTime = System.currentTimeMillis(); + //每200ms调用一次 + // 每隔 60 秒,写一次 check point 到服务端,如果 60 秒内,worker crash, + // 新启动的 worker 会从上一个 checkpoint 其消费数据,有可能有重复数据 + try { + if (curTime - mLastCheckTime > channel.getCheckpointTime()) { + mLastCheckTime = curTime; + LOGGER.info("[{}][{}] Source_Delayed_On_Queue({})_At({})_For({})", IdUtil.instanceId(), this.channel.getName(), shardId, curTime, curTime - mLastDataTime * 1000L); + channel.sendCheckpoint(String.valueOf(shardId)); + checkPointTracker.saveCheckPoint(true); + } else { + checkPointTracker.saveCheckPoint(false); + } + } catch (Exception e) { + e.printStackTrace(); + } + + // 返回空表示正常处理数据, 如果需要回滚到上个 check point 的点进行重试的话,可以 return checkPointTracker.getCheckpoint() + return null; + } + + protected Long createOffset(String checkpoint) { + try { + byte[] bytes = (Base64Utils.decode(checkpoint)); + String offsetStr = new String(bytes); + return Long.valueOf(offsetStr); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + // 当 worker 退出的时候,会调用该函数,用户可以在此处做些清理工作。 + @Override public void shutdown(ILogHubCheckPointTracker checkPointTracker) { + //Integer shardId = ReflectUtil.getBeanFieldOrJsonValue(checkPointTracker, "shardID"); + //channel.sendCheckpoint(shardId+""); + Set shards = new HashSet<>(); + shards.add(String.valueOf(mShardId)); + this.channel.removeSplit(shards); + } + + public List doMessage(LogGroupData logGroup, ILogHubCheckPointTracker checkPointTracker) { + List messageList = new ArrayList<>(); + FastLogGroup flg = logGroup.GetFastLogGroup(); + for (int lIdx = 0; lIdx < flg.getLogsCount(); lIdx++) { + String message = null; + FastLog log = flg.getLogs(lIdx); + this.mLastDataTime = log.getTime(); + if (!channel.getJsonData()) { + if (log.getContentsCount() == 0) { + continue; + } + FastLogContent content = log.getContents(0); + message = content.getValue(); + } else { + JSONObject jsonObject = new JSONObject(); + jsonObject.put("logTime", log.getTime()); + for (int cIdx = 0; cIdx < log.getContentsCount(); cIdx++) { + FastLogContent content = log.getContents(cIdx); + jsonObject.put(content.getKey(), content.getValue()); + } + //为了兼容用户自定义类型,不要删除这句 + message = jsonObject.toJSONString(); + } + Map property = createHeaderProperty(log, flg, checkPointTracker); + JSONObject msg = channel.create(message, property); + messageList.add(msg); + } + return messageList; + } + + private Map createHeaderProperty(FastLog log, FastLogGroup flg, ILogHubCheckPointTracker checkPointTracker) { + try { + + if (channel.getHeaderFieldNames() != null) { + Map property = new HashMap<>(); + property = new HashMap<>(); + property.put("__source__", flg.getSource()); + property.put("__topic__", flg.getTopic()); + property.put("__timestamp__", log.getTime()); + if (checkPointTracker instanceof DefaultLogHubCheckPointTracker) { + DefaultLogHubCheckPointTracker defaultLogHubCheckPointTracker = (DefaultLogHubCheckPointTracker) checkPointTracker; + LogHubClientAdapter logHubClientAdept = ReflectUtil.getDeclaredField(defaultLogHubCheckPointTracker, "loghubClient"); + if (logHubClientAdept != null) { + Client client = ReflectUtil.getDeclaredField(logHubClientAdept, "client"); + if (client != null) { + String hostName = ReflectUtil.getDeclaredField(client, "hostName"); + property.put("__hostname__", hostName); + } + } + } + return property; + } + } catch (Exception e) { + e.printStackTrace(); + } + return null; + } +} \ No newline at end of file diff --git a/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/LogHubProcessorFactory.java b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/LogHubProcessorFactory.java new file mode 100644 index 00000000..1edce587 --- /dev/null +++ b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/LogHubProcessorFactory.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.sls.source; + +import com.aliyun.openservices.loghub.client.interfaces.ILogHubProcessor; +import com.aliyun.openservices.loghub.client.interfaces.ILogHubProcessorFactory; + +public class LogHubProcessorFactory implements ILogHubProcessorFactory { + private SLSSource channel; + + public LogHubProcessorFactory(SLSSource channel) { + this.channel = channel; + } + + @Override + public ILogHubProcessor generatorProcessor() { + // 生成一个消费实例 + return new LogHubProcessor(channel); + } +} \ No newline at end of file diff --git a/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/SLSPullSource.java b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/SLSPullSource.java new file mode 100644 index 00000000..6477499f --- /dev/null +++ b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/SLSPullSource.java @@ -0,0 +1,315 @@ +/* + * 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.sls.source; + +import com.alibaba.fastjson.JSONObject; +import com.aliyun.openservices.log.Client; +import com.aliyun.openservices.log.common.Consts; +import com.aliyun.openservices.log.common.LogGroupData; +import com.aliyun.openservices.log.common.Shard; +import com.aliyun.openservices.log.exception.LogException; +import com.aliyun.openservices.log.request.PullLogsRequest; +import com.aliyun.openservices.log.response.GetCursorResponse; +import com.aliyun.openservices.log.response.GetCursorTimeResponse; +import com.aliyun.openservices.log.response.PullLogsResponse; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +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.MessageOffset; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.connectors.model.PullMessage; +import org.apache.rocketmq.streams.connectors.reader.AbstractSplitReader; +import org.apache.rocketmq.streams.connectors.reader.ISplitReader; +import org.apache.rocketmq.streams.connectors.source.AbstractPullSource; +import org.apache.rocketmq.streams.sls.sink.SLSSplit; +import org.apache.rocketmq.streams.sts.StsIdentity; +import org.apache.rocketmq.streams.sts.StsService; + +public class SLSPullSource extends AbstractPullSource { + + @ENVDependence public String project; + @ENVDependence public String logStore; + @ENVDependence private String endPoint; + @ENVDependence private String accessId; + + @ENVDependence private String accessKey; + + @ENVDependence private boolean isSts; + + @ENVDependence private String stsRoleArn; + + @ENVDependence private String stsAssumeRoleFor; + + @ENVDependence private String stsSessionPrefix; + + @ENVDependence private int stsExpireSeconds = 86400; + + @ENVDependence private String ramEndpoint = "sts-inner.aliyuncs.com"; + + private transient StsService stsService; + private LogHubProcessor logHubProcessor; + private transient Client client;//不建议直接试用,直接试用会对sts场景造成冲突 + + @Override protected boolean initConfigurable() { + initSts(); + if (this.logHubProcessor == null) { + this.logHubProcessor = new LogHubProcessor(this); + } + return super.initConfigurable(); + } + + @Override protected ISplitReader createSplitReader(ISplit split) { + + return new AbstractSplitReader(split) { + private String cursor; + + @Override public void open() { + try { + SLSSplit slsSplit = (SLSSplit) split; + GetCursorResponse cursorResponse = createClient().GetCursor(project, logStore, slsSplit.getShardId(), Consts.CursorMode.END); + cursor = cursorResponse.GetCursor(); + } catch (Exception e) { + throw new RuntimeException("获取游标错误" + "project:" + project + ";logstore:" + logStore + ";endpoint:" + endPoint, e); + } + + } + + @Override public boolean next() { + return true; + } + + @Override public Iterator> getMessage() { + SLSSplit slsSplit = (SLSSplit) split; + List> pullMessages = new ArrayList<>(); + try { + PullLogsRequest request = new PullLogsRequest(project, logStore, slsSplit.getShardId(), pullSize, cursor); + PullLogsResponse response = createClient().pullLogs(request); + // 日志都在日志组(LogGroup)中,按照逻辑拆分即可。 + List logGroups = response.getLogGroups(); + AtomicInteger atomicInteger = new AtomicInteger(0); + + String currentCursor = cursor; + for (LogGroupData logGroup : logGroups) { + + List msgs = logHubProcessor.doMessage(logGroup, null); + for (JSONObject jsonObject : msgs) { + PullMessage pullMessage = new PullMessage<>(); + pullMessage.setMessage(jsonObject); + MessageOffset messageOffset = new MessageOffset(currentCursor); + messageOffset.addLayerOffset(atomicInteger.incrementAndGet()); + pullMessage.setMessageOffset(messageOffset); + pullMessages.add(pullMessage); + } + } + + this.cursor = response.getNextCursor(); + } catch (Exception e) { + throw new RuntimeException("拉取日志错误" + "project:" + project + ";logstore:" + logStore + ";endpoint:" + endPoint, e); + } + return pullMessages.iterator(); + } + + @Override public void seek(String cursor) { + + } + + @Override public long getDelay() { + SLSSplit slsSplit = (SLSSplit) split; + try { + GetCursorTimeResponse response = createClient().GetCursorTime(project, logStore, slsSplit.getShardId(), cursor); + return System.currentTimeMillis() - response.GetCursorTime() * 1000L; + } catch (Exception e) { + throw new RuntimeException("拉取日志错误" + "project:" + project + ";logstore:" + logStore + ";endpoint:" + endPoint, e); + } + } + + @Override public long getFetchedDelay() { + return 0; + } + + @Override public String getCursor() { + return cursor; + } + }; + } + + @Override public List> fetchAllSplits() { + try { + List shards = createClient().ListShard(project, logStore).GetShards(); + List> splits = new ArrayList<>(); + for (Shard shard : shards) { + splits.add(new SLSSplit(shard)); + } + return splits; + } catch (LogException e) { + throw new RuntimeException(e); + } + + } + + public Client createClient() { + try { + Client client; + if (isSts) { + StsIdentity stsIdentity = this.stsService.getStsIdentity(); + client = new Client(endPoint, stsIdentity.getAccessKeyId(), stsIdentity.getAccessKeySecret()); + client.setSecurityToken(stsIdentity.getSecurityToken()); + return client; + } else { + if (this.client != null) { + return this.client; + } + this.client = new Client(endPoint, accessId, accessKey); + return this.client; + } + } catch (Exception e) { + throw new RuntimeException("create sls client error " + "project:" + project + ";logstore:" + logStore + ";endpoint:" + endPoint, e); + } + + } + + public boolean initSts() { + if (isSts) { + if (StringUtil.isEmpty(stsRoleArn) || StringUtil.isEmpty(stsAssumeRoleFor)) { + return false; + } + if (this.stsService == null) { + this.stsService = new StsService(); + this.stsService.setAccessId(accessId); + this.stsService.setAccessKey(accessKey); + this.stsService.setRamEndPoint(ramEndpoint); + this.stsService.setStsExpireSeconds(stsExpireSeconds); + this.stsService.setStsSessionPrefix(stsSessionPrefix); + this.stsService.setRoleArn(stsRoleArn); + this.stsService.setStsAssumeRoleFor(stsAssumeRoleFor); + } + } + return true; + } + + @Override public void destroy() { + super.destroy(); + } + + public String getEndPoint() { + return endPoint; + } + + public void setEndPoint(String endPoint) { + this.endPoint = endPoint; + } + + public String getProject() { + return project; + } + + public void setProject(String project) { + this.project = project; + } + + public String getLogStore() { + return logStore; + } + + public void setLogStore(String logStore) { + this.logStore = logStore; + } + + public String getAccessId() { + return accessId; + } + + public void setAccessId(String accessId) { + this.accessId = accessId; + } + + public String getAccessKey() { + return accessKey; + } + + public void setAccessKey(String accessKey) { + this.accessKey = accessKey; + } + + public boolean isSts() { + return isSts; + } + + public void setSts(boolean sts) { + isSts = sts; + } + + public String getStsRoleArn() { + return stsRoleArn; + } + + public void setStsRoleArn(String stsRoleArn) { + this.stsRoleArn = stsRoleArn; + } + + public String getStsAssumeRoleFor() { + return stsAssumeRoleFor; + } + + public void setStsAssumeRoleFor(String stsAssumeRoleFor) { + this.stsAssumeRoleFor = stsAssumeRoleFor; + } + + public String getStsSessionPrefix() { + return stsSessionPrefix; + } + + public void setStsSessionPrefix(String stsSessionPrefix) { + this.stsSessionPrefix = stsSessionPrefix; + } + + public int getStsExpireSeconds() { + return stsExpireSeconds; + } + + public void setStsExpireSeconds(int stsExpireSeconds) { + this.stsExpireSeconds = stsExpireSeconds; + } + + public String getRamEndpoint() { + return ramEndpoint; + } + + public void setRamEndpoint(String ramEndpoint) { + this.ramEndpoint = ramEndpoint; + } + + public StsService getStsService() { + return stsService; + } + + public void setStsService(StsService stsService) { + this.stsService = stsService; + } + + public Client getClient() { + return client; + } + + public void setClient(Client client) { + this.client = client; + } + +} diff --git a/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/SLSSearchSource.java b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/SLSSearchSource.java new file mode 100644 index 00000000..e1c8a060 --- /dev/null +++ b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/SLSSearchSource.java @@ -0,0 +1,189 @@ +/* + * 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.sls.source; + +import com.alibaba.fastjson.JSONObject; +import com.aliyun.openservices.log.Client; +import com.aliyun.openservices.log.common.LogContent; +import com.aliyun.openservices.log.common.LogItem; +import com.aliyun.openservices.log.common.QueriedLog; +import com.aliyun.openservices.log.exception.LogException; +import com.aliyun.openservices.log.response.GetLogsResponse; +import java.util.ArrayList; +import java.util.Date; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; +import org.apache.rocketmq.streams.common.context.Message; +import org.apache.rocketmq.streams.common.interfaces.IScheduleExecutor; +import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.common.utils.RuntimeUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.schedule.ScheduleComponent; + +public class SLSSearchSource extends SLSSource { + protected Long pollingTimeSecond; + protected String cron;// 定时任务 + protected String slsSQL;//sls sql + protected boolean isStartNow = true;//是否立刻执行一次 + protected int queryTimeSizeSecond = 60 * 10;//查询时间当前时间-offsetGenerator,到当前时间 + protected boolean isPowerSql = true;//是否启用独享版本 + protected Long lastQueryTime = -1L;//最后查询时间 + + protected transient ScheduleComponent scheduleComponent = ScheduleComponent.getInstance(); + protected transient Client client; + + /** + * 模拟offset生成,递增产生 + */ + protected transient AtomicLong offsetGenerator; + + @Override protected boolean initConfigurable() { + offsetGenerator = new AtomicLong(System.currentTimeMillis()); + return super.initConfigurable(); + } + + /** + * 模拟offset生成,递增产生 + */ + @Override public boolean startSource() { + updateLastQueryTime(); + SLSSearchSource source = this; + client = new Client(this.getEndPoint(), getAccessId(), getAccessKey()); + scheduleComponent.getService().startSchedule(new IScheduleExecutor() { + @Override public void doExecute() throws InterruptedException { + executeSQL(slsSQL); + } + + @Override public String getName() { + return source.getName(); + } + + @Override public String getNameSpace() { + return source.getNameSpace(); + } + + @Override public String getType() { + return source.getType(); + } + }, cron, isStartNow); + return false; + } + + protected void updateLastQueryTime() { + + } + + protected void executeSQL(String slsSQL) { + Date now = DateUtil.getCurrentTime(); + Date from = DateUtil.addSecond(now, this.queryTimeSizeSecond); + if (StringUtil.isNotEmpty(this.cron)) { + queryFromSLS(slsSQL, from.getTime(), now.getTime()); + } + + if (this.pollingTimeSecond != null) { + if (this.lastQueryTime == null) { + queryFromSLS(slsSQL, from.getTime(), now.getTime()); + } else { + //如果系统挂掉,可以断点续传的方式执行查询 + Date virtualNow = DateUtil.addSecond(new Date(this.lastQueryTime), this.pollingTimeSecond.intValue()); + Date virtualFrom = DateUtil.addSecond(virtualNow, this.queryTimeSizeSecond); + while (Math.abs(virtualNow.getTime() - now.getTime()) < 1000) { + queryFromSLS(slsSQL, virtualFrom.getTime(), virtualNow.getTime()); + virtualNow = DateUtil.addSecond(virtualNow, pollingTimeSecond.intValue()); + virtualFrom = DateUtil.addSecond(virtualNow, this.queryTimeSizeSecond); + } + } + + } + } + + protected void queryFromSLS(String sql, Long startTime, Long toTime) { + try { + String queueId = RuntimeUtil.getDipperInstanceId(); + GetLogsResponse logsResponse = client.executeLogstoreSql(project, logStore, startTime.intValue(), toTime.intValue(), sql, isPowerSql); + List rows = new ArrayList<>(); + for (QueriedLog log : logsResponse.getLogs()) { + LogItem item = log.GetLogItem(); + for (LogContent content : item.mContents) { + JSONObject row = new JSONObject(); + row.put(content.mKey, content.mValue); + rows.add(row); + } + } + if (rows != null) { + for (JSONObject msg : rows) { + doReceiveMessage(msg, false, queueId, offsetGenerator.incrementAndGet() + ""); + } + sendCheckpoint(queueId); + executeMessage((Message) BatchFinishMessage.create()); + } + this.lastQueryTime = System.currentTimeMillis(); + } catch (LogException e) { + e.printStackTrace(); + throw new RuntimeException("execute sls sql error " + sql, e); + } + } + + public String getCron() { + return cron; + } + + public void setCron(String cron) { + this.cron = cron; + } + + public String getSlsSQL() { + return slsSQL; + } + + public void setSlsSQL(String slsSQL) { + this.slsSQL = slsSQL; + } + + public boolean isStartNow() { + return isStartNow; + } + + public void setStartNow(boolean startNow) { + isStartNow = startNow; + } + + public int getQueryTimeSizeSecond() { + return queryTimeSizeSecond; + } + + public void setQueryTimeSizeSecond(int queryTimeSizeSecond) { + this.queryTimeSizeSecond = queryTimeSizeSecond; + } + + public boolean isPowerSql() { + return isPowerSql; + } + + public void setPowerSql(boolean powerSql) { + isPowerSql = powerSql; + } + + public Long getLastQueryTime() { + return lastQueryTime; + } + + public void setLastQueryTime(Long lastQueryTime) { + this.lastQueryTime = lastQueryTime; + } +} diff --git a/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/SLSSource.java b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/SLSSource.java new file mode 100644 index 00000000..97ea8b65 --- /dev/null +++ b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/SLSSource.java @@ -0,0 +1,400 @@ +/* + * 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.sls.source; + +import com.aliyun.openservices.log.Client; +import com.aliyun.openservices.log.common.ConsumerGroup; +import com.aliyun.openservices.log.common.Shard; +import com.aliyun.openservices.log.exception.LogException; +import com.aliyun.openservices.log.response.ListConsumerGroupResponse; +import com.aliyun.openservices.loghub.client.ClientWorker; +import com.aliyun.openservices.loghub.client.config.LogHubConfig; +import com.google.common.collect.Lists; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import org.apache.rocketmq.streams.common.channel.source.AbstractPushSource; +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.threadpool.ScheduleFactory; +import org.apache.rocketmq.streams.common.utils.IdUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.sls.sink.SLSSplit; +import org.apache.rocketmq.streams.sts.StsIdentity; +import org.apache.rocketmq.streams.sts.StsService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 做为metaq的一个消息队列。每次增加一个队列只需要在数据库中增加一条Channel记录即可。 记录中的字端代表了metaq队列的参数 要求必须有无参数构造函数 + */ +public class SLSSource extends AbstractPushSource { + + private static final Logger LOGGER = LoggerFactory.getLogger(SLSSource.class); + + private static final long serialVersionUID = 5429201258366881915L; + private final transient LogHubConfig.ConsumePosition cursorPosition = LogHubConfig.ConsumePosition.END_CURSOR; + + @ENVDependence public String project; + + @ENVDependence public String logStore; + @ENVDependence private String endPoint; + @ENVDependence private String accessId; + @ENVDependence private String accessKey; + @ENVDependence private boolean isSts; + @ENVDependence private String stsRoleArn; + @ENVDependence private String stsAssumeRoleFor; + @ENVDependence private String stsSessionPrefix; + @ENVDependence private int stsExpireSeconds = 86400; + @ENVDependence private String ramEndpoint = "sts-inner.aliyuncs.com"; + private transient StsService stsService; + private transient ClientWorker clientWorker; + // worker 向服务端汇报心跳的时间间隔,单位是毫秒,建议取值 10000ms。 + private long heartBeatIntervalMillis = 10000; + + // 是否按序消费 + private boolean consumeInOrder = true; + + private transient volatile boolean isFinished = false; // 如果消息被销毁,会通过这个标记停止消息的消费 + + //sls数据保存周期,默认7天 + private int ttl = 7; + + //sls shard个数 默认8 + private int shardCount = 8; + + public SLSSource() { + } + + public SLSSource(String endPoint, String project, String logStore, String accessId, String accessKey, String groupName) { + this.endPoint = endPoint; + this.project = project; + this.logStore = logStore; + this.accessId = accessId; + this.accessKey = accessKey; + this.groupName = groupName; + } + + public static void main(String[] args) { + System.out.println(Runtime.getRuntime().availableProcessors()); + } + + public static StsIdentity getIdentity() { + return null; + } + + @Override protected boolean initConfigurable() { + if (StringUtil.isEmpty(endPoint) || StringUtil.isEmpty(accessId) || StringUtil.isEmpty(accessKey) || StringUtil.isEmpty(logStore)) { + return false; + } + return super.initConfigurable(); + } + + private boolean initSts() { + if (isSts) { + if (StringUtil.isEmpty(stsRoleArn) || StringUtil.isEmpty(stsAssumeRoleFor)) { + return false; + } + if (this.stsService == null) { + this.stsService = new StsService(); + this.stsService.setAccessId(accessId); + this.stsService.setAccessKey(accessKey); + this.stsService.setRamEndPoint(ramEndpoint); + this.stsService.setStsExpireSeconds(stsExpireSeconds); + this.stsService.setStsSessionPrefix(stsSessionPrefix); + this.stsService.setRoleArn(stsRoleArn); + this.stsService.setStsAssumeRoleFor(stsAssumeRoleFor); + } + } + return true; + } + + @Override public boolean startSource() { + initSts(); + startWork(); + startStsRefreshAsync(); + return true; + } + + protected LogHubConfig createLogHubConfig(String accessId, String accessKey, String groupName, String endPoint, String project, String logStore) throws ExecutionException { + String localAccessId = accessId; + String localAccessKey = accessKey; + LogHubConfig config; + if (isSts) { + StsIdentity stsIdentity = this.stsService.getStsIdentity(); + localAccessId = stsIdentity.getAccessKeyId(); + localAccessKey = stsIdentity.getAccessKeySecret(); + String localToken = stsIdentity.getSecurityToken(); + config = new LogHubConfig(groupName, UUID.randomUUID().toString(), endPoint, project, logStore, localAccessId, localAccessKey, cursorPosition); + config.setStsToken(localToken); + } else { + config = new LogHubConfig(groupName, UUID.randomUUID().toString(), endPoint, project, logStore, localAccessId, localAccessKey, cursorPosition); + } + config.setMaxFetchLogGroupSize(getMaxFetchLogGroupSize()); + LOGGER.info("[{}][{}] SLSSource_IsSts({})", IdUtil.instanceId(), getName(), isSts); + return config; + } + + protected Client createClient() throws ExecutionException { + Client client; + if (isSts) { + StsIdentity stsIdentity = this.stsService.getStsIdentity(); + client = new Client(endPoint, stsIdentity.getAccessKeyId(), stsIdentity.getAccessKeySecret()); + client.setSecurityToken(stsIdentity.getSecurityToken()); + } else { + client = new Client(endPoint, accessId, accessKey); + } + return client; + } + + @Override public List> fetchAllSplits() { + try { + List shards = createClient().ListShard(project, logStore).GetShards(); + List> splits = new ArrayList<>(); + for (Shard shard : shards) { + splits.add(new SLSSplit(shard)); + } + return splits; + } catch (ExecutionException ex) { + throw new RuntimeException("Error while listing shards", ex); + } catch (LogException e) { + throw new RuntimeException(e); + } + } + + protected void startStsRefreshAsync() { + try { + if (isSts) { + Runnable stsRefreshTask = () -> { + try { + StsIdentity stsIdentity = stsService.getStsIdentity(); + while (clientWorker == null) { + Thread.sleep(100); + } + clientWorker.SwitchClient(stsIdentity.getAccessKeyId(), stsIdentity.getAccessKeySecret(), stsIdentity.getSecurityToken()); + } catch (ExecutionException | InterruptedException e) { + e.printStackTrace(); + } + }; + + ScheduleFactory.getInstance().execute(getNameSpace() + "-" + getName() + "-source_sts_schedule", stsRefreshTask, 0, stsService.getRefreshTimeSecond(), TimeUnit.SECONDS); + } + } catch (Exception e) { + throw new RuntimeException("Start_Sts_Refresh_Sync_Error ", e); + } + } + + protected void startWork() { + try { + String runtimeEnv = this.getConfiguration().getProperty(ConfigurationKey.RUNTIME_ENV); + if (runtimeEnv == null) { + runtimeEnv = "default"; + } + LogHubConfig config = createLogHubConfig(this.accessId, this.accessKey, this.groupName + "_" + runtimeEnv, this.endPoint, this.project, this.logStore); + if (this.clientWorker == null) { + this.clientWorker = new ClientWorker(new LogHubProcessorFactory(this), config); + } + } catch (Exception e) { + LOGGER.error("[{}][{}] Start_Sls_Channel_Error: {}|{}|{}", IdUtil.instanceId(), getName(), project, logStore, groupName, e); + try { + List existConsumerGroup = Lists.newArrayList(); + ListConsumerGroupResponse groupResponse = createClient().ListConsumerGroup(project, logStore); + List consumerGroups = groupResponse.GetConsumerGroups(); + if (consumerGroups != null) { + for (ConsumerGroup consumerGroup : consumerGroups) { + String consumerGroupName = consumerGroup.getConsumerGroupName(); + LOGGER.info("[{}][{}] List_Consumer_Group_Success:{}|{}|{}", IdUtil.instanceId(), getName(), project, logStore, consumerGroupName); + if (consumerGroupName.length() == 15) { //siem期初生成的consumerGroup + existConsumerGroup.add(consumerGroupName); + } + } + } + + if (e.getMessage().contains("consumer group quota exceed")) { + if (!existConsumerGroup.isEmpty()) { + Collections.sort(existConsumerGroup); + String newConsumerGroup = existConsumerGroup.get(0); + LogHubConfig config = createLogHubConfig(this.accessId, this.accessKey, newConsumerGroup, this.endPoint, this.project, this.logStore); + if (this.clientWorker == null) { + this.clientWorker = new ClientWorker(new LogHubProcessorFactory(this), config); + } + LOGGER.info("[{}][{}] Consumer_Group_Fixed_To:{}|{}|{}", IdUtil.instanceId(), getName(), project, logStore, newConsumerGroup); + } + } + + } catch (Exception ex) { + LOGGER.error("[{}][{}] List_Consumer_Group_Error:{}|{}", IdUtil.instanceId(), getName(), project, logStore, ex); + } + } + if (this.clientWorker != null) { + Thread thread = new Thread(this.clientWorker); + thread.start(); + } else { + LOGGER.error("[{}][{}] Start_Sls_Error_Client_Worker_Is_Null: {}|{}|{}", IdUtil.instanceId(), getName(), project, logStore, groupName); + } + } + + @Override public void destroySource() { + isFinished = true; + if (this.clientWorker != null) { + this.clientWorker.shutdown(); + this.clientWorker = null; + } + ScheduleFactory.getInstance().cancel(getNameSpace() + "-" + getName() + "-source_sts_schedule"); + } + + public LogHubConfig.ConsumePosition getCursorPosition() { + return cursorPosition; + } + + public String getEndPoint() { + return endPoint; + } + + public void setEndPoint(String endPoint) { + this.endPoint = endPoint; + } + + public String getProject() { + return project; + } + + public void setProject(String project) { + this.project = project; + } + + public String getLogStore() { + return logStore; + } + + public void setLogStore(String logStore) { + this.logStore = logStore; + } + + public String getAccessId() { + return accessId; + } + + public void setAccessId(String accessId) { + this.accessId = accessId; + } + + public String getAccessKey() { + return accessKey; + } + + public void setAccessKey(String accessKey) { + this.accessKey = accessKey; + } + + public long getHeartBeatIntervalMillis() { + return heartBeatIntervalMillis; + } + + public void setHeartBeatIntervalMillis(long heartBeatIntervalMillis) { + this.heartBeatIntervalMillis = heartBeatIntervalMillis; + } + + public boolean isConsumeInOrder() { + return consumeInOrder; + } + + public void setConsumeInOrder(boolean consumeInOrder) { + this.consumeInOrder = consumeInOrder; + } + + @Override public boolean isFinished() { + return isFinished; + } + + public void setFinished(boolean finished) { + isFinished = finished; + } + + public int getTtl() { + return ttl; + } + + public void setTtl(int ttl) { + this.ttl = ttl; + } + + public int getShardCount() { + return shardCount; + } + + public void setShardCount(int shardCount) { + this.shardCount = shardCount; + } + + public boolean isSts() { + return isSts; + } + + public void setSts(boolean sts) { + isSts = sts; + } + + public String getStsRoleArn() { + return stsRoleArn; + } + + public void setStsRoleArn(String stsRoleArn) { + this.stsRoleArn = stsRoleArn; + } + + public String getStsAssumeRoleFor() { + return stsAssumeRoleFor; + } + + public void setStsAssumeRoleFor(String stsAssumeRoleFor) { + this.stsAssumeRoleFor = stsAssumeRoleFor; + } + + public String getStsSessionPrefix() { + return stsSessionPrefix; + } + + public void setStsSessionPrefix(String stsSessionPrefix) { + this.stsSessionPrefix = stsSessionPrefix; + } + + public int getStsExpireSeconds() { + return stsExpireSeconds; + } + + public void setStsExpireSeconds(int stsExpireSeconds) { + this.stsExpireSeconds = stsExpireSeconds; + } + + public String getRamEndpoint() { + return ramEndpoint; + } + + public void setRamEndpoint(String ramEndpoint) { + this.ramEndpoint = ramEndpoint; + } + + @Override protected boolean hasListenerSplitChanged() { + return true; + } + +} \ No newline at end of file diff --git a/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/sts/StsClientWorker.java b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/sts/StsClientWorker.java new file mode 100644 index 00000000..8f149c82 --- /dev/null +++ b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/sts/StsClientWorker.java @@ -0,0 +1,60 @@ +/* + * 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.sls.source.sts; + +import com.aliyun.openservices.loghub.client.ClientWorker; +import com.aliyun.openservices.loghub.client.config.LogHubConfig; +import com.aliyun.openservices.loghub.client.exceptions.LogHubClientWorkerException; +import com.aliyun.openservices.loghub.client.interfaces.ILogHubProcessorFactory; + +import java.util.concurrent.ExecutorService; + +import org.apache.rocketmq.streams.sts.StsService; + +public class StsClientWorker extends ClientWorker { + + transient StsService stsService; + + public StsClientWorker(ILogHubProcessorFactory factory, LogHubConfig config) throws LogHubClientWorkerException { + super(factory, config); + } + + public StsClientWorker(ILogHubProcessorFactory factory, LogHubConfig config, ExecutorService service) throws LogHubClientWorkerException { + super(factory, config, service); + } + + public void setStsService(StsService stsService) { + this.stsService = stsService; + } + + @Override + public void run() { + try { + super.run(); + } catch (Exception e) { + try { + Thread.sleep(1000); + } catch (InterruptedException exception) { + exception.printStackTrace(); + } + super.SwitchClient(null, null, null); + super.run(); + + } + } + +} diff --git a/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/utils/SLSUtil.java b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/utils/SLSUtil.java new file mode 100644 index 00000000..099b64c8 --- /dev/null +++ b/rocketmq-streams-channel-sls/src/main/java/org/apache/rocketmq/streams/sls/source/utils/SLSUtil.java @@ -0,0 +1,172 @@ +/* + * 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.sls.source.utils; + +import com.aliyun.openservices.log.Client; +import com.aliyun.openservices.log.common.LogItem; +import com.aliyun.openservices.log.common.LogStore; +import com.aliyun.openservices.log.common.Shard; +import com.aliyun.openservices.log.exception.LogException; +import com.aliyun.openservices.log.response.GetCursorTimeResponse; +import com.aliyun.openservices.log.response.GetLogStoreResponse; +import com.aliyun.openservices.log.response.GetProjectResponse; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.rocketmq.streams.common.channel.source.ISource; +import org.apache.rocketmq.streams.common.channel.source.SplitProgress; +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SLSUtil { + private static final Logger LOGGER = LoggerFactory.getLogger(SLSUtil.class); + + public static LogItem createLogItem(IMessage msg, String logTimeFieldName) { + Map fieldName2Value = null; + Object object = msg.getMessageValue(); + if (!(object instanceof Map)) { + throw new RuntimeException("can not support this class ,expect Map, actual is " + object.getClass()); + } else { + fieldName2Value = (Map) object; + } + LogItem logItem = null; + String itemTime = ""; + if (StringUtil.isNotEmpty(logTimeFieldName)) { + Object start_time = fieldName2Value.get(logTimeFieldName); + itemTime = (String) start_time; + } else { + // LOG.error("logTimeFieldName is null, maybe can not search by time "); + } + //TODO delete it + if (fieldName2Value != null) { + fieldName2Value.put("ability_source", "sql-engine"); + } + if (!"".equals(itemTime)) { + // 创建logItem + try { + int logTime2 = (int) (DateUtil.StringToLong(itemTime) / 1000L); + if (logTime2 == 0L) {// 时间转换出错,默认当前时间 + // 默认当前时间 + logItem = new LogItem(); + } else { + logItem = new LogItem(logTime2); + } + } catch (Exception e) { + // 默认当前时间 + logItem = new LogItem(); + } + } else { + logItem = new LogItem(); + } + + Iterator> it = fieldName2Value.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry entry = it.next(); + String value = entry.getValue() == null ? "" : String.valueOf(entry.getValue()); + if (value == null) { + // LOG.error("SLSDataSource value is null"); + value = ""; + } + + logItem.PushBack(entry.getKey(), value); + } + return logItem; + } + + public static boolean existProject(Client client, String project) { + try { + GetProjectResponse response = client.GetProject(project); + return response != null; + } catch (LogException e) { + LOGGER.error("existProject is error", e); + return false; + } + } + + public static void createProject(Client client, String project) { + try { + client.CreateProject(project, ""); + } catch (LogException e) { + LOGGER.error("createProject is error", e); + throw new RuntimeException(e); + } + } + + public static boolean existLogstore(Client client, String project, String logstore) { + try { + GetLogStoreResponse response = client.GetLogStore(project, logstore); + return response != null && response.GetLogStore() != null; + } catch (LogException e) { + LOGGER.error("existLogstore is error", e); + return false; + } + } + + public static void createLogstore(Client client, String project, String logstore, int ttl, int shardCount) { + try { + LogStore logStore = new LogStore(logstore, ttl, shardCount); + client.CreateLogStore(project, logStore); + } catch (LogException e) { + LOGGER.error("createLogstore is error", e); + throw new RuntimeException(e); + } + } + + /** + * 获取分片延迟 + * + * @param source + * @param shardId2Cursor + * @param client + * @param project + * @param logStore + * @return + */ + public static List getSplitProgress(ISource source, Map shardId2Cursor, Client client, String project, String logStore) { + try { + List> splits = source.fetchAllSplits(); + List splitProgresses = new ArrayList<>(); + if (splits == null) { + return null; + } + Map shardId2Splits = new HashMap<>(); + for (ISplit split : splits) { + Shard shard = (Shard) split.getQueue(); + shardId2Splits.put(shard.getShardId() + "", split); + } + for (String shardId : shardId2Cursor.keySet()) { + ISplit split = shardId2Splits.get(shardId); + Shard shard = (Shard) split.getQueue(); + String cursor = shardId2Cursor.get(shardId); + GetCursorTimeResponse response = client.GetCursorTime(project, logStore, shard.getShardId(), cursor); + if (response != null) { + SplitProgress splitProgress = new SplitProgress(split.getQueueId(), response.GetCursorTime(), true); + splitProgresses.add(splitProgress); + } + } + return splitProgresses; + } catch (Exception e) { + throw new RuntimeException("Error while listing shards", e); + } + } +} diff --git a/rocketmq-streams-channel-syslog/pom.xml b/rocketmq-streams-channel-syslog/pom.xml index f52ceabf..b2130c21 100755 --- a/rocketmq-streams-channel-syslog/pom.xml +++ b/rocketmq-streams-channel-syslog/pom.xml @@ -1,13 +1,13 @@ - 4.0.0 org.apache.rocketmq rocketmq-streams - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT rocketmq-streams-channel-syslog ROCKETMQ STREAMS :: channel-syslog @@ -26,8 +26,8 @@ commons-collections - commons-lang - commons-lang + org.apache.commons + commons-lang3 diff --git a/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannel.java b/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannel.java index 7869244c..50d67190 100644 --- a/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannel.java +++ b/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannel.java @@ -19,13 +19,12 @@ import com.alibaba.fastjson.JSONObject; import java.net.URLDecoder; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.lang.StringUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import org.apache.commons.lang3.StringUtils; 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; @@ -40,9 +39,11 @@ import org.apache.rocketmq.streams.common.utils.StringUtil; import org.graylog2.syslog4j.Syslog; import org.graylog2.syslog4j.SyslogConfigIF; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class SyslogChannel extends AbstractChannel implements ISyslogRouter { - private static final Log LOG = LogFactory.getLog(SyslogChannel.class); + private static final Logger LOGGER = LoggerFactory.getLogger(SyslogChannel.class); protected transient List ipList = new ArrayList<>(); //消息发送源的ip或域名列表,支持ip范围-192.168.1.1-192.168.1.10,ip端-192.168.0.0/22,和正则表达式 protected String protol = SyslogChannelManager.UDP;//syslog支持的协议 @@ -74,7 +75,7 @@ public SyslogChannel(String serverIp, int port) { @Override protected ISink createSink() { - this.sink= new AbstractSink() { + this.sink = new AbstractSink() { @Override protected boolean batchInsert(List messages) { if (messages == null || !syslogClientInit) { @@ -98,13 +99,13 @@ protected boolean batchInsert(List messages) { syslogClient.getConfig().setLocalName(IPUtil.getLocalIP()); syslogClient.getConfig().setSendLocalTimestamp(true); syslogClient.getConfig().setSendLocalName(true);//如果这个值是false,需要确保json数据无空格 - if("127.0.0.1".equals(syslogClient.getConfig().getHost())){ + if ("127.0.0.1".equals(syslogClient.getConfig().getHost())) { //本机测试必须设置,否则ip地址变成了127.0.0.1,如果是远端server,必须注释掉这一行,否则server发生覆盖 syslogClient.getConfig().setHost(IPUtil.getLocalIP()); } } catch (Exception e) { - LOG.error("syslogClient decode message error " + msg.getMessageValue().toString(), e); + SyslogChannel.LOGGER.error("syslogClient decode message error " + msg.getMessageValue().toString(), e); } syslogClient.log(level, message); } @@ -117,14 +118,18 @@ protected boolean batchInsert(List messages) { @Override protected ISource createSource() { - this.source=new AbstractUnreliableSource() { + this.source = new AbstractUnreliableSource() { @Override protected boolean startSource() { SyslogChannelManager.start(protol); return true; } + + @Override protected void destroySource() { + + } }; - return this.source; + return this.source; } @Override @@ -145,9 +150,9 @@ protected boolean initConfigurable() { syslogClient = Syslog.getInstance(protol); SyslogConfigIF config = syslogClient.getConfig(); config.setHost(serverIp); - config.setPort((protol .equals( SyslogChannelManager.UDP) )? SyslogChannelManager.udpPort : SyslogChannelManager.tcpPort); + config.setPort((protol.equals(SyslogChannelManager.UDP)) ? SyslogChannelManager.udpPort : SyslogChannelManager.tcpPort); } catch (Throwable throwable) { - LOG.warn("syslogClient client init fail " + throwable); + LOGGER.warn("syslogClient client init fail " + throwable); syslogClientInit = false; } @@ -208,16 +213,13 @@ public void setSyslogClientInit(boolean syslogClientInit) { @Override public void destroy() { super.destroy(); - isDestroy = true; } public void addIps(String... ips) { if (ips == null) { return; } - for (String ip : ips) { - this.ipList.add(ip); - } + this.ipList.addAll(Arrays.asList(ips)); } public String getProtol() { @@ -316,7 +318,7 @@ public boolean flushMessage(List messages) { } @Override public String createCheckPointName() { - return getConfigureName(); + return getName(); } public String getIpListStr() { diff --git a/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannelManager.java b/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannelManager.java index 0e1be82a..d12e3d36 100644 --- a/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannelManager.java +++ b/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannelManager.java @@ -17,7 +17,8 @@ package org.apache.rocketmq.streams.syslog; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.configuration.SystemContext; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.graylog2.syslog4j.SyslogConstants; @@ -25,11 +26,8 @@ public class SyslogChannelManager { public static final String TCP = SyslogConstants.TCP;//tcp协议名称 public static final String UDP = SyslogConstants.UDP;//tcp协议名称 - public static final String TCP_PORT_PROPERTY_KEY = "dipper.syslog.server.tcp.port";//当需要改变端口值时,通过配置文件增加dipper.syslog.server.tcp.port=新端口的值 - public static final String UDP_PORT_PROPERTY_KEY = "dipper.syslog.server.udp.port";//当需要改变端口值时,通过配置文件增加dipper.syslog.server.tcp.port=新端口的值 - - public static int tcpPort = 12345;//syslog server默认端口 - public static int udpPort = 12346;//syslog server默认端口 + public static int tcpPort = 12345;//syslog server默认端口 + public static int udpPort = 12346;//syslog server默认端口 private static AtomicBoolean tcpStart = new AtomicBoolean(false);//标记是否启动tcp server,只会启动一次 private static AtomicBoolean updStart = new AtomicBoolean(false);//标记是否启动udp server,只会启动一次 @@ -39,7 +37,7 @@ public class SyslogChannelManager { public static void registeTCP(SyslogChannel syslogRouter) { if (!TCP_CHANNEL.getRouters().contains(syslogRouter)) { TCP_CHANNEL.getRouters().add(syslogRouter); - if (tcpPort == 12345&&syslogRouter.getPort()>0) { + if (tcpPort == 12345 && syslogRouter.getPort() > 0) { tcpPort = syslogRouter.getPort(); } } @@ -73,7 +71,7 @@ public static void start(String protol) { protected static void startTCPServer() { if (tcpStart.compareAndSet(false, true)) { - String value = ComponentCreator.getProperties().getProperty(TCP_PORT_PROPERTY_KEY); + String value = SystemContext.getProperty(ConfigurationKey.SYSLOG_TCP_PORT_PROPERTY_KEY); if (StringUtil.isNotEmpty(value)) { tcpPort = Integer.valueOf(value); } @@ -87,7 +85,7 @@ protected static void startTCPServer() { protected static void startUDPServer() { if (updStart.compareAndSet(false, true)) { - String value = ComponentCreator.getProperties().getProperty(UDP_PORT_PROPERTY_KEY); + String value = SystemContext.getProperty(ConfigurationKey.SYSLOG_UDP_PORT_PROPERTY_KEY); if (StringUtil.isNotEmpty(value)) { udpPort = Integer.valueOf(value); } diff --git a/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogParser.java b/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogParser.java index 47968e85..42d9de81 100644 --- a/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogParser.java +++ b/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogParser.java @@ -82,14 +82,6 @@ public static String parseTags(String message) { if (index == -1) { return null; } - String tags = message.substring(0, index); - return tags; - } - - public static void main(String[] args) { - String dateFormat = "HH:mm:ss"; - String dateStr = DateUtil.format(new Date(), dateFormat); - - System.out.println(parseHost(dateStr)); + return message.substring(0, index); } } diff --git a/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogServer.java b/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogServer.java index b7920527..587d5e93 100644 --- a/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogServer.java +++ b/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogServer.java @@ -22,14 +22,11 @@ import java.util.ArrayList; import java.util.Date; import java.util.List; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.cache.softreference.impl.SoftReferenceCache; import org.apache.rocketmq.streams.common.channel.source.AbstractUnreliableSource; import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.context.UserDefinedMessage; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; import org.apache.rocketmq.streams.common.utils.DateUtil; import org.apache.rocketmq.streams.common.utils.IPUtil; @@ -39,27 +36,26 @@ import org.graylog2.syslog4j.server.SyslogServerIF; import org.graylog2.syslog4j.server.SyslogServerSessionEventHandlerIF; import org.graylog2.syslog4j.server.impl.net.tcp.TCPNetSyslogServerConfigIF; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class SyslogServer extends AbstractUnreliableSource { - private static final Log LOG = LogFactory.getLog(SyslogServer.class); + private static final Logger LOGGER = LoggerFactory.getLogger(SyslogServer.class); + private static final int DEFAULT_TIMEOUT = 10000; //毫秒,只有tcp时有用 + private static final String MSG_ID_NULL = "MSG_ID_NULL"; + private static final String PREFIX = "dipper.upgrade.channel.syslog.envkey"; + protected int timeout; @ENVDependence private String serverHost = IPUtil.getLocalAddress(); @ENVDependence private String serverPort; private String protocol; private Integer ctimeout = DEFAULT_TIMEOUT; - private transient SyslogServerIF syslogServer; - - private static final int DEFAULT_TIMEOUT = 10000; //毫秒,只有tcp时有用 - - private static final String MSG_ID_NULL = "MSG_ID_NULL"; - private volatile boolean isFinished = false; // 如果消息被销毁,会通过这个标记停止消息的消费 - - protected int timeout; /** * 注册路由信息,由主服务做路由分发 */ private transient List routers = new ArrayList<>(); + private volatile transient SoftReferenceCache> cache = new SoftReferenceCache<>(); @Override protected boolean initConfigurable() { @@ -89,11 +85,11 @@ public class SyslogServer extends AbstractUnreliableSource { setReceiver(new IStreamOperator() { @Override public Object doMessage(IMessage message, AbstractContext context) { String hostAddress = message.getMessageBody().getString("hostAddress"); - if(hostAddress==null){ + if (hostAddress == null) { return null; } List syslogChannels = cache.get(hostAddress); - LOG.info("receive syslog msg, ip is " + hostAddress + " msg is " + message.getMessageBody()); + LOGGER.info("receive syslog msg, ip is " + hostAddress + " msg is " + message.getMessageBody()); boolean hasMatch = false; if (syslogChannels == null) { syslogChannels = new ArrayList<>(); @@ -108,33 +104,27 @@ public class SyslogServer extends AbstractUnreliableSource { hasMatch = true; } if (!hasMatch) { - LOG.warn("the syslog msg had been discard, beacuse not match ip list, the ip is " + hostAddress + ". the msg is " + message.getMessageBody()); + LOGGER.warn("the syslog msg had been discard, beacuse not match ip list, the ip is " + hostAddress + ". the msg is " + message.getMessageBody()); return message; } for (SyslogChannel channel : syslogChannels) { - if (channel.isDestroy() == false) { - channel.doReceiveMessage(message.getMessageBody()); - } else { - routers.remove(channel); - } + channel.doReceiveMessage(message.getMessageBody()); } return message; } }); - SyslogServerIF serverIF=org.graylog2.syslog4j.server.SyslogServer.getThreadedInstance(protocol); + SyslogServerIF serverIF = org.graylog2.syslog4j.server.SyslogServer.getThreadedInstance(protocol); return true; } - private static final String PREFIX = "dipper.upgrade.channel.syslog.envkey"; - - @Override public void destroy() { + @Override public void destroySource() { isFinished = true; if (syslogServer != null) { try { syslogServer.shutdown(); Thread.sleep(30 * 1000); } catch (Exception e) { - + throw new RuntimeException("syslog destroy error", e); } } @@ -172,6 +162,7 @@ public void setCtimeout(Integer ctimeout) { this.ctimeout = ctimeout; } + @Override public boolean isFinished() { return isFinished; } @@ -180,7 +171,23 @@ public void setFinished(boolean finished) { isFinished = finished; } - private volatile transient SoftReferenceCache> cache = new SoftReferenceCache<>(); + public int getTimeout() { + return timeout; + } + + public void setTimeout(int timeout) { + this.timeout = timeout; + } + + public void clearCache() { + SoftReferenceCache tmp = cache; + cache = new SoftReferenceCache<>(); + tmp.clear(); + } + + public List getRouters() { + return routers; + } protected class SyslogServerEventHandler implements SyslogServerSessionEventHandlerIF { private static final long serialVersionUID = 6036415838696050746L; @@ -234,7 +241,7 @@ public SyslogServerEventHandler() { } JSONObject msg = new JSONObject(); - msg.put("data",message); + msg.put("data", message); msg.put("facility", var4.getFacility()); msg.put("hostName", hostName); msg.put("hostAddress", hostAddress); @@ -256,22 +263,4 @@ public SyslogServerEventHandler() { @Override public void destroy(SyslogServerIF var1) { } } - - public int getTimeout() { - return timeout; - } - - public void setTimeout(int timeout) { - this.timeout = timeout; - } - - public void clearCache() { - SoftReferenceCache tmp = cache; - cache = new SoftReferenceCache<>(); - tmp.clear(); - } - - public List getRouters() { - return routers; - } } diff --git a/rocketmq-streams-channel-syslog/src/test/java/org/apache/rocketmq/streams/syslog/SyslogClient.java b/rocketmq-streams-channel-syslog/src/test/java/org/apache/rocketmq/streams/syslog/SyslogClient.java index d3c1229c..8ce69274 100644 --- a/rocketmq-streams-channel-syslog/src/test/java/org/apache/rocketmq/streams/syslog/SyslogClient.java +++ b/rocketmq-streams-channel-syslog/src/test/java/org/apache/rocketmq/streams/syslog/SyslogClient.java @@ -36,7 +36,7 @@ public void sendSyslog() throws InterruptedException { syslogChannel.start(new IStreamOperator() { @Override public Object doMessage(IMessage message, AbstractContext context) { - if(!message.getHeader().isSystemMessage()){ + if (!message.getHeader().isSystemMessage()) { System.out.println(message.getMessageBody()); } return null; @@ -48,20 +48,19 @@ public Object doMessage(IMessage message, AbstractContext context) { Thread.sleep(1000000000l); } - @Test public void sendTestData() throws InterruptedException { - IChannel channel=createSyslogChannel(); + IChannel channel = createSyslogChannel(); JSONObject msg = new JSONObject(); msg.put("name", "chris"); - msg.put("host",IPUtil.getLocalIP()); + msg.put("host", IPUtil.getLocalIP()); channel.batchAdd(new Message(msg)); channel.flush(); Thread.sleep(3000); } private SyslogChannel createSyslogChannel() { - SyslogChannel syslogChannel = new SyslogChannel(IPUtil.getLocalIP(),SyslogChannelManager.tcpPort); + SyslogChannel syslogChannel = new SyslogChannel(IPUtil.getLocalIP(), SyslogChannelManager.tcpPort); syslogChannel.setTCPProtol(); syslogChannel.addIps(IPUtil.getLocalIP()); System.out.println(IPUtil.getLocalIP()); diff --git a/rocketmq-streams-channel-tencent/pom.xml b/rocketmq-streams-channel-tencent/pom.xml new file mode 100644 index 00000000..ca6e6628 --- /dev/null +++ b/rocketmq-streams-channel-tencent/pom.xml @@ -0,0 +1,42 @@ + + + 4.0.0 + + org.apache.rocketmq + rocketmq-streams + 2.0.1-SNAPSHOT + + + rocketmq-streams-channel-tencent + ROCKETMQ STREAMS :: channel-tencent + + + + + org.apache.rocketmq + rocketmq-streams-connectors + + + com.tencentcloudapi + tencentcloud-sdk-java + 3.1.715 + + + junit + junit + test + + + org.apache.commons + commons-compress + 1.21 + + + com.opencsv + opencsv + 5.8 + + + \ No newline at end of file diff --git a/rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/TencentOpenAPIClient.java b/rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/TencentOpenAPIClient.java new file mode 100644 index 00000000..7200497f --- /dev/null +++ b/rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/TencentOpenAPIClient.java @@ -0,0 +1,55 @@ +/* + * 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.tencent; + +import com.tencentcloudapi.common.AbstractClient; +import com.tencentcloudapi.common.Credential; +import com.tencentcloudapi.common.profile.ClientProfile; + +public class TencentOpenAPIClient extends AbstractClient { + + private String endpoint; + private String version; + + public TencentOpenAPIClient(String endpoint, String version, Credential credential, String region) { + super(endpoint, version, credential, region); + this.endpoint = endpoint; + this.version = version; + } + + public TencentOpenAPIClient(String endpoint, String version, Credential credential, String region, ClientProfile profile) { + super(endpoint, version, credential, region, profile); + this.endpoint = endpoint; + this.version = version; + } + + public String getEndpoint() { + return endpoint; + } + + public void setEndpoint(String endpoint) { + this.endpoint = endpoint; + } + + public String getVersion() { + return version; + } + + public void setVersion(String version) { + this.version = version; + } +} diff --git a/rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/TencentOpenAPISplit.java b/rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/TencentOpenAPISplit.java new file mode 100644 index 00000000..17de2cda --- /dev/null +++ b/rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/TencentOpenAPISplit.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.tencent; + +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; + +public class TencentOpenAPISplit extends BasedConfigurable implements ISplit { + protected String queueId; + protected Long initStartTime; + protected long dateAdd; + protected Long pollingMinute; + + public TencentOpenAPISplit(String queueId, Long initStartTime, long dateAdd, Long pollingMinute) { + this.queueId = queueId; + this.initStartTime = initStartTime; + this.dateAdd = dateAdd; + this.pollingMinute = pollingMinute; + } + + public TencentOpenAPISplit() { + } + + @Override + public String getQueueId() { + return queueId; + } + + public void setQueueId(String queueId) { + this.queueId = queueId; + } + + @Override + public String getQueue() { + return queueId; + } + + @Override + public int compareTo(TencentOpenAPISplit o) { + return queueId.compareTo(o.queueId); + } + + public Long getInitStartTime() { + return initStartTime; + } + + public void setInitStartTime(Long initStartTime) { + this.initStartTime = initStartTime; + } + + public long getDateAdd() { + return dateAdd; + } + + public void setDateAdd(long dateAdd) { + this.dateAdd = dateAdd; + } + + public Long getPollingMinute() { + return pollingMinute; + } + + public void setPollingMinute(Long pollingMinute) { + this.pollingMinute = pollingMinute; + } +} diff --git a/rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/common/TencentOpenAPIChannelBuilder.java b/rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/common/TencentOpenAPIChannelBuilder.java new file mode 100644 index 00000000..c1d783a9 --- /dev/null +++ b/rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/common/TencentOpenAPIChannelBuilder.java @@ -0,0 +1,81 @@ +/* + * 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.tencent.common; + +import com.alibaba.fastjson.JSONObject; +import com.google.auto.service.AutoService; +import java.util.Properties; +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.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 = TencentOpenAPIChannelBuilder.TYPE, aliasName = "TencentOpenAPISource") +public class TencentOpenAPIChannelBuilder extends AbstractSupportShuffleChannelBuilder { + public static final String TYPE = "tencent_open_api"; + + @Override + public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) { + return (TencentOpenAPISource) ConfigurableUtil.create(TencentOpenAPISource.class.getName(), namespace, name, createFormatProperty(properties), null); + } + + @Override + public String getType() { + return TYPE; + } + + @Override + public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { + throw new RuntimeException("can not support this method"); + } + + 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.get(key)); + } + + IChannelBuilder.formatPropertiesName(formatProperties, properties, "endPoint", "end_point"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "accessId", "ak"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "accessKey", "sk"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "action", "action"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "version", "version"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "region", "region"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "jsonPayLoad", "json_payload"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "startField", "start_field"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "endField", "end_field"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "pageNumberField", "page_num_field"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "pageSizeField", "page_size_field"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "totalField", "total_field"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "dataField", "data_field"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "requestIdField", "request_id_field"); + return formatProperties; + } + + @Override + public ISink createBySource(ISource pipelineSource) { + return null; + } +} diff --git a/rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/common/TencentOpenAPISource.java b/rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/common/TencentOpenAPISource.java new file mode 100644 index 00000000..0a34db94 --- /dev/null +++ b/rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/common/TencentOpenAPISource.java @@ -0,0 +1,373 @@ +/* + * 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.tencent.common; + +import com.alibaba.fastjson.JSONArray; +import com.alibaba.fastjson.JSONObject; + +import com.tencentcloudapi.common.Credential; +import com.tencentcloudapi.common.exception.TencentCloudSDKException; + +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.context.MessageOffset; +import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.common.utils.IdUtil; +import org.apache.rocketmq.streams.common.utils.ReflectUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.connectors.model.PullMessage; +import org.apache.rocketmq.streams.connectors.reader.AbstractSplitReader; +import org.apache.rocketmq.streams.connectors.reader.ISplitReader; +import org.apache.rocketmq.streams.connectors.source.AbstractPullSource; +import org.apache.rocketmq.streams.tencent.TencentOpenAPIClient; +import org.apache.rocketmq.streams.tencent.TencentOpenAPISplit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TencentOpenAPISource extends AbstractPullSource { + + private static final Logger LOGGER = LoggerFactory.getLogger(TencentOpenAPISource.class); + + protected String endPoint; + + protected String accessId; + + protected String accessKey; + + protected String version; + + protected String action; + + protected String region; + + protected String jsonPayLoad; + + protected String startField; + + protected String endField; + + protected String pageNumberField; + + protected String pageSizeField; + + protected String totalField; + + protected String dataField; + + protected String requestIdField; + + /** + * 每10分钟拉去一次 + */ + protected Long pollingMinute = 10L; + /** + * 在一个周期内的并发性 + */ + protected int splitCount = 1; + + /** + * 调度开始时间 + */ + protected String initFirstTime = DateUtil.getCurrentTimeString(); + + /** + * 最大一页数据数量 + */ + protected int maxPageSize = 10; + + /** + * 时间戳是毫秒还是秒, true是毫秒,false是秒 + */ + protected boolean isTimeStamp = true; + + @Override + protected boolean initConfigurable() { + return super.initConfigurable(); + } + + @Override + protected ISplitReader createSplitReader(ISplit split) { + return new AbstractSplitReader(split) { + + private long startTime; + private long dateAdd; + private long pollingMinute; + private int pageNum = 1; + private transient TencentOpenAPIClient tencentOpenAPIClient; + + @Override + public void open() { + Credential cred = new Credential(accessId, accessKey); + TencentOpenAPISplit openAPISplit = (TencentOpenAPISplit) split; + this.startTime = openAPISplit.getInitStartTime(); + this.dateAdd = openAPISplit.getDateAdd(); + this.pollingMinute = openAPISplit.getPollingMinute(); + this.tencentOpenAPIClient = new TencentOpenAPIClient(endPoint, version, cred, region); + } + + @Override + public boolean next() { + return this.startTime + this.pollingMinute * 60 * 1000 < System.currentTimeMillis(); + } + + @Override + public Iterator> getMessage() { + Map parameters = new HashMap<>(); + if (StringUtil.isNotEmpty(jsonPayLoad)) { + parameters = JSONObject.parseObject(jsonPayLoad); + } + + if (StringUtil.isNotEmpty(startField)) { + parameters.put(startField, createTime(startTime)); + } + if (StringUtil.isNotEmpty(endField)) { + parameters.put(endField, createTime(startTime + dateAdd * 60 * 1000)); + } + if (StringUtil.isNotEmpty(pageNumberField)) { + parameters.put(pageNumberField, pageNum); + } + if (StringUtil.isNotEmpty(pageSizeField)) { + parameters.put(pageSizeField, maxPageSize); + } + Object queryStartTime = parameters.get(startField); + Object queryEndTime = parameters.get(endField); + JSONObject msg = null; + try { + String resultString = tencentOpenAPIClient.call(action, JSONObject.toJSONString(parameters)); + JSONObject result = JSONObject.parseObject(resultString); + msg = result == null ? new JSONObject() : result.getJSONObject("Response"); + } catch (TencentCloudSDKException e) { + throw new RuntimeException(e); + } + JSONArray data = msg.getJSONArray(dataField); + if (data == null || data.isEmpty()) { + startTime = startTime + pollingMinute * 60 * 1000; + pageNum = 1; + return null; + } else if (StringUtil.isNotEmpty(totalField)) { + Integer totalCount = ReflectUtil.getBeanFieldOrJsonValue(data, totalField); + if (totalCount == null) { + throw new RuntimeException("expect get totalCount from result msg by " + totalField + ", real is not find"); + } + long currentCount = (long) (pageNum) * maxPageSize; + if (currentCount >= totalCount) { + startTime = startTime + pollingMinute * 60 * 1000; + pageNum = 1; + } else { + pageNum++; + } + } else if (data.size() < maxPageSize) { + startTime = startTime + pollingMinute * 60 * 1000; + pageNum = 1; + } else { + pageNum++; + } + List> msgs = new ArrayList<>(); + msg.remove(dataField); + LOGGER.info("[{}][{}] Execute_Openapi_Query_Result_Size({})_Query_Time_Gap({}-{})_Query_Page({})", IdUtil.instanceId(), getName(), data.size(), DateUtil.format(new Date(startTime)), DateUtil.format(new Date(startTime + dateAdd * 60 * 1000)), pageNum); + for (int i = 0; i < data.size(); i++) { + JSONObject message = new JSONObject(); + message.putAll(msg); + message.putAll(data.getJSONObject(i)); + message.put(startField, queryStartTime); + message.put(endField, queryEndTime); + message.put(requestIdField, msg.get(requestIdField)); + + PullMessage pullMessage = new PullMessage<>(); + pullMessage.setMessage(message); + MessageOffset messageOffset = new MessageOffset(startTime + ";" + (10000 + pageNum)); + messageOffset.addLayerOffset(i); + pullMessage.setMessageOffset(messageOffset); + msgs.add(pullMessage); + } + return msgs.iterator(); + } + + @Override + public long getDelay() { + return System.currentTimeMillis() - startTime; + } + + @Override + public long getFetchedDelay() { + return 0; + } + + @Override + public void seek(String cursor) { + if (cursor == null) { + this.cursor = this.startTime + ";" + (pageNum + 10000); + return; + } + this.cursor = cursor; + String[] values = cursor.split(";"); + this.startTime = Long.parseLong(values[0]); + this.pageNum = Integer.parseInt(values[1]) - 10000; + } + + @Override + public String getCursor() { + this.cursor = this.startTime + ";" + (pageNum + 10000); + return this.cursor; + } + + }; + } + + protected Object createTime(long time) { + if (isTimeStamp) { + return time; + } else { + return time / 1000; + } + } + + @Override + public List> fetchAllSplits() { + List> splits = new ArrayList<>(); + long baseDataAdd = pollingMinute / splitCount; + long remainder = pollingMinute % splitCount; + Date date = DateUtil.getWindowBeginTime(DateUtil.parseTime(initFirstTime).getTime(), pollingMinute); + for (int i = 0; i < splitCount; i++) { + long dataAdd = baseDataAdd + (remainder > 0 ? 1 : 0); + remainder--; + Date endDate = DateUtil.addMinute(date, (int) dataAdd); + TencentOpenAPISplit openAPISplit = new TencentOpenAPISplit(i + "", date.getTime(), dataAdd, pollingMinute); + splits.add(openAPISplit); + date = endDate; + } + return splits; + } + + public String getEndPoint() { + return endPoint; + } + + public void setEndPoint(String endPoint) { + this.endPoint = endPoint; + } + + public String getAccessId() { + return accessId; + } + + public void setAccessId(String accessId) { + this.accessId = accessId; + } + + public String getAccessKey() { + return accessKey; + } + + public void setAccessKey(String accessKey) { + this.accessKey = accessKey; + } + + public String getVersion() { + return version; + } + + public void setVersion(String version) { + this.version = version; + } + + public String getAction() { + return action; + } + + public void setAction(String action) { + this.action = action; + } + + public String getRegion() { + return region; + } + + public void setRegion(String region) { + this.region = region; + } + + public String getJsonPayLoad() { + return jsonPayLoad; + } + + public void setJsonPayLoad(String jsonPayLoad) { + this.jsonPayLoad = jsonPayLoad; + } + + public String getStartField() { + return startField; + } + + public void setStartField(String startField) { + this.startField = startField; + } + + public String getEndField() { + return endField; + } + + public void setEndField(String endField) { + this.endField = endField; + } + + public String getPageNumberField() { + return pageNumberField; + } + + public void setPageNumberField(String pageNumberField) { + this.pageNumberField = pageNumberField; + } + + public String getPageSizeField() { + return pageSizeField; + } + + public void setPageSizeField(String pageSizeField) { + this.pageSizeField = pageSizeField; + } + + public String getTotalField() { + return totalField; + } + + public void setTotalField(String totalField) { + this.totalField = totalField; + } + + public String getDataField() { + return dataField; + } + + public void setDataField(String dataField) { + this.dataField = dataField; + } + + public String getRequestIdField() { + return requestIdField; + } + + public void setRequestIdField(String requestIdField) { + this.requestIdField = requestIdField; + } +} diff --git a/rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/waf/TencentWafAttackChannelBuilder.java b/rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/waf/TencentWafAttackChannelBuilder.java new file mode 100644 index 00000000..47c4abfa --- /dev/null +++ b/rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/waf/TencentWafAttackChannelBuilder.java @@ -0,0 +1,76 @@ +/* + * 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.tencent.waf; + +import com.alibaba.fastjson.JSONObject; +import com.google.auto.service.AutoService; +import java.util.Properties; +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.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 = TencentWafAttackChannelBuilder.TYPE, aliasName = "TencentWafAttackSource") +public class TencentWafAttackChannelBuilder extends AbstractSupportShuffleChannelBuilder { + public static final String TYPE = "tencent_waf_attack"; + + @Override + public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) { + return (TencentWafAttackSource) ConfigurableUtil.create(TencentWafAttackSource.class.getName(), namespace, name, createFormatProperty(properties), null); + } + + @Override + public String getType() { + return TYPE; + } + + @Override + public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { + throw new RuntimeException("can not support this method"); + } + + 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.get(key)); + } + + IChannelBuilder.formatPropertiesName(formatProperties, properties, "endPoint", "endpoint"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "accessId", "ak"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "accessKey", "sk"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "action", "action"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "version", "version"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "region", "region"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "domain", "domain"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "queryString", "queryString"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "sort", "sort"); + return formatProperties; + } + + @Override + public ISink createBySource(ISource pipelineSource) { + return null; + } +} diff --git a/rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/waf/TencentWafAttackSource.java b/rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/waf/TencentWafAttackSource.java new file mode 100644 index 00000000..d99c0486 --- /dev/null +++ b/rocketmq-streams-channel-tencent/src/main/java/org/apache/rocketmq/streams/tencent/waf/TencentWafAttackSource.java @@ -0,0 +1,377 @@ +/* + * 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.tencent.waf; + +import com.alibaba.fastjson.JSONArray; +import com.alibaba.fastjson.JSONObject; + +import com.opencsv.CSVReader; +import com.opencsv.CSVReaderBuilder; +import com.opencsv.RFC4180Parser; +import com.opencsv.exceptions.CsvValidationException; +import com.tencentcloudapi.common.Credential; +import com.tencentcloudapi.common.exception.TencentCloudSDKException; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.net.URL; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.zip.GZIPInputStream; + +import org.apache.commons.compress.archivers.tar.TarArchiveEntry; +import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; +import org.apache.commons.lang3.time.DateFormatUtils; +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.context.MessageOffset; +import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.common.utils.IdUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.connectors.model.PullMessage; +import org.apache.rocketmq.streams.connectors.reader.AbstractSplitReader; +import org.apache.rocketmq.streams.connectors.reader.ISplitReader; +import org.apache.rocketmq.streams.connectors.source.AbstractPullSource; +import org.apache.rocketmq.streams.tencent.TencentOpenAPIClient; +import org.apache.rocketmq.streams.tencent.TencentOpenAPISplit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TencentWafAttackSource extends AbstractPullSource { + + private static final Logger LOGGER = LoggerFactory.getLogger(TencentWafAttackSource.class); + protected final int BUFFER_SIZE = 10240; + protected String endPoint; + protected String accessId; + protected String accessKey; + protected String version; + protected String region; + protected String domain; + protected String queryString; + protected String sort; + protected String taskNameField; + protected String startField; + protected String endField; + /** + * 每10分钟拉去一次 + */ + protected Long pollingMinute = 10L; + /** + * 在一个周期内的并发性 + */ + protected int splitCount = 1; + /** + * 调度开始时间 + */ + protected String initFirstTime = DateUtil.getCurrentTimeString(); + + protected int retryTimes = 10; + + protected int sleepTime = 10000; + + @Override + protected boolean initConfigurable() { + return super.initConfigurable(); + } + + @Override + protected ISplitReader createSplitReader(ISplit split) { + return new AbstractSplitReader(split) { + private String id; + private long startTime; + private long dateAdd; + private long pollingMinute; + private int pageNum = 1; + + private transient TencentOpenAPIClient tencentOpenAPIClient; + + @Override + public void open() { + Credential cred = new Credential(accessId, accessKey); + TencentOpenAPISplit openAPISplit = (TencentOpenAPISplit)split; + this.id = openAPISplit.getQueueId(); + this.startTime = openAPISplit.getInitStartTime(); + this.dateAdd = openAPISplit.getDateAdd(); + this.pollingMinute = openAPISplit.getPollingMinute(); + this.tencentOpenAPIClient = new TencentOpenAPIClient(endPoint, version, cred, region); + } + + @Override + public boolean next() { + return this.startTime + this.pollingMinute * 60 * 1000 < System.currentTimeMillis(); + } + + @Override + public Iterator> getMessage() { + Iterator> iterator = null; + Map parameters = new HashMap<>(); + if (StringUtil.isNotEmpty(startField)) { + parameters.put(startField, createTime(startTime)); + //parameters.put(startField, "2023-03-27T00:45:09+08:00"); + } + + long endTime = startTime + dateAdd * 60 * 1000; + if (StringUtil.isNotEmpty(endField)) { + parameters.put(endField, createTime(endTime)); + } + if (StringUtil.isNotEmpty(domain)) { + parameters.put("Domain", domain); + } + if (StringUtil.isNotEmpty(queryString)) { + parameters.put("QueryString", queryString); + } + if (StringUtil.isNotEmpty(sort)) { + parameters.put("Sort", sort); + } else { + parameters.put("Sort", "desc"); + } + + String taskName = this.id + "_" + DateUtil.format(new Date(startTime), "yyyyMMddHHmmss") + "_" + DateUtil.format(new Date(endTime), "yyyyMMddHHmmss"); + + if (StringUtil.isNotEmpty(taskNameField)) { + parameters.put(taskNameField, taskName); + } + try { + String payLoad = JSONObject.toJSONString(parameters); + String resultString = tencentOpenAPIClient.call("PostAttackDownloadTask", payLoad); + LOGGER.info("[{}][{}] Execute_Openapi_Query({})_Result({})_Query_Time_Gap({}-{})_Query_Page({})", IdUtil.instanceId(), getName(), payLoad, resultString, DateUtil.format(new Date(startTime)), DateUtil.format(new Date(endTime)), pageNum); + JSONObject result = JSONObject.parseObject(resultString); + if (result != null) { + JSONObject data = result.getJSONObject("Response"); + if (data != null) { + String taskId = data.getString("Flow"); + if (taskId != null) { + String url = null; + int times = 1; + do { + String rspStr = tencentOpenAPIClient.call("GetAttackDownloadRecords", "{}"); + JSONObject rsp = JSONObject.parseObject(rspStr).getJSONObject("Response"); + JSONArray array = rsp.getJSONArray("Records"); + LOGGER.info("[{}][{}] Execute_Openapi_Download_ResultSize({})", IdUtil.instanceId(), getName(), array.size()); + for (int i = 0; i < array.size(); i++) { + JSONObject jsonObject = array.getJSONObject(i); + if (jsonObject.getString("Id") != null && taskId.equals(jsonObject.getString("Id"))) { + url = jsonObject.getString("Url"); + if (url == null || url.isEmpty()) { + Thread.sleep(sleepTime); + } + break; + } + } + times++; + } + while (times <= retryTimes && (url == null || url.isEmpty())); + if (url != null && !url.isEmpty()) { + try { + iterator = extract(download(url)).iterator(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } else { + LOGGER.info("[{}][{}] Can_Not_Get_The_Url_After_Retry_Times({})", IdUtil.instanceId(), getName(), retryTimes); + } + + } + } + } + this.startTime = this.startTime + this.pollingMinute * 60 * 1000; + this.pageNum = 1; + } catch (TencentCloudSDKException | InterruptedException e) { + LOGGER.error("Tencent_OpenAPI_Call_Error", e); + } + return iterator; + } + + @Override + public long getDelay() { + return System.currentTimeMillis() - startTime; + } + + @Override + public long getFetchedDelay() { + return 0; + } + + @Override + public void seek(String cursor) { + if (cursor == null) { + this.cursor = this.startTime + ";" + (pageNum + 10000); + return; + } + this.cursor = cursor; + String[] values = cursor.split(";"); + this.startTime = Long.parseLong(values[0]); + this.pageNum = Integer.parseInt(values[1]) - 10000; + } + + @Override + public String getCursor() { + this.cursor = this.startTime + ";" + (pageNum + 10000); + return this.cursor; + } + + private byte[] download(String url) throws Exception { + byte[] tarData; + try (InputStream inputStream = new URL(url).openStream()) { + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + byte[] buffer = new byte[BUFFER_SIZE]; + int n; + while ((n = inputStream.read(buffer, 0, BUFFER_SIZE)) >= 0) { + outputStream.write(buffer, 0, n); + } + outputStream.close(); + tarData = outputStream.toByteArray(); + } + return tarData; + } + + public List> extract(byte[] tarData) throws IOException, CsvValidationException { + List> msgs = new ArrayList<>(); + try (ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(tarData); GZIPInputStream gzipInputStream = new GZIPInputStream(byteArrayInputStream); TarArchiveInputStream tarArchiveInputStream = new TarArchiveInputStream(gzipInputStream)) { + TarArchiveEntry tarEntry; + while ((tarEntry = tarArchiveInputStream.getNextTarEntry()) != null) { + if (tarEntry.isDirectory()) { + continue; + } + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + byte[] buffer = new byte[BUFFER_SIZE]; + int n; + while ((n = tarArchiveInputStream.read(buffer, 0, BUFFER_SIZE)) >= 0) { + outputStream.write(buffer, 0, n); + } + outputStream.close(); + byte[] fileContent = outputStream.toByteArray(); + // 将CSV数据解析为Java对象 + ByteArrayInputStream fileInputStream = new ByteArrayInputStream(fileContent); + InputStreamReader inputStreamReader = new InputStreamReader(fileInputStream); + try (CSVReader csvReader = new CSVReaderBuilder(inputStreamReader).withCSVParser(new RFC4180Parser()).build()) { + String[] header = csvReader.readNext(); + String[] row; + while ((row = csvReader.readNext()) != null) { + // 将行数据转换为Java对象并进行处理 + // 示例代码中使用了Arrays.toString()打印行数据 + JSONObject message = new JSONObject(); + for (int i = 0; i < header.length; i++) { + message.put(header[i], i < row.length ? row[i] : ""); + } + PullMessage pullMessage = new PullMessage<>(); + pullMessage.setMessage(message); + MessageOffset messageOffset = new MessageOffset(this.startTime + ";" + (pageNum + 10000)); + messageOffset.addLayerOffset(0); + pullMessage.setMessageOffset(messageOffset); + msgs.add(pullMessage); + } + } catch (CsvValidationException e) { + LOGGER.error("CSV_Parse_Error", e); + } + } + } + return msgs; + } + + }; + } + + protected String createTime(long time) { + Date date = new Date(time); + ZonedDateTime localDateTime = date.toInstant().atZone(ZoneId.systemDefault()); + return localDateTime.format(DateTimeFormatter.ISO_OFFSET_DATE_TIME); + } + + @Override + public List> fetchAllSplits() { + List> splits = new ArrayList<>(); + long baseDataAdd = pollingMinute / splitCount; + long remainder = pollingMinute % splitCount; + Date date = DateUtil.getWindowBeginTime(DateUtil.parseTime(initFirstTime).getTime(), pollingMinute); + for (int i = 0; i < splitCount; i++) { + long dataAdd = baseDataAdd + (remainder > 0 ? 1 : 0); + remainder--; + Date endDate = DateUtil.addMinute(date, (int)dataAdd); + LOGGER.info("[{}][{}] Get_Split_From({})_To({})", IdUtil.instanceId(), getName(), DateFormatUtils.format(date, "yyyy-MM-dd HH:mm:ss"), DateFormatUtils.format(endDate, "yyyy-MM-dd HH:mm:ss")); + TencentOpenAPISplit openAPISplit = new TencentOpenAPISplit(getName() + "$#" + i, date.getTime(), dataAdd, pollingMinute); + splits.add(openAPISplit); + date = endDate; + } + return splits; + } + + public String getEndPoint() { + return endPoint; + } + + public void setEndPoint(String endPoint) { + this.endPoint = endPoint; + } + + public String getAccessId() { + return accessId; + } + + public void setAccessId(String accessId) { + this.accessId = accessId; + } + + public String getAccessKey() { + return accessKey; + } + + public void setAccessKey(String accessKey) { + this.accessKey = accessKey; + } + + public String getVersion() { + return version; + } + + public void setVersion(String version) { + this.version = version; + } + + public String getRegion() { + return region; + } + + public void setRegion(String region) { + this.region = region; + } + + public int getRetryTimes() { + return retryTimes; + } + + public void setRetryTimes(int retryTimes) { + this.retryTimes = retryTimes; + } + + public int getSleepTime() { + return sleepTime; + } + + public void setSleepTime(int sleepTime) { + this.sleepTime = sleepTime; + } +} diff --git a/rocketmq-streams-channel-tencent/src/test/java/org/apache/rocketmq/streams/tencent/waf/TestCommonAPI.java b/rocketmq-streams-channel-tencent/src/test/java/org/apache/rocketmq/streams/tencent/waf/TestCommonAPI.java new file mode 100644 index 00000000..329ac23a --- /dev/null +++ b/rocketmq-streams-channel-tencent/src/test/java/org/apache/rocketmq/streams/tencent/waf/TestCommonAPI.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.tencent.waf; + +import com.opencsv.CSVReader; +import com.tencentcloudapi.common.Credential; +import com.tencentcloudapi.common.exception.TencentCloudSDKException; +import com.tencentcloudapi.common.profile.ClientProfile; +import com.tencentcloudapi.common.profile.HttpProfile; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.net.URL; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.util.Arrays; +import java.util.Date; +import java.util.zip.GZIPInputStream; + +import org.apache.commons.compress.archivers.tar.TarArchiveEntry; +import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; +import org.apache.rocketmq.streams.tencent.TencentOpenAPIClient; +import org.junit.Before; +import org.junit.Test; + +public class TestCommonAPI { + + private Credential cred; + + @Before + public void init() { + // 实例化一个认证对象,入参需要传入腾讯云账户 SecretId,SecretKey。 + // 为了保护密钥安全,建议将密钥设置在环境变量中或者配置文件中,请参考本文凭证管理章节。 + // 硬编码密钥到代码中有可能随代码泄露而暴露,有安全隐患,并不推荐。 + // Credential cred = new Credential("SecretId", "SecretKey"); + this.cred = new Credential("", ""); + + } + + @Test + public void test() { + + // 实例化一个http选项,可选的,没有特殊需求可以跳过 + HttpProfile httpProfile = new HttpProfile(); + // 实例化一个client选项,可选的,没有特殊需求可以跳过 + ClientProfile clientProfile = new ClientProfile(); + clientProfile.setHttpProfile(httpProfile); + // 实例化要请求产品的client对象,clientProfile是可选的 + + String endpoint = "waf.tencentcloudapi.com"; + String version = "2018-01-25"; + TencentOpenAPIClient client = new TencentOpenAPIClient(endpoint, version, cred, "ap-guangzhou", clientProfile) { + }; + + try { + String rspStr = client.call("GetAttackDownloadRecords", "{}"); + System.out.println(rspStr); + } catch (TencentCloudSDKException e) { + throw new RuntimeException(e); + } + } + + @Test + public void test1() { + + // 实例化一个http选项,可选的,没有特殊需求可以跳过 + HttpProfile httpProfile = new HttpProfile(); + // 实例化一个client选项,可选的,没有特殊需求可以跳过 + ClientProfile clientProfile = new ClientProfile(); + clientProfile.setHttpProfile(httpProfile); + // 实例化要请求产品的client对象,clientProfile是可选的 + + String endpoint = "waf.tencentcloudapi.com"; + String version = "2018-01-25"; + TencentOpenAPIClient client = new TencentOpenAPIClient(endpoint, version, cred, "ap-guangzhou", clientProfile) { + }; + + try { + + String rspStr = client.call("PostAttackDownloadTask", "{\n" + + " \"EndTime\": \"2023-03-27T10:55:09+08:00\",\n" + + " \"TaskName\": \"test$#0_1679882628000\",\n" + + " \"QueryString\": \"method:GET\",\n" + + " \"StartTime\": \"2023-03-27T00:45:09+08:00\",\n" + + " \"Sort\": \"desc\",\n" + + " \"Domain\": \"all\"\n" + + "}"); + + System.out.println(rspStr); + } catch (TencentCloudSDKException e) { + throw new RuntimeException(e); + } + } + + @Test + public void test2() { + // 实例化一个http选项,可选的,没有特殊需求可以跳过 + HttpProfile httpProfile = new HttpProfile(); + // 实例化一个client选项,可选的,没有特殊需求可以跳过 + ClientProfile clientProfile = new ClientProfile(); + clientProfile.setHttpProfile(httpProfile); + // 实例化要请求产品的client对象,clientProfile是可选的 + + String endpoint = "waf.tencentcloudapi.com"; + String version = "2018-01-25"; + TencentOpenAPIClient client = new TencentOpenAPIClient(endpoint, version, cred, "ap-guangzhou", clientProfile) { + }; + + try { + String rspStr = client.call("DescribeRuleLimit", "{\n" + + " \"Domain\": \"leixi6.aliyundemo.com\"\n" + + "}"); + System.out.println(rspStr); + } catch (TencentCloudSDKException e) { + throw new RuntimeException(e); + } + } + + @Test + public void test3() { + // 实例化一个http选项,可选的,没有特殊需求可以跳过 + HttpProfile httpProfile = new HttpProfile(); + // 实例化一个client选项,可选的,没有特殊需求可以跳过 + ClientProfile clientProfile = new ClientProfile(); + clientProfile.setHttpProfile(httpProfile); + // 实例化要请求产品的client对象,clientProfile是可选的 + + String endpoint = "waf.tencentcloudapi.com"; + String version = "2018-01-25"; + TencentOpenAPIClient client = new TencentOpenAPIClient(endpoint, version, cred, "ap-guangzhou", clientProfile) { + }; + + try { + String rspStr = client.call("DescribeAccessExports", "{\n" + + " \"TopicId\": \"1ae37c76-df99-4e2b-998c-20f39eba6226\"\n" + + "}"); + System.out.println(rspStr); + } catch (TencentCloudSDKException e) { + throw new RuntimeException(e); + } + } + + @Test + public void test4() { + Date date = new Date(System.currentTimeMillis()); + ZonedDateTime localDateTime = date.toInstant().atZone(ZoneId.systemDefault()); + String dateFormatter = localDateTime.format(DateTimeFormatter.ISO_OFFSET_DATE_TIME); + System.out.println(dateFormatter); + + } + + @Test + public void test6() { + + // 实例化一个http选项,可选的,没有特殊需求可以跳过 + HttpProfile httpProfile = new HttpProfile(); + // 实例化一个client选项,可选的,没有特殊需求可以跳过 + ClientProfile clientProfile = new ClientProfile(); + clientProfile.setHttpProfile(httpProfile); + // 实例化要请求产品的client对象,clientProfile是可选的 + + String endpoint = "waf.tencentcloudapi.com"; + String version = "2018-01-25"; + TencentOpenAPIClient client = new TencentOpenAPIClient(endpoint, version, cred, "ap-guangzhou", clientProfile) { + }; + + try { + String rspStr = client.call("DescribeWafAutoDenyStatus", "{}"); + System.out.println(rspStr); + } catch (TencentCloudSDKException e) { + throw new RuntimeException(e); + } + } + + @Test + public void test5() { + try { + String fileUrl = ""; + int BUFFER_SIZE = 10240; + byte[] tarData; + try (InputStream inputStream = new URL(fileUrl).openStream()) { + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + byte[] buffer = new byte[BUFFER_SIZE]; + int n; + while ((n = inputStream.read(buffer, 0, BUFFER_SIZE)) >= 0) { + outputStream.write(buffer, 0, n); + } + outputStream.close(); + tarData = outputStream.toByteArray(); + } + + try (ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(tarData); GZIPInputStream gzipInputStream = new GZIPInputStream(byteArrayInputStream); TarArchiveInputStream tarArchiveInputStream = new TarArchiveInputStream(gzipInputStream)) { + TarArchiveEntry tarEntry; + while ((tarEntry = tarArchiveInputStream.getNextTarEntry()) != null) { + if (tarEntry.isDirectory()) { + continue; + } + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + byte[] buffer = new byte[BUFFER_SIZE]; + int n; + while ((n = tarArchiveInputStream.read(buffer, 0, BUFFER_SIZE)) >= 0) { + outputStream.write(buffer, 0, n); + } + outputStream.close(); + byte[] fileContent = outputStream.toByteArray(); + // 将CSV数据解析为Java对象 + ByteArrayInputStream fileInputStream = new ByteArrayInputStream(fileContent); + InputStreamReader inputStreamReader = new InputStreamReader(fileInputStream); + CSVReader csvReader = new CSVReader(inputStreamReader); + String[] header = csvReader.readNext(); + + String[] row; + while ((row = csvReader.readNext()) != null) { + // 将行数据转换为Java对象并进行处理 + // 示例代码中使用了Arrays.toString()打印行数据 + System.out.println(Arrays.toString(row)); + } + csvReader.close(); + } + } + } catch (Exception e) { + + } + } + +} diff --git a/rocketmq-streams-channel-tencent/src/test/java/org/apache/rocketmq/streams/tencent/waf/TestWafAPI.java b/rocketmq-streams-channel-tencent/src/test/java/org/apache/rocketmq/streams/tencent/waf/TestWafAPI.java new file mode 100644 index 00000000..de25c2ab --- /dev/null +++ b/rocketmq-streams-channel-tencent/src/test/java/org/apache/rocketmq/streams/tencent/waf/TestWafAPI.java @@ -0,0 +1,87 @@ +/* + * 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.tencent.waf; + +import com.tencentcloudapi.common.Credential; +import com.tencentcloudapi.common.exception.TencentCloudSDKException; +import com.tencentcloudapi.common.profile.ClientProfile; +import com.tencentcloudapi.common.profile.HttpProfile; +import com.tencentcloudapi.waf.v20180125.WafClient; +import com.tencentcloudapi.waf.v20180125.models.GetAttackDownloadRecordsRequest; +import com.tencentcloudapi.waf.v20180125.models.GetAttackDownloadRecordsResponse; +import com.tencentcloudapi.waf.v20180125.models.PostAttackDownloadTaskRequest; +import com.tencentcloudapi.waf.v20180125.models.PostAttackDownloadTaskResponse; +import org.junit.Before; +import org.junit.Test; + +public class TestWafAPI { + + private WafClient client; + + @Before + public void init() { + // 实例化一个认证对象,入参需要传入腾讯云账户 SecretId,SecretKey。 + // 为了保护密钥安全,建议将密钥设置在环境变量中或者配置文件中,请参考本文凭证管理章节。 + // 硬编码密钥到代码中有可能随代码泄露而暴露,有安全隐患,并不推荐。 + // Credential cred = new Credential("SecretId", "SecretKey"); + Credential cred = new Credential("", ""); + //Credential cred = new Credential("", ""); + // 实例化一个http选项,可选的,没有特殊需求可以跳过 + HttpProfile httpProfile = new HttpProfile(); + httpProfile.setEndpoint("waf.tencentcloudapi.com"); + // 实例化一个client选项,可选的,没有特殊需求可以跳过 + ClientProfile clientProfile = new ClientProfile(); + clientProfile.setHttpProfile(httpProfile); + // 实例化要请求产品的client对象,clientProfile是可选的 + client = new WafClient(cred, "ap-guangzhou", clientProfile); + + } + + @Test + public void testWafGetAttackDownloadRecords() { + try { + // 实例化一个请求对象,每个接口都会对应一个request对象 + GetAttackDownloadRecordsRequest req = new GetAttackDownloadRecordsRequest(); + + // 返回的resp是一个GetAttackDownloadRecordsResponse的实例,与请求对象对应 + GetAttackDownloadRecordsResponse resp = client.GetAttackDownloadRecords(req); + // 输出json格式的字符串回包 + System.out.println(GetAttackDownloadRecordsResponse.toJsonString(resp)); + } catch (TencentCloudSDKException e) { + System.out.println(e.toString()); + } + } + + @Test + public void testPostAttackDownloadTask() { + PostAttackDownloadTaskRequest postAttackDownloadTaskRequest = new PostAttackDownloadTaskRequest(); + postAttackDownloadTaskRequest.setDomain("all"); + postAttackDownloadTaskRequest.setStartTime("2023-09-01T10:20:00+08:00"); + postAttackDownloadTaskRequest.setEndTime("2023-09-01T11:10:00+08:00"); + postAttackDownloadTaskRequest.setQueryString("method:*"); + postAttackDownloadTaskRequest.setTaskName("test_waf_download_1"); + + PostAttackDownloadTaskResponse postAttackDownloadTaskResponse = null; + try { + postAttackDownloadTaskResponse = client.PostAttackDownloadTask(postAttackDownloadTaskRequest); + } catch (TencentCloudSDKException e) { + throw new RuntimeException(e); + } + System.out.println(PostAttackDownloadTaskResponse.toJsonString(postAttackDownloadTaskResponse)); + } + +} diff --git a/rocketmq-streams-channel-view/.gitignore b/rocketmq-streams-channel-view/.gitignore new file mode 100644 index 00000000..b83d2226 --- /dev/null +++ b/rocketmq-streams-channel-view/.gitignore @@ -0,0 +1 @@ +/target/ diff --git a/rocketmq-streams-channel-view/pom.xml b/rocketmq-streams-channel-view/pom.xml new file mode 100755 index 00000000..d5431ca2 --- /dev/null +++ b/rocketmq-streams-channel-view/pom.xml @@ -0,0 +1,30 @@ + + + + 4.0.0 + + org.apache.rocketmq + rocketmq-streams + 2.0.1-SNAPSHOT + + rocketmq-streams-channel-view + ROCKETMQ STREAMS :: channel-view + jar + + + + org.apache.rocketmq + rocketmq-streams-serviceloader + + + + + + + elastic.co + https://artifacts.elastic.co/maven + + + diff --git a/rocketmq-streams-channel-view/src/main/java/org/apache/rocketmq/streams/view/ViewChannelBuilder.java b/rocketmq-streams-channel-view/src/main/java/org/apache/rocketmq/streams/view/ViewChannelBuilder.java new file mode 100644 index 00000000..2edd882d --- /dev/null +++ b/rocketmq-streams-channel-view/src/main/java/org/apache/rocketmq/streams/view/ViewChannelBuilder.java @@ -0,0 +1,108 @@ +/* + * 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.view; + +import com.alibaba.fastjson.JSONObject; +import com.google.auto.service.AutoService; +import java.util.Properties; +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.builder.IShuffleChannelBuilder; +import org.apache.rocketmq.streams.common.channel.impl.view.ViewSink; +import org.apache.rocketmq.streams.common.channel.impl.view.ViewSource; +import org.apache.rocketmq.streams.common.channel.sink.ISink; +import org.apache.rocketmq.streams.common.channel.source.ISource; +import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.metadata.MetaData; +import org.apache.rocketmq.streams.common.model.ServiceName; +import org.apache.rocketmq.streams.common.utils.ConfigurableUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.serviceloader.ServiceLoaderComponent; + +@AutoService(IChannelBuilder.class) +@ServiceName(value = ViewChannelBuilder.TYPE, aliasName = "ViewSource") +public class ViewChannelBuilder extends AbstractSupportShuffleChannelBuilder { + public static final String TYPE = "view"; + + @Override public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) { + ViewSource viewSource = (ViewSource) ConfigurableUtil.create(ViewSource.class.getName(), namespace, name, createFormatProperty(properties), null); + if (StringUtil.isEmpty(viewSource.getTableName())) { + viewSource.setTableName(metaData.getTableName()); + } + return viewSource; + } + + @Override public String getType() { + return TYPE; + } + + @Override public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { + ViewSink viewSink = (ViewSink) ConfigurableUtil.create(ViewSink.class.getName(), namespace, name, createFormatProperty(properties), null); + if (StringUtil.isEmpty(viewSink.getViewTableName())) { + viewSink.setViewTableName(metaData.getTableName()); + } + return viewSink; + } + + /** + * 创建标准的属性文件 + * + * @param properties + * @return + */ + @Override + 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, "viewTableName", "tableName"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "viewTableName", "viewName"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "viewTableName", "name"); +// IChannelBuilder.formatPropertiesName(formatProperties, properties, "maxThread", "thread.max.count"); + return formatProperties; + } + + @Override + public ISource copy(ISource pipelineSource) { + ViewSource viewSource = (ViewSource) pipelineSource; + if (viewSource.getRootSource() == null) { + return null; + } + ServiceLoaderComponent serviceLoaderComponent = ComponentCreator.getComponent(IChannelBuilder.class.getName(), ServiceLoaderComponent.class); + IChannelBuilder builder = (IChannelBuilder) serviceLoaderComponent.loadService(viewSource.getRootSource().getClass().getSimpleName()); + IShuffleChannelBuilder shuffleChannelBuilder = (IShuffleChannelBuilder) builder; + return shuffleChannelBuilder.copy(((ViewSource) pipelineSource).getRootSource()); + } + + @Override + public ISink createBySource(ISource pipelineSource) { + ViewSource viewSource = (ViewSource) pipelineSource; + if (viewSource.getRootSource() == null) { + return null; + } + ServiceLoaderComponent serviceLoaderComponent = ComponentCreator.getComponent(IChannelBuilder.class.getName(), ServiceLoaderComponent.class); + IChannelBuilder builder = (IChannelBuilder) serviceLoaderComponent.loadService(viewSource.getRootSource().getClass().getSimpleName()); + IShuffleChannelBuilder shuffleChannelBuilder = (IShuffleChannelBuilder) builder; + return shuffleChannelBuilder.createBySource(viewSource.getRootSource()); + } +} diff --git a/rocketmq-streams-channel-view/src/test/java/org/apache/rocketmq/streams/db/sink/es/EsChannelTest.java b/rocketmq-streams-channel-view/src/test/java/org/apache/rocketmq/streams/db/sink/es/EsChannelTest.java new file mode 100644 index 00000000..9795ae5f --- /dev/null +++ b/rocketmq-streams-channel-view/src/test/java/org/apache/rocketmq/streams/db/sink/es/EsChannelTest.java @@ -0,0 +1,43 @@ +/* + * 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.db.sink.es; + +public class EsChannelTest { + String host = "es-cn.elasticsearch.xxxxx.com"; + String port = "9200"; + +// @Test +// public void testEsChannelInsert() { +// JSONObject message = JSONObject.parseObject("{\"data\": \"xxx test message\", \"time\":\"2020-07-25\"}"); +// List messageList = new ArrayList<>(); +// messageList.add(new Message(message)); +// ISink esChannel = createChannel(); +// esChannel.batchSave(messageList); +// } + +// private ISink createChannel() { +// ESSinkOnlyChannel esChannel = new ESSinkOnlyChannel(); +// esChannel.setHost(host); +// esChannel.setPort(port + ""); +// esChannel.setEsIndex("es_index_test"); +// esChannel.setNeedAuth(true); +// esChannel.setAuthUsername("*"); +// esChannel.setAuthPassword("*"); +// esChannel.init(); +// return esChannel; +// } +} diff --git a/rocketmq-streams-checkpoint/src/main/java/org/apache/rocketmq/streams/checkpoint/db/DBCheckPointStorage.java b/rocketmq-streams-checkpoint/src/main/java/org/apache/rocketmq/streams/checkpoint/db/DBCheckPointStorage.java deleted file mode 100644 index 69150acf..00000000 --- a/rocketmq-streams-checkpoint/src/main/java/org/apache/rocketmq/streams/checkpoint/db/DBCheckPointStorage.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * 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.checkpoint.db; - -import java.util.List; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.channel.source.ISource; -import org.apache.rocketmq.streams.common.checkpoint.AbstractCheckPointStorage; -import org.apache.rocketmq.streams.common.checkpoint.CheckPoint; -import org.apache.rocketmq.streams.common.checkpoint.CheckPointManager; -import org.apache.rocketmq.streams.common.checkpoint.SourceSnapShot; -import org.apache.rocketmq.streams.db.driver.orm.ORMUtil; - -/** - * @description - */ -public class DBCheckPointStorage extends AbstractCheckPointStorage { - - static final Log logger = LogFactory.getLog(DBCheckPointStorage.class); - static final String STORAGE_NAME = "DB"; - - public DBCheckPointStorage() { - - } - - @Override - public String getStorageName() { - return STORAGE_NAME; - } - - @Override - public void save(List checkPointState) { - logger.info(String.format("save checkpoint size %d", checkPointState.size())); - ORMUtil.batchReplaceInto(checkPointState); - } - - @Override - public void finish() { - - } - - @Override - //todo - public CheckPoint recover(ISource iSource, String queueId) { - String sourceName = CheckPointManager.createSourceName(iSource, null); - String key = CheckPointManager.createCheckPointKey(sourceName, queueId); - String sql = "select * from source_snap_shot where `key` = " + "'" + key + "';"; - SourceSnapShot snapShot = ORMUtil.queryForObject(sql, null, SourceSnapShot.class); - - logger.info(String.format("checkpoint recover key is %s, sql is %s, recover sourceSnapShot : %s", key, sql, snapShot == null ? "null snapShot" : snapShot.toString())); - return new CheckPoint().fromSnapShot(snapShot); - } -} diff --git a/rocketmq-streams-clients/pom.xml b/rocketmq-streams-clients/pom.xml index 94c7f3e7..0ef3aa17 100644 --- a/rocketmq-streams-clients/pom.xml +++ b/rocketmq-streams-clients/pom.xml @@ -15,13 +15,13 @@ See the License for the specific language governing permissions and limitations under the License. --> - rocketmq-streams org.apache.rocketmq - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT 4.0.0 @@ -30,32 +30,43 @@ org.apache.rocketmq - rocketmq-streams-commons + rocketmq-streams-cep org.apache.rocketmq - rocketmq-streams-channel-rocketmq + rocketmq-streams-channel-db org.apache.rocketmq - rocketmq-streams-channel-mqtt + rocketmq-streams-channel-es + + + org.apache.rocketmq + rocketmq-streams-channel-http - org.apache.rocketmq rocketmq-streams-channel-kafka org.apache.rocketmq - rocketmq-streams-channel-db + rocketmq-streams-channel-metaq org.apache.rocketmq - rocketmq-streams-script + rocketmq-streams-channel-mqtt org.apache.rocketmq - rocketmq-streams-filter + rocketmq-streams-channel-openapi + + + org.apache.rocketmq + rocketmq-streams-channel-rocketmq + + + org.apache.rocketmq + rocketmq-streams-channel-sls org.apache.rocketmq @@ -63,42 +74,40 @@ org.apache.rocketmq - rocketmq-streams-channel-kafka + rocketmq-streams-channel-tencent org.apache.rocketmq - rocketmq-streams-dbinit + rocketmq-streams-channel-huawei org.apache.rocketmq - rocketmq-streams-window + rocketmq-streams-channel-view org.apache.rocketmq - rocketmq-streams-channel-kafka + rocketmq-streams-tasks org.apache.rocketmq - rocketmq-streams-connectors + rocketmq-streams-script - org.slf4j - slf4j-api - 1.7.26 + org.apache.rocketmq + rocketmq-streams-filter - ch.qos.logback - logback-classic + org.apache.rocketmq + rocketmq-streams-db-init - ch.qos.logback - logback-core + org.apache.rocketmq + rocketmq-streams-dim + + + org.apache.rocketmq + rocketmq-streams-window - - 8 - 8 - - \ No newline at end of file diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/AbstractExecutionEnvironment.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/AbstractExecutionEnvironment.java new file mode 100644 index 00000000..b058646a --- /dev/null +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/AbstractExecutionEnvironment.java @@ -0,0 +1,107 @@ +package org.apache.rocketmq.streams.client; + +import java.util.Properties; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.configuration.SystemContext; +import org.apache.rocketmq.streams.common.utils.StringUtil; + +public class AbstractExecutionEnvironment> { + + private final Properties properties; + + protected AbstractExecutionEnvironment() { + this.properties = new Properties(); + this.properties.putAll(SystemContext.getProperties()); + } + + public T db(String url, String userName, String password, String driver) { + this.properties.put(ConfigurationKey.JDBC_URL, url); + this.properties.put(ConfigurationKey.JDBC_USERNAME, userName); + this.properties.put(ConfigurationKey.JDBC_PASSWORD, password); + if (StringUtil.isNotEmpty(driver)) { + this.properties.put(ConfigurationKey.JDBC_DRIVER, driver); + } + + this.properties.put(ConfigurationKey.INTELLIGENCE_JDBC_URL, url); + this.properties.put(ConfigurationKey.INTELLIGENCE_JDBC_USERNAME, userName); + this.properties.put(ConfigurationKey.INTELLIGENCE_JDBC_PASSWORD, password); + return (T) this; + + } + + /** + * 配置情报的非db连接 + */ + public T intelligence(String endpoint, String ak, String sk, String region) { + this.properties.put(ConfigurationKey.INTELLIGENCE_SWITCH, "true"); + this.properties.put(ConfigurationKey.INTELLIGENCE_TIP_DB_ENDPOINT, endpoint); + this.properties.put(ConfigurationKey.INTELLIGENCE_AK, ak); + this.properties.put(ConfigurationKey.INTELLIGENCE_SK, sk); + this.properties.put(ConfigurationKey.INTELLIGENCE_REGION, region); + return (T) this; + } + + /** + * 打开pipeline html monitor + */ + public T dispatcherTime(int timeSecond) { + this.properties.put(ConfigurationKey.DIPPER_DISPATCHER_SCHEDULE_TIME, timeSecond); + return (T) this; + + } + + /** + * 打开pipeline html monitor + */ + public T regexEngine() { + this.properties.put(ConfigurationKey.DIPPER_REGEX_ENGINE, "re2j"); + return (T) this; + + } + + public T miniIo(String endpoint, String ak, String sk, String dir) { + this.properties.put(ConfigurationKey.FILE_TRANSPORT_AK, endpoint); + this.properties.put(ConfigurationKey.FILE_TRANSPORT_SK, ak); + this.properties.put(ConfigurationKey.FILE_TRANSPORT_ENDPOINT, sk); + this.properties.put(ConfigurationKey.FILE_TRANSPORT_DIPPER_DIR, dir); + return (T) this; + } + + /** + * 设置指纹缓存大小 + */ + + public T cache(int rowSize) { + this.properties.put(ConfigurationKey.FINGERPRINT_CACHE_SIZE, rowSize); + return (T) this; + } + + /** + * 设置指纹缓存大小 + */ + + public T monitor(String updateType, String updateTopic, String updateTag, String downTopic, String downTag, String ruleTopic) { + this.properties.put(ConfigurationKey.UPDATE_TYPE, updateType); + this.properties.put(ConfigurationKey.RULE_UP_TOPIC, updateTopic); + this.properties.put(ConfigurationKey.RULE_UP_TAG, updateTag); + this.properties.put(ConfigurationKey.RULE_DOWN_TOPIC, downTopic); + this.properties.put(ConfigurationKey.RULE_DOWN_TAG, downTag); + this.properties.put(ConfigurationKey.RULE_TOPIC_TAG, ruleTopic); + return (T) this; + } + + /** + * 设置指纹缓存大小 + */ + + public T httpRam(String endpoint, String ak, String sk) { + this.properties.put(ConfigurationKey.HTTP_SERVICE_ENDPOINT, endpoint); + this.properties.put(ConfigurationKey.HTTP_AK, ak); + this.properties.put(ConfigurationKey.HTTP_SK, sk); + return (T) this; + } + + public Properties getProperties() { + return properties; + } +} diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/ScheduledStreamBuilder.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/ScheduledStreamBuilder.java deleted file mode 100644 index af7d5cdb..00000000 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/ScheduledStreamBuilder.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.client; - -import java.util.Arrays; -import java.util.Set; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import org.apache.commons.lang3.concurrent.BasicThreadFactory; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.utils.ThreadUtil; - -/** - * @description - */ -public class ScheduledStreamBuilder { - - static final Log logger = LogFactory.getLog(ScheduledStreamBuilder.class); - - protected ScheduledExecutorService balanceExecutor; - - TimeUnit timeUnit; - - int interval; - - ScheduledTask task; - - public ScheduledStreamBuilder(int interval, TimeUnit timeUnit) { - this.interval = interval; - this.timeUnit = timeUnit; - balanceExecutor = new ScheduledThreadPoolExecutor(1, new BasicThreadFactory.Builder().namingPattern("cycle-builder-task-%d").daemon(true).build()); - } - - public void setTask(ScheduledTask task) { - this.task = task; - } - - public void start() { - balanceExecutor.scheduleAtFixedRate(task, 0, interval, timeUnit); - - while (true) { - Set threadSet = Thread.getAllStackTraces().keySet(); - for (Thread th : threadSet) { - logger.error(String.format("CycleStreamBuilder size %d, name is %s, stack is %s. ", threadSet.size(), th.getName(), Arrays.toString(th.getStackTrace()))); - } - - ThreadUtil.sleep(10000); - } - } -} diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/ScheduledTask.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/ScheduledTask.java deleted file mode 100644 index 1af571c7..00000000 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/ScheduledTask.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * 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.client; - -import java.util.Date; -import org.apache.rocketmq.streams.client.source.DataStreamSource; -import org.apache.rocketmq.streams.client.transform.DataStream; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; -import org.apache.rocketmq.streams.connectors.source.filter.CycleSchedule; -import org.apache.rocketmq.streams.db.sink.EnhanceDBSink; - -/** - * @description - */ -public class ScheduledTask implements Runnable { - - CycleSchedule schedule; - String sinkTableName; - String sourceTableName; - String url; - String userName; - String password; - - public ScheduledTask(String expression, String url, String userName, String password, String sourceTableName, - String sinkTableName) { - schedule = CycleSchedule.getInstance(expression, new Date()); - this.sourceTableName = sourceTableName; - this.sinkTableName = sinkTableName; - this.url = url; - this.userName = userName; - this.password = password; - ComponentCreator.getProperties().put(ConfigureFileKey.CHECKPOINT_STORAGE_NAME, "db"); - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_URL, url);//数据库连接url - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_USERNAME, userName);//用户名 - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_PASSWORD, password);//password - ComponentCreator.getProperties().put(ConfigureFileKey.IS_ATOMIC_DB_SINK, "true"); - - } - - @Override - public void run() { - CycleSchedule.Cycle cycle = schedule.nextCycle(new Date()); - DataStreamSource dataStreamsource = StreamBuilder.dataStream("test_baseline" + "_" + cycle.getCycleDateStr(), "baseline_pipeline"); - DataStream datastream = dataStreamsource.fromCycleSource(url, userName, password, sourceTableName, cycle, 3); - EnhanceDBSink sink = new EnhanceDBSink(); - sink.setAtomic(true); - sink.setTableName(sinkTableName); - sink.setUrl(url); - sink.setUserName(userName); - sink.setPassword(password); - sink.init(); - datastream.to(sink).start(true); - } -} diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/StreamExecutionEnvironment.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/StreamExecutionEnvironment.java new file mode 100644 index 00000000..69376004 --- /dev/null +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/StreamExecutionEnvironment.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.client; + +import java.util.Properties; +import org.apache.rocketmq.streams.client.source.DataStreamSource; +import org.apache.rocketmq.streams.common.configuration.JobConfiguration; + +public class StreamExecutionEnvironment extends AbstractExecutionEnvironment { + + private StreamExecutionEnvironment() { + super(); + } + + public static StreamExecutionEnvironment getExecutionEnvironment() { + return StreamExecutionEnvironmentFactory.Instance; + } + + public DataStreamSource create(String namespace, String jobName, JobConfiguration jobConfiguration) { + Properties properties = this.getProperties(); + properties.putAll(jobConfiguration.getProperties()); + return new DataStreamSource(namespace, jobName, properties); + } + + public DataStreamSource create(String namespace, String jobName) { + return new DataStreamSource(namespace, jobName, this.getProperties()); + } + + private static class StreamExecutionEnvironmentFactory { + private static final StreamExecutionEnvironment Instance = new StreamExecutionEnvironment(); + } + +} diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/source/DataStreamSource.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/source/DataStreamSource.java index 5004f230..8d8ed41e 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/source/DataStreamSource.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/source/DataStreamSource.java @@ -35,7 +35,7 @@ import com.alibaba.fastjson.JSONObject; import com.google.common.collect.Sets; -import java.util.Set; +import java.util.Properties; import org.apache.rocketmq.streams.client.transform.DataStream; import org.apache.rocketmq.streams.common.channel.impl.CollectionSource; import org.apache.rocketmq.streams.common.channel.impl.file.FileSource; @@ -43,28 +43,24 @@ import org.apache.rocketmq.streams.common.channel.impl.memory.MemorySource; import org.apache.rocketmq.streams.common.channel.source.ISource; import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; -import org.apache.rocketmq.streams.connectors.source.CycleDynamicMultipleDBScanSource; -import org.apache.rocketmq.streams.connectors.source.DynamicMultipleDBScanSource; +import org.apache.rocketmq.streams.db.source.CycleDynamicMultipleDBScanSource; +import org.apache.rocketmq.streams.db.source.DynamicMultipleDBScanSource; import org.apache.rocketmq.streams.connectors.source.filter.CycleSchedule; import org.apache.rocketmq.streams.kafka.source.KafkaSource; import org.apache.rocketmq.streams.mqtt.source.PahoSource; -import org.apache.rocketmq.streams.source.RocketMQSource; +import org.apache.rocketmq.streams.rocketmq.source.RocketMQSource; +import org.apache.rocketmq.streams.sls.source.SLSSource; public class DataStreamSource { - protected PipelineBuilder mainPipelineBuilder; - protected Set otherPipelineBuilders; - public DataStreamSource(String namespace, String pipelineName) { - this.mainPipelineBuilder = new PipelineBuilder(namespace, pipelineName); - this.otherPipelineBuilders = Sets.newHashSet(); - } - - public static DataStreamSource create(String namespace, String pipelineName) { - return new DataStreamSource(namespace, pipelineName); - } + private final String namespace; + private final String jobName; + private final Properties properties; - public static DataStreamSource create(String namespace, String pipelineName, String[] duplicateKeys, Long windowSize) { - return new DataStreamSource(namespace, pipelineName); + public DataStreamSource(String namespace, String jobName, Properties properties) { + this.namespace = namespace; + this.jobName = jobName; + this.properties = properties; } public DataStream fromArray(Object[] o) { @@ -74,41 +70,67 @@ public DataStream fromArray(Object[] o) { public DataStream fromMemory(MemoryCache memoryCache, boolean isJson) { MemorySource memorySource = new MemorySource(); - this.mainPipelineBuilder.addConfigurables(memoryCache); memorySource.setMemoryCache(memoryCache); memorySource.setJsonData(isJson); - this.mainPipelineBuilder.setSource(memorySource); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, null); + PipelineBuilder rootPipelineBuilder = new PipelineBuilder(namespace, jobName, properties); + rootPipelineBuilder.addConfigurables(memoryCache); + rootPipelineBuilder.setSource(memorySource); + return new DataStream(rootPipelineBuilder, Sets.newHashSet(), null); } public DataStream fromFile(String filePath) { return fromFile(filePath, true); } + public DataStream fromCSVFile(String filePath) { + FileSource fileChannel = new FileSource(filePath); + fileChannel.setCSV(true); + PipelineBuilder rootPipelineBuilder = new PipelineBuilder(namespace, jobName, properties); + rootPipelineBuilder.setSource(fileChannel); + return new DataStream(rootPipelineBuilder, Sets.newHashSet(), null); + } + public DataStream fromFile(String filePath, Boolean isJsonData) { FileSource fileChannel = new FileSource(filePath); fileChannel.setJsonData(isJsonData); - this.mainPipelineBuilder.setSource(fileChannel); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, null); + PipelineBuilder rootPipelineBuilder = new PipelineBuilder(namespace, jobName, properties); + rootPipelineBuilder.setSource(fileChannel); + return new DataStream(rootPipelineBuilder, Sets.newHashSet(), null); } - public DataStream fromRocketmq(String topic, String groupName, String namesrvAddress) { - return fromRocketmq(topic, groupName, false, namesrvAddress); + public DataStream fromSls(String endPoint, String project, String logStore, String ak, String sk, String groupName) { + SLSSource slsSource = new SLSSource(endPoint, project, logStore, ak, sk, groupName); + slsSource.setJsonData(true); + PipelineBuilder rootPipelineBuilder = new PipelineBuilder(namespace, jobName, properties); + rootPipelineBuilder.setSource(slsSource); + return new DataStream(rootPipelineBuilder, Sets.newHashSet(), null); } - public DataStream fromRocketmq(String topic, String groupName, boolean isJson, String namesrvAddress) { - return fromRocketmq(topic, groupName, "*", isJson, namesrvAddress); + public DataStream fromRocketmq(String topic, String groupName, String nameServer) { + return fromRocketmq(topic, groupName, false, nameServer); } - public DataStream fromRocketmq(String topic, String groupName, String tags, boolean isJson, String namesrvAddress) { + public DataStream fromRocketmq(String topic, String groupName, boolean isJson, String nameServer) { + return fromRocketmq(topic, groupName, "*", isJson, nameServer); + } + + public DataStream fromRocketmq(String topic, String groupName, String tags, boolean isJson, String nameServer) { + return fromRocketmq(topic, groupName, tags, isJson, nameServer, false); + } + + public DataStream fromRocketmq(String topic, String groupName, String tags, boolean isJson, String nameServer, boolean isMessageListenerConcurrently) { + RocketMQSource rocketMQSource = new RocketMQSource(); rocketMQSource.setTopic(topic); rocketMQSource.setTags(tags); rocketMQSource.setGroupName(groupName); rocketMQSource.setJsonData(isJson); - rocketMQSource.setNamesrvAddr(namesrvAddress); - this.mainPipelineBuilder.setSource(rocketMQSource); - return new DataStream(this.mainPipelineBuilder, null); + rocketMQSource.setNamesrvAddr(nameServer); + rocketMQSource.setMessageListenerConcurrently(isMessageListenerConcurrently); + + PipelineBuilder rootPipelineBuilder = new PipelineBuilder(namespace, jobName, properties); + rootPipelineBuilder.setSource(rocketMQSource); + return new DataStream(rootPipelineBuilder, Sets.newHashSet(), null); } public DataStream fromMultipleDB(String url, String userName, String password, String tablePattern) { @@ -118,65 +140,81 @@ public DataStream fromMultipleDB(String url, String userName, String password, S source.setPassword(password); source.setBatchSize(10); source.setLogicTableName(tablePattern); - this.mainPipelineBuilder.setSource(source); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, null); + + PipelineBuilder rootPipelineBuilder = new PipelineBuilder(namespace, jobName, properties); + rootPipelineBuilder.setSource(source); + return new DataStream(rootPipelineBuilder, Sets.newHashSet(), null); } - public DataStream fromCycleSource(String url, String userName, String password, String tablePattern, - CycleSchedule.Cycle cycle, int balanceSec) { + public DataStream fromCycleSource(String url, String userName, String password, String tablePattern, CycleSchedule.Cycle cycle, int balanceSec) { CycleDynamicMultipleDBScanSource source = new CycleDynamicMultipleDBScanSource(cycle); source.setUrl(url); source.setUserName(userName); source.setPassword(password); source.setBatchSize(10); source.setLogicTableName(tablePattern); - source.setBalanceTimeSecond(balanceSec); - - this.mainPipelineBuilder.setSource(source); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, null); + PipelineBuilder rootPipelineBuilder = new PipelineBuilder(namespace, jobName, properties); + rootPipelineBuilder.setSource(source); + return new DataStream(rootPipelineBuilder, Sets.newHashSet(), null); } public DataStream fromCollection(JSONObject... elements) { CollectionSource source = new CollectionSource(); source.addAll(elements); - this.mainPipelineBuilder.setSource(source); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, null); + + PipelineBuilder rootPipelineBuilder = new PipelineBuilder(namespace, jobName, properties); + rootPipelineBuilder.setSource(source); + return new DataStream(rootPipelineBuilder, Sets.newHashSet(), null); } public DataStream fromMqtt(String url, String clientId, String topic) { PahoSource mqttSource = new PahoSource(url, clientId, topic); mqttSource.setJsonData(true); - this.mainPipelineBuilder.setSource(mqttSource); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, null); + + PipelineBuilder rootPipelineBuilder = new PipelineBuilder(namespace, jobName, properties); + rootPipelineBuilder.setSource(mqttSource); + return new DataStream(rootPipelineBuilder, Sets.newHashSet(), null); } public DataStream fromMqtt(String url, String clientId, String topic, String username, String password) { PahoSource mqttSource = new PahoSource(url, clientId, topic, username, password); mqttSource.setJsonData(true); - this.mainPipelineBuilder.setSource(mqttSource); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, null); + + PipelineBuilder rootPipelineBuilder = new PipelineBuilder(namespace, jobName, properties); + rootPipelineBuilder.setSource(mqttSource); + return new DataStream(rootPipelineBuilder, Sets.newHashSet(), null); + } + + public DataStream fromMqttForJsonArray(String url, String clientId, String topic, String username, String password) { + PahoSource mqttSource = new PahoSource(url, clientId, topic, username, password); + mqttSource.setJsonData(true); + mqttSource.setMsgIsJsonArray(true); + + PipelineBuilder rootPipelineBuilder = new PipelineBuilder(namespace, jobName, properties); + rootPipelineBuilder.setSource(mqttSource); + return new DataStream(rootPipelineBuilder, Sets.newHashSet(), null); } - public DataStream fromMqtt(String url, String clientId, String topic, String username, String password, - Boolean cleanSession, Integer connectionTimeout, Integer aliveInterval, Boolean automaticReconnect) { + public DataStream fromMqtt(String url, String clientId, String topic, String username, String password, Boolean cleanSession, Integer connectionTimeout, Integer aliveInterval, Boolean automaticReconnect) { return fromMqtt(url, clientId, topic, username, password, cleanSession, connectionTimeout, aliveInterval, automaticReconnect, true); } - public DataStream fromMqtt(String url, String clientId, String topic, String username, String password, - Boolean cleanSession, Integer connectionTimeout, Integer aliveInterval, Boolean automaticReconnect, - Boolean jsonData) { + public DataStream fromMqtt(String url, String clientId, String topic, String username, String password, Boolean cleanSession, Integer connectionTimeout, Integer aliveInterval, Boolean automaticReconnect, Boolean jsonData) { PahoSource mqttSource = new PahoSource(url, clientId, topic, username, password, cleanSession, connectionTimeout, aliveInterval, automaticReconnect); mqttSource.setJsonData(jsonData); - this.mainPipelineBuilder.setSource(mqttSource); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, null); + + PipelineBuilder rootPipelineBuilder = new PipelineBuilder(namespace, jobName, properties); + rootPipelineBuilder.setSource(mqttSource); + return new DataStream(rootPipelineBuilder, Sets.newHashSet(), null); } - public DataStream fromMqtt(String url, String clientId, String topic, String username, String password, - Boolean jsonData) { + public DataStream fromMqtt(String url, String clientId, String topic, String username, String password, Boolean jsonData) { PahoSource mqttSource = new PahoSource(url, clientId, topic, username, password); mqttSource.setJsonData(jsonData); - this.mainPipelineBuilder.setSource(mqttSource); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, null); + + PipelineBuilder rootPipelineBuilder = new PipelineBuilder(namespace, jobName, properties); + rootPipelineBuilder.setSource(mqttSource); + return new DataStream(rootPipelineBuilder, Sets.newHashSet(), null); } public DataStream fromKafka(String endpoint, String topic, String groupName) { @@ -194,13 +232,16 @@ public DataStream fromKafka(String endpoint, String topic, String groupName, Boo kafkaChannel.setGroupName(groupName); kafkaChannel.setJsonData(isJson); kafkaChannel.setMaxThread(maxThread); - this.mainPipelineBuilder.setSource(kafkaChannel); - return new DataStream(this.mainPipelineBuilder, null); + + PipelineBuilder rootPipelineBuilder = new PipelineBuilder(namespace, jobName, properties); + rootPipelineBuilder.setSource(kafkaChannel); + return new DataStream(rootPipelineBuilder, Sets.newHashSet(), null); } public DataStream from(ISource source) { - this.mainPipelineBuilder.setSource(source); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, null); + PipelineBuilder rootPipelineBuilder = new PipelineBuilder(namespace, jobName, properties); + rootPipelineBuilder.setSource(source); + return new DataStream(rootPipelineBuilder, Sets.newHashSet(), null); } } diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/strategy/LogFingerprintStrategy.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/strategy/LogFingerprintStrategy.java index 4072e608..853b161f 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/strategy/LogFingerprintStrategy.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/strategy/LogFingerprintStrategy.java @@ -17,13 +17,12 @@ package org.apache.rocketmq.streams.client.strategy; import java.util.Properties; -import org.apache.rocketmq.streams.common.component.AbstractComponent; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; public class LogFingerprintStrategy implements Strategy { - protected String[] logFingerprintFields; - private final Properties properties; + protected String[] logFingerprintFields; private LogFingerprintStrategy() { properties = new Properties(); @@ -37,11 +36,11 @@ private LogFingerprintStrategy(String... fieldNames) { private LogFingerprintStrategy(String url, String username, String password) { 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, AbstractComponent.DEFAULT_JDBC_TABLE_NAME); + properties.put(ConfigurationKey.JDBC_DRIVER, ConfigurationKey.DEFAULT_JDBC_DRIVER); + properties.put(ConfigurationKey.JDBC_URL, url); + properties.put(ConfigurationKey.JDBC_USERNAME, username); + properties.put(ConfigurationKey.JDBC_PASSWORD, password); + // properties.put(AbstractComponent.JDBC_TABLE_NAME, AbstractComponent.DEFAULT_JDBC_TABLE_NAME); } public static Strategy configLogFingerprint(String... fieldNames) { diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/strategy/ShuffleStrategy.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/strategy/ShuffleStrategy.java index a6b1742d..b617b82b 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/strategy/ShuffleStrategy.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/strategy/ShuffleStrategy.java @@ -16,9 +16,8 @@ */ package org.apache.rocketmq.streams.client.strategy; -import org.apache.rocketmq.streams.common.component.AbstractComponent; - import java.util.Properties; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; public class ShuffleStrategy implements Strategy { @@ -26,18 +25,17 @@ public class ShuffleStrategy implements Strategy { private ShuffleStrategy(String windowShuffleType) { properties = new Properties(); - properties.put(AbstractComponent.WINDOW_SHUFFLE_CHANNEL_TYPE, windowShuffleType); - } - - @Override - public Properties getStrategyProperties() { - return this.properties; + properties.put(ConfigurationKey.WINDOW_SHUFFLE_CHANNEL_TYPE, windowShuffleType); } public static Strategy shuffleWithMemory() { return new ShuffleStrategy("memory"); } + @Override public Properties getStrategyProperties() { + return this.properties; + } + } diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/strategy/WindowStrategy.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/strategy/WindowStrategy.java index c111f0b6..ee4777a7 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/strategy/WindowStrategy.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/strategy/WindowStrategy.java @@ -17,9 +17,7 @@ package org.apache.rocketmq.streams.client.strategy; import java.util.Properties; -import org.apache.rocketmq.streams.common.component.AbstractComponent; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; public class WindowStrategy implements Strategy { @@ -31,16 +29,11 @@ private WindowStrategy() { private WindowStrategy(String url, String username, String password) { 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, AbstractComponent.DEFAULT_JDBC_TABLE_NAME); - } - - @Override - public Properties getStrategyProperties() { - return this.properties; + properties.put(ConfigurationKey.JDBC_DRIVER, ConfigurationKey.DEFAULT_JDBC_DRIVER); + properties.put(ConfigurationKey.JDBC_URL, url); + properties.put(ConfigurationKey.JDBC_USERNAME, username); + properties.put(ConfigurationKey.JDBC_PASSWORD, password); + // properties.put(AbstractComponent.JDBC_TABLE_NAME, AbstractComponent.DEFAULT_JDBC_TABLE_NAME); } public static Strategy exactlyOnce(String url, String username, String password) { @@ -52,9 +45,14 @@ public static Strategy highPerformance() { return new WindowStrategy(); } - public static Strategy windowDefaultSiZe(int defualtSize) { - ComponentCreator.getProperties().put(ConfigureFileKey.DIPPER_WINDOW_DEFAULT_INERVAL_SIZE, defualtSize); + public static Strategy windowDefaultSiZe(int defaultSize) { +// ComponentCreator.getProperties().put(ConfigureFileKey.DIPPER_WINDOW_DEFAULT_INERVAL_SIZE, defualtSize); return null; } + @Override + public Properties getStrategyProperties() { + return this.properties; + } + } diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/DataStream.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/DataStream.java index 96e24a0a..83ad03f8 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/DataStream.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/DataStream.java @@ -18,24 +18,23 @@ package org.apache.rocketmq.streams.client.transform; import com.alibaba.fastjson.JSONObject; -import com.google.common.collect.Sets; import java.io.Serializable; import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Properties; import java.util.Set; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.streams.client.strategy.LogFingerprintStrategy; import org.apache.rocketmq.streams.client.strategy.Strategy; +import org.apache.rocketmq.streams.client.transform.window.Time; +import org.apache.rocketmq.streams.client.transform.window.TumblingWindow; import org.apache.rocketmq.streams.client.transform.window.WindowInfo; import org.apache.rocketmq.streams.common.channel.impl.OutputPrintChannel; import org.apache.rocketmq.streams.common.channel.impl.file.FileSink; import org.apache.rocketmq.streams.common.channel.sink.AbstractSink; import org.apache.rocketmq.streams.common.channel.source.AbstractSource; import org.apache.rocketmq.streams.common.channel.source.ISource; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.Message; @@ -47,20 +46,17 @@ import org.apache.rocketmq.streams.common.functions.ForEachMessageFunction; import org.apache.rocketmq.streams.common.functions.MapFunction; import org.apache.rocketmq.streams.common.functions.SplitFunction; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; -import org.apache.rocketmq.streams.common.topology.ChainStage; -import org.apache.rocketmq.streams.common.topology.builder.IStageBuilder; +import org.apache.rocketmq.streams.common.topology.IStageBuilder; import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; -import org.apache.rocketmq.streams.common.topology.model.AbstractRule; +import org.apache.rocketmq.streams.common.topology.model.AbstractChainStage; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; import org.apache.rocketmq.streams.common.topology.model.Union; import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; -import org.apache.rocketmq.streams.common.topology.stages.ShuffleConsumerChainStage; -import org.apache.rocketmq.streams.common.topology.stages.ShuffleProducerChainStage; +import org.apache.rocketmq.streams.common.topology.stages.ShuffleChainStage; import org.apache.rocketmq.streams.common.topology.stages.udf.StageBuilder; import org.apache.rocketmq.streams.common.topology.stages.udf.UDFChainStage; import org.apache.rocketmq.streams.common.topology.stages.udf.UDFUnionChainStage; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; -import org.apache.rocketmq.streams.configurable.ConfigurableComponent; import org.apache.rocketmq.streams.db.sink.DBSink; import org.apache.rocketmq.streams.db.sink.DynamicMultipleDBSink; import org.apache.rocketmq.streams.db.sink.EnhanceDBSink; @@ -70,8 +66,8 @@ import org.apache.rocketmq.streams.filter.operator.Rule; import org.apache.rocketmq.streams.kafka.sink.KafkaSink; import org.apache.rocketmq.streams.mqtt.sink.PahoSink; +import org.apache.rocketmq.streams.rocketmq.sink.RocketMQSink; import org.apache.rocketmq.streams.script.operator.impl.ScriptOperator; -import org.apache.rocketmq.streams.sink.RocketMQSink; import org.apache.rocketmq.streams.window.builder.WindowBuilder; import org.apache.rocketmq.streams.window.operator.AbstractWindow; import org.apache.rocketmq.streams.window.operator.impl.OverWindow; @@ -80,52 +76,42 @@ public class DataStream implements Serializable { - protected PipelineBuilder mainPipelineBuilder; + protected PipelineBuilder rootPipelineBuilder; protected Set otherPipelineBuilders; - protected ChainStage currentChainStage; + protected AbstractChainStage currentChainStage; - public DataStream(String namespace, String pipelineName) { - this.mainPipelineBuilder = new PipelineBuilder(namespace, pipelineName); - this.otherPipelineBuilders = Sets.newHashSet(); - } - - public DataStream(PipelineBuilder pipelineBuilder, ChainStage currentChainStage) { - this.mainPipelineBuilder = pipelineBuilder; - this.otherPipelineBuilders = Sets.newHashSet(); - this.currentChainStage = currentChainStage; - } - - public DataStream(PipelineBuilder pipelineBuilder, Set pipelineBuilders, - ChainStage currentChainStage) { - this.mainPipelineBuilder = pipelineBuilder; + public DataStream(PipelineBuilder pipelineBuilder, Set pipelineBuilders, AbstractChainStage currentChainStage) { + this.rootPipelineBuilder = pipelineBuilder; this.otherPipelineBuilders = pipelineBuilders; this.currentChainStage = currentChainStage; } - - public DataStream increaseConcurrencyByShuffle(int shuffleConcurrentCount){ - if(this.mainPipelineBuilder.getPipeline().getChannelNextStageLabel().size()>0||currentChainStage!=null){ - throw new RuntimeException("can only set after source"); + /** + * 启动流任务 + */ + public void start() { + if (this.rootPipelineBuilder == null) { + return; + } + ChainPipeline pipeline = this.rootPipelineBuilder.build(); + pipeline.startJob(); + for (PipelineBuilder builder : this.otherPipelineBuilders) { + ChainPipeline otherPipeline = builder.build(); + otherPipeline.startJob(); + } + while (true) { + try { + Thread.sleep(10000); + } catch (InterruptedException e) { + e.printStackTrace(); + } } - ChainStage stage = new ShuffleProducerChainStage(); - ((ShuffleProducerChainStage) stage).setShuffleOwnerName(MapKeyUtil.createKey(this.mainPipelineBuilder.getPipelineNameSpace(),this.mainPipelineBuilder.getPipelineName(),this.mainPipelineBuilder.getPipeline().getChannelName())); - ((ShuffleProducerChainStage) stage).setSplitCount(shuffleConcurrentCount); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, stage); - - this.currentChainStage = stage; - - stage = new ShuffleConsumerChainStage<>(); - ((ShuffleConsumerChainStage) stage).setShuffleOwnerName(MapKeyUtil.createKey(this.mainPipelineBuilder.getPipelineNameSpace(),this.mainPipelineBuilder.getPipelineName(),this.mainPipelineBuilder.getPipeline().getChannelName())); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, stage); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, stage); } - public DataStream with(Strategy... strategies) { - Properties properties = new Properties(); for (Strategy strategy : strategies) { if (strategy instanceof LogFingerprintStrategy) { - ISource source = this.mainPipelineBuilder.getPipeline().getSource(); + ISource source = this.rootPipelineBuilder.getPipeline().getSource(); if (source instanceof AbstractSource) { AbstractSource abstractSource = (AbstractSource) source; String[] logFingerprintFields = ((LogFingerprintStrategy) strategy).getLogFingerprintFields(); @@ -136,37 +122,29 @@ public DataStream with(Strategy... strategies) { } } } - properties.putAll(strategy.getStrategyProperties()); } - ComponentCreator.createProperties(properties); return this; } public DataStream script(String script) { - ChainStage stage = this.mainPipelineBuilder.createStage(new ScriptOperator(script)); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, stage); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, stage); + AbstractChainStage stage = this.rootPipelineBuilder.createStage(new ScriptOperator(script)); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, stage); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, stage); } public DataStream filterByExpression(String expression, String... logFingerFieldNames) { - return filterByExpression(expression, false, logFingerFieldNames); - } - - public DataStream filterByExpression(String expression, boolean openHyperscan, String... logFingerFieldNames) { Rule rule = new FilterOperator(expression); - FilterChainStage stage = (FilterChainStage) this.mainPipelineBuilder.createStage(rule); + FilterChainStage stage = (FilterChainStage) this.rootPipelineBuilder.createStage(rule); if (logFingerFieldNames != null && logFingerFieldNames.length > 0) { stage.setFilterFieldNames(MapKeyUtil.createKeyBySign(",", logFingerFieldNames)); } - stage.setOpenHyperscan(openHyperscan); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, stage); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, stage); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, stage); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, stage); } public DataStream map(MapFunction mapFunction) { StageBuilder stageBuilder = new StageBuilder() { - @Override - protected T operate(IMessage message, AbstractContext context) { + @Override protected IMessage handleMessage(IMessage message, AbstractContext context) { try { O o = (O) (message.getMessageValue()); T result = (T) mapFunction.map(o); @@ -177,22 +155,21 @@ protected T operate(IMessage message, AbstractContext context) { message.setMessageBody(new UserDefinedMessage(result)); } } - return null; + return message; } catch (Exception e) { e.printStackTrace(); } return null; } }; - ChainStage stage = this.mainPipelineBuilder.createStage(stageBuilder); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, stage); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, stage); + AbstractChainStage stage = this.rootPipelineBuilder.createStage(stageBuilder); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, stage); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, stage); } public DataStream flatMap(FlatMapFunction mapFunction) { StageBuilder stageBuilder = new StageBuilder() { - @Override - protected T operate(IMessage message, AbstractContext context) { + @Override protected IMessage handleMessage(IMessage message, AbstractContext context) { try { O o = (O) (message.getMessageValue()); List result = (List) mapFunction.flatMap(o); @@ -215,12 +192,13 @@ protected T operate(IMessage message, AbstractContext context) { } catch (Exception e) { e.printStackTrace(); } - return null; + return message; } + }; - ChainStage stage = this.mainPipelineBuilder.createStage(stageBuilder); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, stage); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, stage); + AbstractChainStage stage = this.rootPipelineBuilder.createStage(stageBuilder); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, stage); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, stage); } public DataStream filter(final FilterFunction filterFunction) { @@ -230,8 +208,7 @@ public DataStream filter(final FilterFunction filterFunction) { public DataStream filter(final FilterFunction filterFunction, String... logFingerFieldNames) { StageBuilder mapUDFOperator = new StageBuilder() { - @Override - protected T operate(IMessage message, AbstractContext context) { + @Override protected IMessage handleMessage(IMessage message, AbstractContext context) { try { boolean isMatch = filterFunction.filter((O) message.getMessageValue()); if (!isMatch) { @@ -241,15 +218,93 @@ protected T operate(IMessage message, AbstractContext context) { } catch (Exception e) { e.printStackTrace(); } - return null; + return message; } + }; - UDFChainStage stage = (UDFChainStage) this.mainPipelineBuilder.createStage(mapUDFOperator); + UDFChainStage stage = (UDFChainStage) this.rootPipelineBuilder.createStage(mapUDFOperator); if (logFingerFieldNames != null && logFingerFieldNames.length > 0) { stage.setFilterFieldNames(MapKeyUtil.createKeyBySign(",", logFingerFieldNames)); } - this.mainPipelineBuilder.setTopologyStages(currentChainStage, stage); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, stage); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, stage); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, stage); + } + + public DataStream count() { + return count(-1); + } + + public DataStream count(int emitSecond) { + WindowInfo windowInfo = TumblingWindow.of(Time.seconds(10)); + WindowStream windowStream = window(windowInfo); + windowStream.count("_count"); + if (emitSecond > 0) { + windowStream.emitBeforeFire(Time.seconds(emitSecond)); + } + return windowStream.toDataSteam(); + } + + public DataStream sum(String fieldName) { + return sum(fieldName, -1); + } + + public DataStream sum(String fieldName, int emitSecond) { + WindowInfo windowInfo = TumblingWindow.of(Time.seconds(10)); + WindowStream windowStream = window(windowInfo); + windowStream.sum(fieldName, fieldName); + if (emitSecond > 0) { + windowStream.emitBeforeFire(Time.seconds(emitSecond)); + } + return windowStream.toDataSteam(); + } + + public DataStream avg(String fieldName) { + return avg(fieldName, -1); + } + + public DataStream avg(String fieldName, int emitSecond) { + WindowInfo windowInfo = TumblingWindow.of(Time.seconds(10)); + WindowStream windowStream = window(windowInfo); + windowStream.avg(fieldName, fieldName); + if (emitSecond > 0) { + windowStream.emitBeforeFire(Time.seconds(emitSecond)); + } + return windowStream.toDataSteam(); + } + + public DataStream min(String fieldName) { + return min(fieldName, -1); + } + + public DataStream min(String fieldName, int emitSecond) { + WindowInfo windowInfo = TumblingWindow.of(Time.seconds(10)); + WindowStream windowStream = window(windowInfo); + windowStream.min(fieldName); + if (emitSecond > 0) { + windowStream.emitBeforeFire(Time.seconds(emitSecond)); + } + return windowStream.toDataSteam(); + } + + public DataStream max(String fieldName) { + return max(fieldName, -1); + } + + public DataStream max(String fieldName, int emitSecond) { + WindowInfo windowInfo = TumblingWindow.of(Time.seconds(10)); + WindowStream windowStream = window(windowInfo); + windowStream.max(fieldName); + if (emitSecond > 0) { + windowStream.emitBeforeFire(Time.seconds(emitSecond)); + } + return windowStream.toDataSteam(); + } + + public WindowStream groupBy(String... fieldName) { + WindowInfo windowInfo = TumblingWindow.of(Time.hours(1)); + WindowStream windowStream = window(windowInfo); + windowStream.groupBy(fieldName); + return windowStream; } /** @@ -260,9 +315,21 @@ protected T operate(IMessage message, AbstractContext context) { */ public WindowStream window(WindowInfo windowInfo) { AbstractWindow window = windowInfo.createWindow(); - ChainStage stage = this.mainPipelineBuilder.createStage(window); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, stage); - return new WindowStream(window, this.mainPipelineBuilder, this.otherPipelineBuilders, stage); + AbstractChainStage stage = this.rootPipelineBuilder.createStage(window); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, stage); + if (stage instanceof ShuffleChainStage) { + this.rootPipelineBuilder.getPipeline().getStages().remove(stage); + + ShuffleChainStage shuffleChainStage = (ShuffleChainStage) stage; + this.rootPipelineBuilder.addChainStage(shuffleChainStage.getOutputChainStage()); + this.rootPipelineBuilder.addChainStage(shuffleChainStage.getConsumeChainStage()); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, shuffleChainStage.getOutputChainStage()); + stage = shuffleChainStage.getConsumeChainStage(); + } else { + this.rootPipelineBuilder.setTopologyStages(currentChainStage, stage); + } + + return new WindowStream(window, this.rootPipelineBuilder, this.otherPipelineBuilders, stage); } public DataStream distinct(String... groupByFieldNames) { @@ -288,9 +355,9 @@ public DataStream distinct(int windowSizeSecond, String... groupByFieldNames) { for (String fieldName : groupByFieldNames) { window.getSelectMap().put(fieldName, fieldName); } - ChainStage stage = this.mainPipelineBuilder.createStage(window); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, stage); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, stage); + AbstractChainStage stage = this.rootPipelineBuilder.createStage(window); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, stage); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, stage); } /** @@ -303,9 +370,9 @@ public OverWindowStream topN(String asRowNumFieldName, int topN, String... group ShuffleOverWindow window = new ShuffleOverWindow(); window.setTopN(topN); window.setRowNumerName(asRowNumFieldName); - ChainStage stage = this.mainPipelineBuilder.createStage(window); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, stage); - OverWindowStream overWindowStream = new OverWindowStream(window, this.mainPipelineBuilder, this.otherPipelineBuilders, stage); + AbstractChainStage stage = this.rootPipelineBuilder.createStage(window); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, stage); + OverWindowStream overWindowStream = new OverWindowStream(window, this.rootPipelineBuilder, this.otherPipelineBuilders, stage); overWindowStream.groupBy(groupByFieldNames); return overWindowStream; } @@ -317,9 +384,9 @@ public OverWindowStream topN(String asRowNumFieldName, int topN, String... group * @return DataStream */ public DataStream addStage(IStageBuilder stageBuilder) { - ChainStage stage = this.mainPipelineBuilder.createStage(stageBuilder); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, stage); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, stage); + AbstractChainStage stage = this.rootPipelineBuilder.createStage(stageBuilder); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, stage); + return this; } /** @@ -338,19 +405,42 @@ public JoinStream leftJoin(DataStream rightStream) { protected JoinStream join(DataStream rightStream, JoinStream.JoinType joinType) { JoinWindow window = WindowBuilder.createDefaultJoinWindow(); - //处理左边分支,增加map,主要是增加msg msgRouteFromLable->增加窗口stage - ChainStage leftScriptStage = this.mainPipelineBuilder.createStage(new ScriptOperator("setHeader(msgRouteFromLable,'" + MessageHeader.JOIN_LEFT + "')")); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, leftScriptStage); + //处理左边分支,增加map,主要是增加msg msgRouteFromLabel->增加窗口stage + AbstractChainStage leftScriptStage = this.rootPipelineBuilder.createStage(new ScriptOperator("setHeader(msgRouteFromLable,'" + MessageHeader.JOIN_LEFT + "')")); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, leftScriptStage); this.currentChainStage = leftScriptStage; - ChainStage leftWindowStage = this.mainPipelineBuilder.createStage(window); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, leftWindowStage); + AbstractChainStage leftWindowStage = this.rootPipelineBuilder.createStage(window); + if (leftWindowStage instanceof ShuffleChainStage) { + this.rootPipelineBuilder.getPipeline().getStages().remove(leftWindowStage); + + ShuffleChainStage shuffleChainStage = (ShuffleChainStage) leftWindowStage; + this.rootPipelineBuilder.addChainStage(shuffleChainStage.getOutputChainStage()); + this.rootPipelineBuilder.addChainStage(shuffleChainStage.getConsumeChainStage()); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, shuffleChainStage.getOutputChainStage()); + leftWindowStage = shuffleChainStage.getConsumeChainStage(); + } else { + this.rootPipelineBuilder.setTopologyStages(currentChainStage, leftWindowStage); + } //处理右流,右流增加script - DataStream dataStream = rightStream.script("setHeader(msgRouteFromLable,'" + MessageHeader.JOIN_RIGHT + "')").addStage(window); + rightStream.script("setHeader(msgRouteFromLable,'" + MessageHeader.JOIN_RIGHT + "')").addRigthWindow(window); //dataStream.addStage(window); addOtherDataStream(rightStream); - return new JoinStream(window, this.mainPipelineBuilder, this.otherPipelineBuilders, leftWindowStage, joinType); + return new JoinStream(window, this.rootPipelineBuilder, this.otherPipelineBuilders, leftWindowStage, joinType); + } + + protected void addRigthWindow(JoinWindow window) { + AbstractChainStage rigthWindowStage = this.rootPipelineBuilder.createStage(window); + if (rigthWindowStage instanceof ShuffleChainStage) { + this.rootPipelineBuilder.getPipeline().getStages().remove(rigthWindowStage); + + ShuffleChainStage shuffleChainStage = (ShuffleChainStage) rigthWindowStage; + this.rootPipelineBuilder.addChainStage(shuffleChainStage.getOutputChainStage()); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, shuffleChainStage.getOutputChainStage()); + } else { + this.rootPipelineBuilder.setTopologyStages(currentChainStage, rigthWindowStage); + } } /** @@ -363,15 +453,15 @@ public DataStream union(DataStream rightStream) { Union union = new Union(); //处理左流,做流的isMain设置成true - UDFUnionChainStage chainStage = (UDFUnionChainStage) this.mainPipelineBuilder.createStage(union); + UDFUnionChainStage chainStage = (UDFUnionChainStage) this.rootPipelineBuilder.createStage(union); chainStage.setMainStream(true); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, chainStage); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, chainStage); //处理右流,做流的isMain设置成true rightStream.addStage(union); addOtherDataStream(rightStream); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, chainStage); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, chainStage); } /** @@ -382,16 +472,16 @@ public DataStream union(DataStream rightStream) { */ public SplitStream split(SplitFunction splitFunction) { StageBuilder operator = new StageBuilder() { - @Override - protected T operate(IMessage message, AbstractContext context) { + + @Override protected IMessage handleMessage(IMessage message, AbstractContext context) { String labelName = splitFunction.split(message.getMessageValue()); message.getHeader().addRouteLabel(labelName); - return null; + return message; } }; - ChainStage stage = this.mainPipelineBuilder.createStage(operator); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, stage); - return new SplitStream(this.mainPipelineBuilder, this.otherPipelineBuilders, stage); + AbstractChainStage stage = this.rootPipelineBuilder.createStage(operator); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, stage); + return new SplitStream(this.rootPipelineBuilder, this.otherPipelineBuilders, stage); } /** @@ -403,9 +493,7 @@ protected T operate(IMessage message, AbstractContext context) { * @param sqlOrTableName * @return */ - @Deprecated - public JoinStream join(String url, String userName, String password, String sqlOrTableName, - long pollingTimeMintue) { + public JoinStream join(String url, String userName, String password, String sqlOrTableName, long pollingTimeMintue) { return join(url, userName, password, sqlOrTableName, null, pollingTimeMintue); } @@ -418,67 +506,60 @@ public JoinStream join(String url, String userName, String password, String sqlO * @param sqlOrTableName * @return */ - @Deprecated - public JoinStream join(String url, String userName, String password, String sqlOrTableName, String jdbcDriver, - long pollingTimeMinute) { + public JoinStream join(String url, String userName, String password, String sqlOrTableName, String jdbcDriver, long pollingTimeSeconds) { DBDim dbDim = new DBDim(); dbDim.setUrl(url); dbDim.setUserName(userName); dbDim.setPassword(password); dbDim.setSql(sqlOrTableName); - dbDim.setPollingTimeMinute(pollingTimeMinute); - dbDim.setJdbcdriver(jdbcDriver); - this.mainPipelineBuilder.addConfigurables(dbDim); - return new JoinStream(dbDim, mainPipelineBuilder, otherPipelineBuilders, currentChainStage, true); + dbDim.setPollingTimeSeconds(pollingTimeSeconds); + dbDim.setJdbcDriver(jdbcDriver); + this.rootPipelineBuilder.addConfigurables(dbDim); + return new JoinStream(dbDim, rootPipelineBuilder, otherPipelineBuilders, currentChainStage, true); } - public JoinStream dimJoin(String url, String userName, String password, String sqlOrTableName, - Long pollingTimeMinute) { - return dimJoin(url, userName, password, sqlOrTableName, "com.mysql.jdbc.Driver", pollingTimeMinute); + public JoinStream dimJoin(String url, String userName, String password, String sqlOrTableName, Long pollingTimeSeconds) { + return dimJoin(url, userName, password, sqlOrTableName, ConfigurationKey.DEFAULT_JDBC_DRIVER, pollingTimeSeconds); } - public JoinStream dimJoin(String url, String userName, String password, String sqlOrTableName, String jdbcDriver, - Long pollingTimeMinute) { - return dimJoin(url, userName, password, sqlOrTableName, jdbcDriver, pollingTimeMinute, JoinStream.JoinType.INNER_JOIN); + public JoinStream dimJoin(String url, String userName, String password, String sqlOrTableName, String jdbcDriver, Long pollingTimeSeconds) { + return dimJoin(url, userName, password, sqlOrTableName, jdbcDriver, pollingTimeSeconds, JoinStream.JoinType.INNER_JOIN); } - public JoinStream dimJoin(String filePath, Long pollingTimeMinute) { - return dimJoin(filePath, pollingTimeMinute, JoinStream.JoinType.INNER_JOIN); + public JoinStream dimJoin(String filePath, Long pollingTimeSeconds) { + return dimJoin(filePath, pollingTimeSeconds, JoinStream.JoinType.INNER_JOIN); } - public JoinStream dimLeftJoin(String url, String userName, String password, String sqlOrTableName, - Long pollingTimeMinute) { - return dimLeftJoin(url, userName, password, sqlOrTableName, "com.mysql.jdbc.Driver", pollingTimeMinute); + public JoinStream dimLeftJoin(String url, String userName, String password, String sqlOrTableName, Long pollingTimeSeconds) { + return dimLeftJoin(url, userName, password, sqlOrTableName, ConfigurationKey.DEFAULT_JDBC_DRIVER, pollingTimeSeconds); } - public JoinStream dimLeftJoin(String url, String userName, String password, String sqlOrTableName, - String jdbcDriver, Long pollingTimeMinute) { - return dimJoin(url, userName, password, sqlOrTableName, jdbcDriver, pollingTimeMinute, JoinStream.JoinType.LEFT_JOIN); + public JoinStream dimLeftJoin(String url, String userName, String password, String sqlOrTableName, String jdbcDriver, Long pollingTimeSeconds) { + return dimJoin(url, userName, password, sqlOrTableName, jdbcDriver, pollingTimeSeconds, JoinStream.JoinType.LEFT_JOIN); } - public JoinStream dimLeftJoin(String filePath, Long pollingTimeMinute) { - return dimJoin(filePath, pollingTimeMinute, JoinStream.JoinType.LEFT_JOIN); + public JoinStream dimLeftJoin(String filePath, Long pollingTimeSeconds) { + return dimJoin(filePath, pollingTimeSeconds, JoinStream.JoinType.LEFT_JOIN); } - protected JoinStream dimJoin(String filePath, Long pollingTimeMinute, JoinStream.JoinType joinType) { + protected JoinStream dimJoin(String filePath, Long pollingTimeSeconds, JoinStream.JoinType joinType) { FileDim fileDim = new FileDim(); fileDim.setFilePath(filePath); - fileDim.setPollingTimeMinute(pollingTimeMinute); - this.mainPipelineBuilder.addConfigurables(fileDim); - return new JoinStream(fileDim, mainPipelineBuilder, otherPipelineBuilders, currentChainStage, true, joinType); + fileDim.setPollingTimeSeconds(pollingTimeSeconds); + this.rootPipelineBuilder.addConfigurables(fileDim); + return new JoinStream(fileDim, rootPipelineBuilder, otherPipelineBuilders, currentChainStage, true, joinType); } - protected JoinStream dimJoin(String url, String userName, String password, String sqlOrTableName, String jdbcDriver, - Long pollingTimeMinute, JoinStream.JoinType joinType) { + protected JoinStream dimJoin(String url, String userName, String password, String sqlOrTableName, String jdbcDriver, Long pollingTimeSeconds, JoinStream.JoinType joinType) { DBDim dbDim = new DBDim(); dbDim.setUrl(url); dbDim.setUserName(userName); dbDim.setPassword(password); dbDim.setSql(sqlOrTableName); - dbDim.setPollingTimeMinute(pollingTimeMinute); - dbDim.setJdbcdriver(jdbcDriver); - this.mainPipelineBuilder.addConfigurables(dbDim); - return new JoinStream(dbDim, mainPipelineBuilder, otherPipelineBuilders, currentChainStage, true, joinType); + dbDim.setPollingTimeSeconds(pollingTimeSeconds); + dbDim.setJdbcDriver(jdbcDriver); + this.rootPipelineBuilder.addConfigurables(dbDim); + return new JoinStream(dbDim, rootPipelineBuilder, otherPipelineBuilders, currentChainStage, true, joinType); } /** @@ -487,22 +568,20 @@ protected JoinStream dimJoin(String url, String userName, String password, Strin * @param filePath * @return */ - @Deprecated - public JoinStream join(String filePath, long pollingTimeMinute) { + @Deprecated public JoinStream join(String filePath, long pollingTimeSeconds) { FileDim fileDim = new FileDim(); fileDim.setFilePath(filePath); - fileDim.setPollingTimeMinute(pollingTimeMinute); - this.mainPipelineBuilder.addConfigurables(fileDim); - return new JoinStream(fileDim, mainPipelineBuilder, otherPipelineBuilders, currentChainStage, true); + fileDim.setPollingTimeSeconds(pollingTimeSeconds); + this.rootPipelineBuilder.addConfigurables(fileDim); + return new JoinStream(fileDim, rootPipelineBuilder, otherPipelineBuilders, currentChainStage, true); } - @Deprecated - public JoinStream innerJoin(String filePath, long pollingTimeMinute) { + @Deprecated public JoinStream innerJoin(String filePath, long pollingTimeSeconds) { FileDim fileDim = new FileDim(); fileDim.setFilePath(filePath); - fileDim.setPollingTimeMinute(pollingTimeMinute); - this.mainPipelineBuilder.addConfigurables(fileDim); - return new JoinStream(fileDim, mainPipelineBuilder, otherPipelineBuilders, currentChainStage, true); + fileDim.setPollingTimeSeconds(pollingTimeSeconds); + this.rootPipelineBuilder.addConfigurables(fileDim); + return new JoinStream(fileDim, rootPipelineBuilder, otherPipelineBuilders, currentChainStage, true); } /** @@ -514,15 +593,15 @@ public JoinStream innerJoin(String filePath, long pollingTimeMinute) { */ public DataStream forEach(ForEachFunction forEachFunction) { StageBuilder selfChainStage = new StageBuilder() { - @Override - protected T operate(IMessage message, AbstractContext context) { + @Override protected IMessage handleMessage(IMessage message, AbstractContext context) { forEachFunction.foreach((O) message.getMessageValue()); return null; } + }; - ChainStage stage = this.mainPipelineBuilder.createStage(selfChainStage); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, stage); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, stage); + AbstractChainStage stage = this.rootPipelineBuilder.createStage(selfChainStage); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, stage); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, stage); } /** @@ -534,15 +613,15 @@ protected T operate(IMessage message, AbstractContext context) { */ public DataStream forEachMessage(ForEachMessageFunction forEachFunction) { StageBuilder selfChainStage = new StageBuilder() { - @Override - protected T operate(IMessage message, AbstractContext context) { + @Override protected IMessage handleMessage(IMessage message, AbstractContext context) { forEachFunction.foreach(message, context); return null; } + }; - ChainStage stage = this.mainPipelineBuilder.createStage(selfChainStage); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, stage); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, stage); + AbstractChainStage stage = this.rootPipelineBuilder.createStage(selfChainStage); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, stage); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, stage); } /** @@ -551,9 +630,9 @@ protected T operate(IMessage message, AbstractContext context) { * @param fieldNames */ public DataStream selectFields(String... fieldNames) { - ChainStage stage = this.mainPipelineBuilder.createStage(new ScriptOperator("retain(" + MapKeyUtil.createKeyBySign(",", fieldNames) + ")")); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, stage); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, stage); + AbstractChainStage stage = this.rootPipelineBuilder.createStage(new ScriptOperator("retain(" + MapKeyUtil.createKeyBySign(",", fieldNames) + ")")); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, stage); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, stage); } /** @@ -563,8 +642,8 @@ public DataStream selectFields(String... fieldNames) { */ protected void addOtherDataStream(DataStream rightSource) { //如果是多流join,需要把把piplinebuider保存下来,在启动时,启动多个pipline - if (!rightSource.mainPipelineBuilder.equals(this.mainPipelineBuilder)) { - this.otherPipelineBuilders.add(rightSource.mainPipelineBuilder); + if (!rightSource.rootPipelineBuilder.equals(this.rootPipelineBuilder)) { + this.otherPipelineBuilders.add(rightSource.rootPipelineBuilder); } this.otherPipelineBuilders.addAll(rightSource.otherPipelineBuilders); @@ -575,23 +654,23 @@ public DataStream toFile(String filePath, int batchSize, boolean isAppend) { if (batchSize > 0) { fileChannel.setBatchSize(batchSize); } - ChainStage output = mainPipelineBuilder.createStage(fileChannel); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, output); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, output); + AbstractChainStage output = rootPipelineBuilder.createStage(fileChannel); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, output); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, output); } public DataStream toFile(String filePath, boolean isAppend) { FileSink fileChannel = new FileSink(filePath, isAppend); - ChainStage output = mainPipelineBuilder.createStage(fileChannel); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, output); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, output); + AbstractChainStage output = rootPipelineBuilder.createStage(fileChannel); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, output); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, output); } public DataStream toFile(String filePath) { FileSink fileChannel = new FileSink(filePath); - ChainStage output = mainPipelineBuilder.createStage(fileChannel); - mainPipelineBuilder.setTopologyStages(currentChainStage, output); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, output); + AbstractChainStage output = rootPipelineBuilder.createStage(fileChannel); + rootPipelineBuilder.setTopologyStages(currentChainStage, output); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, output); } public DataStream toPrint() { @@ -603,31 +682,30 @@ public DataStream toPrint(int batchSize) { if (batchSize > 0) { outputPrintChannel.setBatchSize(batchSize); } - ChainStage output = this.mainPipelineBuilder.createStage(outputPrintChannel); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, output); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, output); + AbstractChainStage output = this.rootPipelineBuilder.createStage(outputPrintChannel); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, output); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, output); } public DataStream toDB(String url, String userName, String password, String tableName) { DBSink dbChannel = new DBSink(url, userName, password, tableName); - ChainStage output = this.mainPipelineBuilder.createStage(dbChannel); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, output); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, output); + AbstractChainStage output = this.rootPipelineBuilder.createStage(dbChannel); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, output); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, output); } public DataStream toDB(String url, String userName, String password, String tableName, String sqlMode) { DBSink dbChannel = new DBSink(url, userName, password, tableName); - ChainStage output = this.mainPipelineBuilder.createStage(dbChannel); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, output); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, output); + AbstractChainStage output = this.rootPipelineBuilder.createStage(dbChannel); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, output); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, output); } - public DataStream toDB(String url, String userName, String password, String tableName, String sqlMode, - Boolean sqlCache) { + public DataStream toDB(String url, String userName, String password, String tableName, String sqlMode, Boolean sqlCache) { DBSink dbChannel = new DBSink(url, userName, password, tableName); - ChainStage output = this.mainPipelineBuilder.createStage(dbChannel); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, output); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, output); + AbstractChainStage output = this.rootPipelineBuilder.createStage(dbChannel); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, output); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, output); } public DataStream toRocketmq(String topic, String nameServerAddress) { @@ -642,8 +720,7 @@ public DataStream toRocketmq(String topic, String tags, String groupName, String return toRocketmq(topic, tags, groupName, -1, nameServerAddress, null, false); } - public DataStream toRocketmq(String topic, String tags, String groupName, int batchSize, String nameServerAddress, - String clusterName, boolean order) { + public DataStream toRocketmq(String topic, String tags, String groupName, int batchSize, String nameServerAddress, String clusterName, boolean order) { RocketMQSink rocketMQSink = new RocketMQSink(); if (StringUtils.isNotBlank(topic)) { rocketMQSink.setTopic(topic); @@ -664,109 +741,50 @@ public DataStream toRocketmq(String topic, String tags, String groupName, int ba rocketMQSink.setBatchSize(batchSize); } rocketMQSink.setOrder(order); - ChainStage output = this.mainPipelineBuilder.createStage(rocketMQSink); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, output); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, output); + AbstractChainStage output = this.rootPipelineBuilder.createStage(rocketMQSink); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, output); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, output); } public DataStream toKafka(String bootstrapServers, String topic) { KafkaSink kafkaSink = new KafkaSink(bootstrapServers, topic); - ChainStage output = this.mainPipelineBuilder.createStage(kafkaSink); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, output); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, output); + AbstractChainStage output = this.rootPipelineBuilder.createStage(kafkaSink); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, output); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, output); } public DataStream toEnhanceDBSink(String url, String userName, String password, String tableName) { EnhanceDBSink sink = new EnhanceDBSink(url, userName, password, tableName); - ChainStage output = this.mainPipelineBuilder.createStage(sink); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, output); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, output); + AbstractChainStage output = this.rootPipelineBuilder.createStage(sink); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, output); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, output); } public DataStream toMultiDB(String url, String userName, String password, String logicTableName, String fieldName) { DynamicMultipleDBSink sink = new DynamicMultipleDBSink(url, userName, password, logicTableName, fieldName); - ChainStage output = this.mainPipelineBuilder.createStage(sink); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, output); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, output); + AbstractChainStage output = this.rootPipelineBuilder.createStage(sink); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, output); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, output); } public DataStream toMqtt(String url, String clientId, String topic) { PahoSink pahoSink = new PahoSink(url, clientId, topic); - ChainStage output = this.mainPipelineBuilder.createStage(pahoSink); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, output); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, output); + AbstractChainStage output = this.rootPipelineBuilder.createStage(pahoSink); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, output); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, output); } public DataStream toMqtt(String url, String clientId, String topic, String username, String password) { PahoSink pahoSink = new PahoSink(url, clientId, topic, username, password); - ChainStage output = this.mainPipelineBuilder.createStage(pahoSink); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, output); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, output); + AbstractChainStage output = this.rootPipelineBuilder.createStage(pahoSink); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, output); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, output); } public DataStream to(AbstractSink sink) { - ChainStage output = this.mainPipelineBuilder.createStage(sink); - this.mainPipelineBuilder.setTopologyStages(currentChainStage, output); - return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, output); - } - - /** - * 启动流任务 - */ - public void start() { - start(false); - } - - /** - * 启动流任务 - */ - public void asyncStart() { - start(true); + AbstractChainStage output = this.rootPipelineBuilder.createStage(sink); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, output); + return new DataStream(this.rootPipelineBuilder, this.otherPipelineBuilders, output); } - /** - * 启动流任务 - * - * @param isAsyn - */ - public void start(boolean isAsyn) { - if (this.mainPipelineBuilder == null) { - return; - } - - ConfigurableComponent configurableComponent = ComponentCreator.getComponent(mainPipelineBuilder.getPipelineNameSpace(), ConfigurableComponent.class, ConfigureFileKey.CONNECT_TYPE + ":memory"); - ChainPipeline pipeline = this.mainPipelineBuilder.build(configurableComponent.getService()); - - if (this.otherPipelineBuilders != null) { - Thread mainThread = new Thread(new Runnable() { - @Override - public void run() { - pipeline.startChannel(); - } - }); - mainThread.start(); - for (PipelineBuilder builder : otherPipelineBuilders) { - ChainPipeline otherPipeline = builder.build(configurableComponent.getService()); - Thread thread = new Thread(new Runnable() { - @Override - public void run() { - otherPipeline.startChannel(); - } - }); - thread.start(); - } - } else { - pipeline.startChannel(); - } - if (isAsyn) { - return; - } - while (true) { - try { - Thread.sleep(10000); - } catch (InterruptedException e) { - e.printStackTrace(); - } - } - } } diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/JoinStream.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/JoinStream.java index 975c1a59..9215ca6e 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/JoinStream.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/JoinStream.java @@ -24,10 +24,10 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.rocketmq.streams.client.transform.window.Time; -import org.apache.rocketmq.streams.common.model.NameCreator; import org.apache.rocketmq.streams.common.model.NameCreatorContext; -import org.apache.rocketmq.streams.common.topology.ChainStage; import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; +import org.apache.rocketmq.streams.common.topology.model.AbstractChainStage; +import org.apache.rocketmq.streams.common.topology.stages.DimChainStage; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.dim.model.AbstractDim; import org.apache.rocketmq.streams.filter.builder.ExpressionBuilder; @@ -48,29 +48,27 @@ public class JoinStream { protected JoinType joinType;//连接类型 //用于返回DataStream流 - protected PipelineBuilder pipelineBuilder; + protected PipelineBuilder rootPipelineBuilder; protected Set otherPipelineBuilders; - protected ChainStage currentChainStage; + protected AbstractChainStage currentChainStage; /** * 双流join 场景 * - * @param joinWindow - * @param pipelineBuilder - * @param pipelineBuilders - * @param currentChainStage + * @param joinWindow join窗口 + * @param rootPipelineBuilder rootPipelineBuilder + * @param otherPipelineBuilders otherPipelineBuilders + * @param currentChainStage currentChainStage */ - public JoinStream(JoinWindow joinWindow, PipelineBuilder pipelineBuilder, Set pipelineBuilders, - ChainStage currentChainStage) { - this.pipelineBuilder = pipelineBuilder; - this.otherPipelineBuilders = pipelineBuilders; + public JoinStream(JoinWindow joinWindow, PipelineBuilder rootPipelineBuilder, Set otherPipelineBuilders, AbstractChainStage currentChainStage) { + this.rootPipelineBuilder = rootPipelineBuilder; + this.otherPipelineBuilders = otherPipelineBuilders; this.currentChainStage = currentChainStage; this.joinWindow = joinWindow; } - public JoinStream(JoinWindow joinWindow, PipelineBuilder pipelineBuilder, Set pipelineBuilders, - ChainStage currentChainStage, JoinType joinType) { - this.pipelineBuilder = pipelineBuilder; + public JoinStream(JoinWindow joinWindow, PipelineBuilder rootPipelineBuilder, Set pipelineBuilders, AbstractChainStage currentChainStage, JoinType joinType) { + this.rootPipelineBuilder = rootPipelineBuilder; this.otherPipelineBuilders = pipelineBuilders; this.currentChainStage = currentChainStage; this.joinWindow = joinWindow; @@ -80,31 +78,28 @@ public JoinStream(JoinWindow joinWindow, PipelineBuilder pipelineBuilder, Set pipelineBuilders, - ChainStage currentChainStage) { + public JoinStream(AbstractDim dim, PipelineBuilder rootPipelineBuilder, Set pipelineBuilders, AbstractChainStage currentChainStage) { this.dim = dim; - this.pipelineBuilder = pipelineBuilder; + this.rootPipelineBuilder = rootPipelineBuilder; this.otherPipelineBuilders = pipelineBuilders; this.currentChainStage = currentChainStage; this.isDimJoin = true; } - public JoinStream(AbstractDim dim, PipelineBuilder pipelineBuilder, Set pipelineBuilders, - ChainStage currentChainStage, Boolean isDimJoin) { + public JoinStream(AbstractDim dim, PipelineBuilder rootPipelineBuilder, Set pipelineBuilders, AbstractChainStage currentChainStage, Boolean isDimJoin) { this.dim = dim; - this.pipelineBuilder = pipelineBuilder; + this.rootPipelineBuilder = rootPipelineBuilder; this.otherPipelineBuilders = pipelineBuilders; this.currentChainStage = currentChainStage; this.isDimJoin = isDimJoin; } - public JoinStream(AbstractDim dim, PipelineBuilder pipelineBuilder, Set pipelineBuilders, - ChainStage currentChainStage, Boolean isDimJoin, JoinType joinType) { - this.pipelineBuilder = pipelineBuilder; + public JoinStream(AbstractDim dim, PipelineBuilder rootPipelineBuilder, Set pipelineBuilders, AbstractChainStage currentChainStage, Boolean isDimJoin, JoinType joinType) { + this.rootPipelineBuilder = rootPipelineBuilder; this.otherPipelineBuilders = pipelineBuilders; this.currentChainStage = currentChainStage; this.dim = dim; @@ -112,6 +107,19 @@ public JoinStream(AbstractDim dim, PipelineBuilder pipelineBuilder, Set stage = this.pipelineBuilder.createStage(new ScriptOperator(script)); - this.pipelineBuilder.setTopologyStages(currentChainStage, stage); - return new DataStream(pipelineBuilder, otherPipelineBuilders, stage); + DimChainStage dimChainStage = new DimChainStage(); + ScriptOperator scriptOperator = new ScriptOperator(script); + dimChainStage.setScript(scriptOperator); + dimChainStage.setDim(dim); + dimChainStage.setExpressionStr(onCondition); + dimChainStage.setInnerJoin(isInnerJoin); + dimChainStage.setLeftJoin(isLeftJoin); + dimChainStage.setSplitFieldName(splitFieldName); + this.rootPipelineBuilder.addConfigurables(scriptOperator); + this.rootPipelineBuilder.addChainStage(dimChainStage); + this.rootPipelineBuilder.setTopologyStages(currentChainStage, dimChainStage); + return new DataStream(rootPipelineBuilder, otherPipelineBuilders, dimChainStage); } /** @@ -209,25 +232,16 @@ protected DataStream doJoin() { AtomicBoolean hasNoEqualsExpression = new AtomicBoolean(false);//是否有非等值的join 条件 Map left2Right = createJoinFieldsFromCondition(onCondition, hasNoEqualsExpression);//把等值条件的左右字段映射成map - List leftList = new ArrayList<>(); - List rightList = new ArrayList<>(); - leftList.addAll(left2Right.keySet()); - rightList.addAll(left2Right.values()); + List leftList = new ArrayList<>(left2Right.keySet()); + List rightList = new ArrayList<>(left2Right.values()); joinWindow.setLeftJoinFieldNames(leftList); joinWindow.setRightJoinFieldNames(rightList); //如果有非等值,则把这个条件设置进去 if (hasNoEqualsExpression.get()) { joinWindow.setExpression(onCondition); } - return new DataStream(pipelineBuilder, otherPipelineBuilders, currentChainStage); - } - - /** - * 支持的连接类型,目前支持inner join和left join - */ - public enum JoinType { - INNER_JOIN, - LEFT_JOIN + joinWindow.setRightDependentTableName("right"); + return new DataStream(rootPipelineBuilder, otherPipelineBuilders, currentChainStage); } /** @@ -254,16 +268,11 @@ public Map createJoinFieldsFromCondition(String onCondition, Ato return left2Right; } - public static String createName(String functionName, String... names) { - if (names == null || names.length == 0) { - return NameCreatorContext.get().createNewName(INNER_VAR_NAME_PREFIX, functionName); - } - String[] values = new String[names.length + 2]; - values[0] = INNER_VAR_NAME_PREFIX; - values[1] = functionName; - for (int i = 2; i < values.length; i++) { - values[i] = names[i - 2]; - } - return NameCreatorContext.get().createNewName(values); + /** + * 支持的连接类型,目前支持inner join和left join + */ + public enum JoinType { + INNER_JOIN, + LEFT_JOIN } } diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/OverWindowStream.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/OverWindowStream.java index 1c517fde..60331ba2 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/OverWindowStream.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/OverWindowStream.java @@ -18,8 +18,8 @@ import java.util.ArrayList; import java.util.Set; -import org.apache.rocketmq.streams.common.topology.ChainStage; import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; +import org.apache.rocketmq.streams.common.topology.model.AbstractChainStage; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.window.operator.impl.ShuffleOverWindow; @@ -31,10 +31,10 @@ public class OverWindowStream { */ protected PipelineBuilder pipelineBuilder; protected Set otherPipelineBuilders; - protected ChainStage currentChainStage; + protected AbstractChainStage currentChainStage; public OverWindowStream(ShuffleOverWindow window, PipelineBuilder pipelineBuilder, - Set pipelineBuilders, ChainStage currentChainStage) { + Set pipelineBuilders, AbstractChainStage currentChainStage) { this.pipelineBuilder = pipelineBuilder; this.otherPipelineBuilders = pipelineBuilders; this.currentChainStage = currentChainStage; diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/SplitStream.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/SplitStream.java index 4d806ed2..e909c67f 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/SplitStream.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/SplitStream.java @@ -21,8 +21,8 @@ import java.util.Set; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.topology.ChainStage; import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; +import org.apache.rocketmq.streams.common.topology.model.AbstractChainStage; import org.apache.rocketmq.streams.common.topology.stages.udf.StageBuilder; public class SplitStream implements Serializable { @@ -32,9 +32,9 @@ public class SplitStream implements Serializable { */ protected PipelineBuilder pipelineBuilder; protected Set otherPipelineBuilders; - protected ChainStage currentChainStage; + protected AbstractChainStage currentChainStage; - public SplitStream(PipelineBuilder pipelineBuilder, Set pipelineBuilders, ChainStage currentChainStage) { + public SplitStream(PipelineBuilder pipelineBuilder, Set pipelineBuilders, AbstractChainStage currentChainStage) { this.pipelineBuilder = pipelineBuilder; this.otherPipelineBuilders = pipelineBuilders; this.currentChainStage = currentChainStage; @@ -48,12 +48,14 @@ public SplitStream(PipelineBuilder pipelineBuilder, Set pipelin */ public DataStream select(String lableName) { StageBuilder operator = new StageBuilder() { - @Override protected T operate(IMessage message, AbstractContext context) { + @Override protected IMessage handleMessage(IMessage message, AbstractContext context) { + System.out.println(message.getMessageBody()); return null; } + }; - ChainStage stage = this.pipelineBuilder.createStage(operator); + AbstractChainStage stage = this.pipelineBuilder.createStage(operator); stage.setLabel(lableName); this.pipelineBuilder.setTopologyStages(currentChainStage, stage); return new DataStream(pipelineBuilder, otherPipelineBuilders, stage); diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/WindowStream.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/WindowStream.java index db6f88d4..387c408d 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/WindowStream.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/WindowStream.java @@ -18,33 +18,19 @@ package org.apache.rocketmq.streams.client.transform; import com.alibaba.fastjson.JSONObject; -import java.util.List; -import java.util.Map; import java.util.Properties; import java.util.Set; -import javafx.util.Pair; import org.apache.rocketmq.streams.client.transform.window.Time; 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.AbstractSource; -import org.apache.rocketmq.streams.common.channel.source.ISource; import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.context.UserDefinedMessage; import org.apache.rocketmq.streams.common.functions.MapFunction; -import org.apache.rocketmq.streams.common.functions.ReduceFunction; -import org.apache.rocketmq.streams.common.model.NameCreator; -import org.apache.rocketmq.streams.common.model.NameCreatorContext; -import org.apache.rocketmq.streams.common.topology.ChainStage; +import org.apache.rocketmq.streams.common.model.Pair; import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; -import org.apache.rocketmq.streams.common.topology.stages.udf.IReducer; -import org.apache.rocketmq.streams.common.utils.Base64Utils; -import org.apache.rocketmq.streams.common.utils.InstantiationUtil; +import org.apache.rocketmq.streams.common.topology.model.AbstractChainStage; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.script.operator.impl.AggregationScript; import org.apache.rocketmq.streams.script.service.IAccumulator; -import org.apache.rocketmq.streams.script.service.udf.SimpleUDAFScript; -import org.apache.rocketmq.streams.script.service.udf.UDAFScript; import org.apache.rocketmq.streams.serviceloader.ServiceLoaderComponent; import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.operator.AbstractWindow; @@ -61,10 +47,10 @@ public class WindowStream { */ protected PipelineBuilder pipelineBuilder; protected Set otherPipelineBuilders; - protected ChainStage currentChainStage; + protected AbstractChainStage currentChainStage; public WindowStream(AbstractWindow window, PipelineBuilder pipelineBuilder, Set pipelineBuilders, - ChainStage currentChainStage) { + AbstractChainStage currentChainStage) { this.pipelineBuilder = pipelineBuilder; this.otherPipelineBuilders = pipelineBuilders; this.currentChainStage = currentChainStage; @@ -112,13 +98,13 @@ public WindowStream distinct(String fieldName, String asName) { * @return */ public WindowStream count_distinct(String fieldName, String asName) { - return count_distinct_2(fieldName,asName); + return count_distinct_2(fieldName, asName); } - public WindowStream addUDAF(IAccumulator accumulator, String asName,String... fieldNames) { - AggregationScript.registUDAF(accumulator.getClass().getSimpleName(),accumulator.getClass()); - String prefix = asName + "="+accumulator.getClass().getSimpleName()+"(" + MapKeyUtil.createKeyBySign(",",fieldNames)+")"; - window.getSelectMap().put(asName,prefix); + public WindowStream addUDAF(IAccumulator accumulator, String asName, String... fieldNames) { + AggregationScript.registUDAF(accumulator.getClass().getSimpleName(), accumulator.getClass()); + String prefix = asName + "=" + accumulator.getClass().getSimpleName() + "(" + MapKeyUtil.createKeyBySign(",", fieldNames) + ")"; + window.getSelectMap().put(asName, prefix); return this; } @@ -130,20 +116,18 @@ public WindowStream count_distinct_2(String fieldName, String asName) { return this; } - public WindowStream saveWindowMsg(MapFunction> mapFunction ,String sinkType, Properties properties) { + public WindowStream saveWindowMsg(MapFunction> mapFunction, String sinkType, Properties properties) { ServiceLoaderComponent serviceLoaderComponent = ComponentCreator.getComponent(IChannelBuilder.class.getName(), ServiceLoaderComponent.class); IChannelBuilder builder = (IChannelBuilder) serviceLoaderComponent.loadService(sinkType.toLowerCase()); if (builder == null) { throw new RuntimeException( - "expect channel creator for " + sinkType+ ". but not found"); + "expect channel creator for " + sinkType + ". but not found"); } - ISink sink= builder.createSink(pipelineBuilder.getPipelineNameSpace(), window.getConfigureName(), properties, null); + ISink sink = builder.createSink(pipelineBuilder.getPipelineNameSpace(), window.getName(), properties, null); this.pipelineBuilder.addConfigurables(sink); - this.window.setContextMsgSinkName(sink.getConfigureName()); - byte[] bytes=InstantiationUtil.serializeObject(mapFunction); - String mapFunctionStr = Base64Utils.encode(bytes); - this.window.setMapFunctionSerializeValue(mapFunctionStr); + this.window.setContextMsgSink(sink); + this.window.setMapFunction(mapFunction); return this; } @@ -276,30 +260,6 @@ public WindowStream timeField(String fieldName) { return this; } - /** - * 用户自定义reduce逻辑 - * - * @param reduceFunction - * @return - */ - public DataStream reduce(ReduceFunction reduceFunction) { - window.setReducer((IReducer) (accumulator, msg) -> { - Object accumulatorValue = accumulator; - Object msgValue = msg; - if (msg instanceof UserDefinedMessage) { - UserDefinedMessage userDefinedMessage = (UserDefinedMessage) msg; - msgValue = userDefinedMessage.getMessageValue(); - } - if (accumulator instanceof UserDefinedMessage) { - UserDefinedMessage userDefinedMessage = (UserDefinedMessage) accumulator; - accumulatorValue = userDefinedMessage.getMessageValue(); - } - R result = reduceFunction.reduce((R) accumulatorValue, (O) msgValue); - return new UserDefinedMessage(result); - }); - return new DataStream(pipelineBuilder, otherPipelineBuilders, currentChainStage); - } - public DataStream toDataSteam() { return new DataStream(pipelineBuilder, otherPipelineBuilders, currentChainStage); } diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ApplicationTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ApplicationTest.java deleted file mode 100644 index 80595333..00000000 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ApplicationTest.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.client; - -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; -import org.apache.rocketmq.streams.common.topology.task.StreamsTask; -import org.apache.rocketmq.streams.configurable.ConfigurableComponent; -import org.junit.Test; - -public class ApplicationTest { - - @Test - public void testApplication() throws InterruptedException { - ComponentCreator.getProperties().put(ConfigureFileKey.POLLING_TIME, "5"); - ComponentCreator.getProperties().put(ConfigureFileKey.CONNECT_TYPE, "DB"); - ComponentCreator.getProperties().put("dipper.rds.jdbc.url", "jdbc:mysql://host:port/database?serverTimezone=Asia/Shanghai"); - ComponentCreator.getProperties().put("dipper.rds.jdbc.username", "username"); - ComponentCreator.getProperties().put("dipper.rds.jdbc.password", "password"); - - ConfigurableComponent configurableComponent = ComponentCreator.getComponent("chris_tmp", ConfigurableComponent.class); - StreamsTask streamsTask = configurableComponent.queryConfigurable(StreamsTask.TYPE, "task"); -// streamsTask.setNameSpace("chris_tmp"); -// streamsTask.setConfigureName("task"); - - if (streamsTask != null) { - StreamsTask copy = new StreamsTask(); - copy.toObject(streamsTask.toJson()); - - copy.setUpdateFlag(copy.getUpdateFlag() + 1); - configurableComponent.insert(copy); - StreamsTask streamsTask1 = configurableComponent.queryConfigurable(StreamsTask.TYPE, "task"); - System.out.println(streamsTask1.getUpdateFlag() == copy.getUpdateFlag()); - } - System.out.println(streamsTask.getUpdateFlag()); - // configurableComponent.refreshConfigurable("chris_tmp"); - // System.out.println(streamsTask.getUpdateFlag()); - while (true) { - Thread.sleep(1000); - - } - } -} diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/DBDriverTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/DBDriverTest.java deleted file mode 100644 index 0894c873..00000000 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/DBDriverTest.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * 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.client; - -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; -import org.apache.rocketmq.streams.configurable.ConfigurableComponent; -import org.apache.rocketmq.streams.configurable.model.Configure; -import org.apache.rocketmq.streams.db.driver.DriverBuilder; -import org.junit.Test; - -import static junit.framework.TestCase.assertNotNull; - -/** - * 数据库的存储,需要配置存储的连接参数,请先完成配置,后执行单元用例 如果未建表,可以通过Configure.createTableSQL() 获取建表语句,创建表后,测试 - */ -public class DBDriverTest { - private String URL = ""; - protected String USER_NAME = ""; - protected String PASSWORD = ""; - protected String TABLE_NAME = "rocketmq_streams_configure_source"; - - @Test - public void testDBConfigurableService() { - String namespace = "streams.db.configurable"; - - //正式使用时,在配置文件配置 - ComponentCreator.getProperties().put(ConfigureFileKey.CONNECT_TYPE, "DB"); - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_URL, URL);//数据库连接url - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_USERNAME, USER_NAME);//用户名 - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_PASSWORD, PASSWORD);//password - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_TABLE_NAME, TABLE_NAME); - - //如果表不存在,创建表 - String sql = (Configure.createTableSQL(TABLE_NAME)); - DriverBuilder.createDriver().execute(sql); - ConfigurableComponent configurableComponent = ConfigurableComponent.getInstance(namespace); - configurableComponent.insert(createPerson(namespace)); - configurableComponent.refreshConfigurable(namespace); - Person person = configurableComponent.queryConfigurable("person", "peronName"); - assertNotNull(person); - } - - /** - * 创建configuable对象 - * - * @param namespace - * @return - */ - protected Person createPerson(String namespace) { - Person person = new Person(); - person.setName("chris"); - person.setAge(18); - person.setNameSpace(namespace); - person.setConfigureName("peronName"); - person.setType("person"); - return person; - } -} diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/DBSinkTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/DBSinkTest.java index 0f151534..3778760c 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/DBSinkTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/DBSinkTest.java @@ -17,16 +17,13 @@ package org.apache.rocketmq.streams.client; import com.alibaba.fastjson.JSONObject; -import org.apache.rocketmq.streams.client.source.DataStreamSource; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; -import org.junit.Before; -import org.junit.Test; - import java.util.ArrayList; import java.util.Date; import java.util.List; import java.util.Random; +import org.apache.rocketmq.streams.client.source.DataStreamSource; +import org.junit.Before; +import org.junit.Test; /** * /** @@ -44,8 +41,8 @@ public class DBSinkTest { @Before public void init() { - dataStream = StreamBuilder.dataStream("test_namespace", "graph_pipeline"); - ComponentCreator.getProperties().put(ConfigureFileKey.CHECKPOINT_STORAGE_NAME, "db"); + dataStream = StreamExecutionEnvironment.getExecutionEnvironment().create("test_namespace", "graph_pipeline"); + // ComponentCreator.getProperties().put(ConfigureFileKey.CHECKPOINT_STORAGE_NAME, "db"); } @Test @@ -57,7 +54,7 @@ public void testToMultiDB() { }; for (int i = 0; i < 100000; i++) { //0,1随机数,整数 - int index = (int)(2 * Math.random()); + int index = (int) (2 * Math.random()); JSONObject msg = new JSONObject(); msg.put("ds", partitions[index]); msg.put("value", String.valueOf(Math.random())); @@ -74,7 +71,7 @@ public void testToEnhanceDB() { List list = new ArrayList(); for (int i = 0; i < 100000; i++) { //0,1随机数,整数 - int index = (int)(2 * Math.random()); + int index = (int) (2 * Math.random()); JSONObject msg = new JSONObject(); msg.put("name", String.valueOf(Math.random())); msg.put("age", new Random().nextInt()); diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/DataStreamTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/DataStreamTest.java index 2484af5e..3018eff5 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/DataStreamTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/DataStreamTest.java @@ -28,6 +28,7 @@ import org.apache.rocketmq.streams.client.strategy.WindowStrategy; import org.apache.rocketmq.streams.client.transform.window.Time; import org.apache.rocketmq.streams.client.transform.window.TumblingWindow; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; import org.apache.rocketmq.streams.common.functions.MapFunction; import org.apache.rocketmq.streams.common.utils.DataTypeUtil; import org.junit.Before; @@ -39,13 +40,13 @@ public class DataStreamTest implements Serializable { @Before public void init() { - dataStream = StreamBuilder.dataStream("test_namespace", "graph_pipeline"); + dataStream = StreamExecutionEnvironment.getExecutionEnvironment().create("test_namespace", "graph_pipeline"); } @Test public void testFromFile() { dataStream - .fromFile("/Users/junjie.cheng/test.sql", false) + .fromFile("window_msg_10.txt", false) .map(message -> message + "--") .toPrint(1) .start(); @@ -53,7 +54,7 @@ public void testFromFile() { @Test public void testRocketmq() { - DataStreamSource dataStream = StreamBuilder.dataStream("test_namespace", "graph_pipeline"); + DataStreamSource dataStream = StreamExecutionEnvironment.getExecutionEnvironment().create("test_namespace", "graph_pipeline"); dataStream .fromRocketmq("topic_xxxx01", "consumer_xxxx01", "127.0.0.1:9876") .map(message -> message + "--") @@ -83,7 +84,7 @@ public void testFileCheckPoint() { @Test public void testWindow() { - DataStreamSource dataStream = StreamBuilder.dataStream("test_namespace", "graph_pipeline"); + DataStreamSource dataStream = StreamExecutionEnvironment.getExecutionEnvironment().create("test_namespace", "graph_pipeline"); dataStream .fromRocketmq("topic_xxxx03", "consumer_xxxx03", "127.0.0.1:9876") .map(new MapFunction() { @@ -110,7 +111,7 @@ public void testFingerPrintStrategy() { .fromFile("/Users/junjie.cheng/text.txt", false) .map(message -> message + "--") .toPrint(1) - .start(true); + .start(); } @@ -130,7 +131,7 @@ public void testBothStrategy() { @Test public void testMeta() { try { - Class.forName("com.mysql.jdbc.Driver"); + Class.forName(ConfigurationKey.DEFAULT_JDBC_DRIVER); Connection connection = DriverManager.getConnection("", "", ""); DatabaseMetaData metaData = connection.getMetaData(); ResultSet dataFilter = metaData.getColumns(connection.getCatalog(), "%", "XXX", null); diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/FileTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/FileTest.java index 22d1aa90..1a124f71 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/FileTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/FileTest.java @@ -22,13 +22,14 @@ import java.util.List; import java.util.Objects; import org.apache.rocketmq.streams.client.source.DataStreamSource; +import org.apache.rocketmq.streams.common.configurable.IConfigurable; import org.apache.rocketmq.streams.common.utils.FileUtil; import org.junit.Test; public class FileTest { @Test public void testFilter() { - DataStreamSource dataStream = DataStreamSource.create("namespace", "name"); + DataStreamSource dataStream = StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "name"); dataStream.fromFile("/tmp/file.txt", false) .filter((message) -> { JSONObject jsonObject = JSON.parseObject((String) message); @@ -58,4 +59,17 @@ public void testWriteFile() { lines.add(""); FileUtil.write("/tmp/file.txt", lines); } + + @Test + public void testt() { + try { + IConfigurable configurable = (IConfigurable) Class.forName("org.apache.rocketmq.streams.common.topology.stages.FilterChainStage").newInstance(); + } catch (InstantiationException e) { + throw new RuntimeException(e); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } catch (ClassNotFoundException e) { + throw new RuntimeException(e); + } + } } diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/FilterTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/FilterTest.java index 19665fdc..4f67a8ca 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/FilterTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/FilterTest.java @@ -28,22 +28,22 @@ public class FilterTest { @Test - public void testFilter(){ - JSONObject msg=new JSONObject(); - msg.put("name","chris"); - msg.put("age",18); - Rule rule= ExpressionBuilder.createRule("tmp","tmp","(name,==,chris)&(age,>=,18)"); - FilterComponent filterComponent=FilterComponent.getInstance(); - List fireRules=filterComponent.excuteRule(msg,rule); - assertTrue(fireRules.size()==1); + public void testFilter() { + JSONObject msg = new JSONObject(); + msg.put("name", "chris"); + msg.put("age", 18); + Rule rule = ExpressionBuilder.createRule("tmp", "tmp", "(name,==,chris)&(age,>=,18)"); + FilterComponent filterComponent = FilterComponent.getInstance(); + List fireRules = filterComponent.excuteRule(msg, rule); + assertTrue(fireRules.size() == 1); } @Test - public void testFilter2(){ - JSONObject msg=new JSONObject(); - msg.put("name","chris"); - msg.put("age",18); - boolean result= ExpressionBuilder.executeExecute("tmp","(name,==,chris)&(age,>,int,18)",msg); + public void testFilter2() { + JSONObject msg = new JSONObject(); + msg.put("name", "chris"); + msg.put("age", 18); + boolean result = ExpressionBuilder.executeExecute("tmp", "(name,==,chris)&(age,>,int,18)", msg); assertTrue(!result); } } diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/JoinTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/JoinTest.java index a4ff64b2..20d34ef3 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/JoinTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/JoinTest.java @@ -22,40 +22,35 @@ import org.apache.rocketmq.streams.client.transform.DataStream; import org.apache.rocketmq.streams.client.transform.JoinStream.JoinType; import org.apache.rocketmq.streams.common.functions.FilterFunction; +import org.apache.rocketmq.streams.common.functions.MapFunction; import org.junit.Test; public class JoinTest implements Serializable { @Test public void testJoin() { - DataStream leftStream = (StreamBuilder.dataStream("namespace", "name") - .fromFile("/Users/yuanxiaodong/chris/sls_1000.txt") - .filter(new FilterFunction() { - - @Override - public boolean filter(JSONObject value) throws Exception { - if (value.getString("ProjectName") != null && value.getString("LogStore") != null) { - return true; - } - return false; + DataStream leftStream = (StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "name") + .fromCSVFile("/Users/yuanxiaodong/Downloads/sample.csv")) +// .selectFields("dst_ip") + .map(new MapFunction() { + @Override public JSONObject map(JSONObject message) throws Exception { + message.put("left", "true"); + return message; } - })); - - DataStream rightStream = (StreamBuilder.dataStream("namespace", "name2") - .fromFile("/Users/yuanxiaodong/chris/sls_1000.txt") - .filter(new FilterFunction() { + }); - @Override - public boolean filter(JSONObject value) throws Exception { - if (value.getString("ProjectName") != null && value.getString("LogStore") != null) { - return true; - } - return false; + DataStream rightStream = (StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "name2") + .fromCSVFile("/Users/yuanxiaodong/Downloads/sample.csv")) +// .selectFields("dst_ip") + .map(new MapFunction() { + @Override public JSONObject map(JSONObject message) throws Exception { + message.put("rigth", "true"); + return message; } - })); + }); leftStream.join(rightStream).setJoinType(JoinType.INNER_JOIN) - .setCondition("(ProjectName,==,ProjectName)&(LogStore,==,LogStore)") + .setCondition("(dst_ip,==,dst_ip)") .toDataSteam() .toPrint() .start(); @@ -64,7 +59,7 @@ public boolean filter(JSONObject value) throws Exception { @Test public void testDim() { - DataStream stream = (StreamBuilder.dataStream("namespace", "name") + DataStream stream = (StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "name") .fromFile("/Users/junjie.cheng/workspace/rocketmq-streams-apache/rocketmq-streams-clients/src/test/resources/window_msg_10.txt") .filter((FilterFunction) value -> { if (value.getString("ProjectName") == null || value.getString("LogStore") == null) { diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/LeaseTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/LeaseTest.java index 1d3ec8a3..d8c78271 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/LeaseTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/LeaseTest.java @@ -18,8 +18,6 @@ package org.apache.rocketmq.streams.client; import java.util.Date; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; import org.apache.rocketmq.streams.db.driver.DriverBuilder; import org.apache.rocketmq.streams.db.driver.JDBCDriver; import org.apache.rocketmq.streams.lease.LeaseComponent; @@ -31,29 +29,28 @@ public class LeaseTest { - private String URL=""; - protected String USER_NAME=""; - protected String PASSWORD=""; + protected String USER_NAME = ""; + protected String PASSWORD = ""; + private String URL = ""; - public LeaseTest(){ + public LeaseTest() { //正式使用时,在配置文件配置 - ComponentCreator.getProperties().put(ConfigureFileKey.CONNECT_TYPE,"DB"); - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_URL,URL);//数据库连接url - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_USERNAME,USER_NAME);//用户名 - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_PASSWORD,PASSWORD);//password +// ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_URL, URL);//数据库连接url +// ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_USERNAME, USER_NAME);//用户名 +// ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_PASSWORD, PASSWORD);//password /** * 创建lease info表 */ - JDBCDriver driver= DriverBuilder.createDriver(); + JDBCDriver driver = DriverBuilder.createDriver(); driver.execute(LeaseInfo.createTableSQL()); } @Test public void testLease() throws InterruptedException { - String leaseName="lease.test"; - int leaseTime=5; + String leaseName = "lease.test"; + int leaseTime = 5; LeaseComponent.getInstance().getService().startLeaseTask(leaseName, leaseTime, new ILeaseGetCallback() { @Override public void callback(Date nextLeaseDate) { @@ -69,13 +66,13 @@ public void callback(Date nextLeaseDate) { @Test public void testLock() throws InterruptedException { - String name="dipper"; - String lockName="lease.test"; - int leaseTime=5; - boolean success=LeaseComponent.getInstance().getService().lock(name,lockName,leaseTime);//锁定5秒钟 + String name = "dipper"; + String lockName = "lease.test"; + int leaseTime = 5; + boolean success = LeaseComponent.getInstance().getService().lock(name, lockName, leaseTime);//锁定5秒钟 assertTrue(success);//获取锁 Thread.sleep(6000); - assertTrue(!LeaseComponent.getInstance().getService().hasHoldLock(name,lockName));//超期释放 + assertTrue(!LeaseComponent.getInstance().getService().hasHoldLock(name, lockName));//超期释放 } /** @@ -85,14 +82,14 @@ public void testLock() throws InterruptedException { */ @Test public void testHoldLock() throws InterruptedException { - String name="dipper"; - String lockName="lease.test"; - int leaseTime=6; - boolean success=LeaseComponent.getInstance().getService().holdLock(name,lockName,leaseTime);//锁定5秒钟 + String name = "dipper"; + String lockName = "lease.test"; + int leaseTime = 6; + boolean success = LeaseComponent.getInstance().getService().holdLock(name, lockName, leaseTime);//锁定5秒钟 assertTrue(success);//获取锁 Thread.sleep(8000); - assertTrue(LeaseComponent.getInstance().getService().hasHoldLock(name,lockName));//会自动续约,不会释放,可以手动释放 - LeaseComponent.getInstance().getService().unlock(name,lockName); - assertTrue(!LeaseComponent.getInstance().getService().hasHoldLock(name,lockName)); + assertTrue(LeaseComponent.getInstance().getService().hasHoldLock(name, lockName));//会自动续约,不会释放,可以手动释放 + LeaseComponent.getInstance().getService().unlock(name, lockName); + assertTrue(!LeaseComponent.getInstance().getService().hasHoldLock(name, lockName)); } } diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/MQTTTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/MQTTTest.java new file mode 100644 index 00000000..f3acea3c --- /dev/null +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/MQTTTest.java @@ -0,0 +1,43 @@ +/* + * 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.client; + +import java.util.Properties; +import org.apache.rocketmq.streams.client.source.DataStreamSource; +import org.apache.rocketmq.streams.client.strategy.ShuffleStrategy; +import org.apache.rocketmq.streams.client.strategy.Strategy; +import org.junit.Test; + +public class MQTTTest { + @Test + public void testMQTT() { + + DataStreamSource dataStream = StreamExecutionEnvironment.getExecutionEnvironment().create("test_namespace", "graph_pipeline"); + dataStream.fromFile("window_msg_10.txt", true) + .script("x=py('filename','字段1',ProjectName,LogStore)") + .with(ShuffleStrategy.shuffleWithMemory()) + .with(new Strategy() { + @Override public Properties getStrategyProperties() { + Properties properties = new Properties(); + properties.put("dipper.udf.jar.path", "python目录"); + return properties; + } + }) + .toPrint() + .start(); + } +} diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/MqttSourceExample.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/MqttSourceExample.java index 0c21055d..085052cb 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/MqttSourceExample.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/MqttSourceExample.java @@ -18,27 +18,19 @@ import com.alibaba.fastjson.JSONArray; import com.alibaba.fastjson.JSONObject; -import com.google.common.collect.Sets; import java.io.Serializable; -import java.util.Iterator; -import java.util.Set; import org.apache.rocketmq.streams.client.source.DataStreamSource; -import org.apache.rocketmq.streams.client.strategy.ShuffleStrategy; import org.apache.rocketmq.streams.client.transform.DataStream; import org.apache.rocketmq.streams.client.transform.SplitStream; -import org.apache.rocketmq.streams.client.transform.window.Time; -import org.apache.rocketmq.streams.client.transform.window.TumblingWindow; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.configuration.JobConfiguration; import org.apache.rocketmq.streams.common.functions.SplitFunction; -import org.apache.rocketmq.streams.configurable.ConfigurableComponent; import org.junit.Test; public class MqttSourceExample implements Serializable { public static void main(String[] args) throws InterruptedException { - DataStreamSource dataStream = DataStreamSource.create("namespace", "name"); - SplitStream ds = StreamBuilder.dataStream("tmp", "tmp") + DataStreamSource dataStream = StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "name", new JobConfiguration()); + SplitStream ds = StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp") .fromFile("window_msg_10.txt", true) .split(new SplitFunction() { @Override public String split(JSONObject o) { @@ -52,9 +44,8 @@ public static void main(String[] args) throws InterruptedException { ds.select("project-1").toPrint().start(); } - @Test public void test1() { - DataStreamSource dataStreamSource = StreamBuilder.dataStream("", ""); + DataStreamSource dataStreamSource = StreamExecutionEnvironment.getExecutionEnvironment().create("", ""); DataStream ds = dataStreamSource.fromMqtt("tcp://host:port", "", "", "", "") .flatMap(message -> { JSONObject obj = ((JSONObject) message); diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ORMUtilTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ORMUtilTest.java index 9b649d5f..606c1e84 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ORMUtilTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ORMUtilTest.java @@ -21,30 +21,29 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.rocketmq.streams.common.component.ComponentCreator; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; import org.apache.rocketmq.streams.db.driver.orm.ORMUtil; import org.junit.Test; public class ORMUtilTest { - private String URL=""; - protected String USER_NAME=""; - protected String PASSWORD=""; + protected String USER_NAME = ""; + protected String PASSWORD = ""; + private String URL = ""; - public ORMUtilTest(){ + public ORMUtilTest() { //正式使用时,在配置文件配置 - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_URL,URL);//数据库连接url - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_USERNAME,USER_NAME);//用户名 - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_PASSWORD,PASSWORD);//password +// ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_URL, URL);//数据库连接url +// ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_USERNAME, USER_NAME);//用户名 +// ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_PASSWORD, PASSWORD);//password } + @Test - public void testInsert(){ - String namespace="org.apache.configuable.test"; - List personList=new ArrayList<>(); - for(int i=0;i<10;i++){ - personList.add(createPerson(namespace,"chris"+i)); + public void testInsert() { + String namespace = "org.apache.configuable.test"; + List personList = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + personList.add(createPerson(namespace, "chris" + i)); } /** * 不带数据库连接信息(url,userName,Password),使用ConfiguableComponet的连接信息 @@ -55,37 +54,37 @@ public void testInsert(){ } @Test - public void testQueryList(){ - Map paras=new HashMap<>(); - paras.put("age",18); - List personList=ORMUtil.queryForList("select * from person where age >${age} limit 100",paras,Person.class); + public void testQueryList() { + Map paras = new HashMap<>(); + paras.put("age", 18); + List personList = ORMUtil.queryForList("select * from person where age >${age} limit 100", paras, Person.class); } @Test - public void testQueryOneRow(){ - Person personPara=new Person(); + public void testQueryOneRow() { + Person personPara = new Person(); personPara.setAge(18); personPara.setName("chris1"); - Person person=ORMUtil.queryForObject("select * from person where age =${age} and name='${name}' ",personPara,Person.class,URL,USER_NAME,PASSWORD); + Person person = ORMUtil.queryForObject("select * from person where age =${age} and name='${name}' ", personPara, Person.class, URL, USER_NAME, PASSWORD); } /** * 创建configuable对象 + * * @param namespace * @return */ - protected Person createPerson(String namespace,String name){ - Person person=new Person(); + protected Person createPerson(String namespace, String name) { + Person person = new Person(); person.setName(name); person.setAge(18); person.setNameSpace(namespace); - person.setConfigureName("peronName"); + person.setName("peronName"); person.setType("person"); return person; } } - class Person extends BasedConfigurable { @ENVDependence private String name; @@ -98,7 +97,7 @@ public static Person createPerson(String namespace) { Person person = new Person(); person.setNameSpace(namespace); person.setType("person"); - person.setConfigureName("Chris"); + person.setName("Chris"); person.setName("Chris"); List addresses = new ArrayList<>(); addresses.add("huilongguan"); @@ -163,7 +162,7 @@ public void setChildName2Age(Map childName2Age) { public Object clone() { Person person = null; try { - person = (Person)super.clone(); + person = (Person) super.clone(); } catch (CloneNotSupportedException e) { System.out.println("clone error " + e); } diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/RocketMQGroupByTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/RocketMQGroupByTest.java index f5e1ab77..3e8572b7 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/RocketMQGroupByTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/RocketMQGroupByTest.java @@ -18,7 +18,6 @@ import java.io.Serializable; import java.util.concurrent.atomic.AtomicLong; -import org.apache.rocketmq.streams.client.source.DataStreamSource; import org.apache.rocketmq.streams.client.transform.window.Time; import org.apache.rocketmq.streams.client.transform.window.TumblingWindow; import org.apache.rocketmq.streams.common.context.AbstractContext; @@ -30,7 +29,7 @@ public class RocketMQGroupByTest implements Serializable { @Test public void testRocketMq() throws Exception { - DataStreamSource.create("tmp", "tmp") + StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp") .fromRocketmq("dipper_test_write_merge5", "dipper_group", true, "localhost:9876") .window(TumblingWindow.of(Time.seconds(5))) .groupBy("host_uuid", "cmdline") diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/RocketMQTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/RocketMQTest.java index e0648d1c..b9142fed 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/RocketMQTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/RocketMQTest.java @@ -22,7 +22,6 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicLong; -import org.apache.rocketmq.streams.client.source.DataStreamSource; 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.sinkcache.impl.MultiSplitMessageCache; @@ -31,14 +30,15 @@ import org.apache.rocketmq.streams.common.context.Message; import org.apache.rocketmq.streams.common.functions.FlatMapFunction; import org.apache.rocketmq.streams.common.functions.ForEachMessageFunction; -import org.apache.rocketmq.streams.sink.RocketMQSink; +import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.rocketmq.sink.RocketMQSink; import org.junit.Test; public class RocketMQTest implements Serializable { @Test public void testCommonWrite() { - DataStreamSource.create("tmp", "tmp") + StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp") .fromFile("/tmp/aegis_proc_public.txt", true) .forEachMessage(new ForEachMessageFunction() { @@ -63,7 +63,7 @@ public void testCommonWrite() { @Test public void testWrite() { - DataStreamSource.create("tmp", "tmp") + StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp") .fromFile("/tmp/aegis_proc_public.txt", true) .forEachMessage(new ForEachMessageFunction() { @@ -85,7 +85,7 @@ public void testWrite() { }) .forEachMessage(new ForEachMessageFunction() { MultiSplitMessageCache messageCache = null; - RocketMQSink sink = new RocketMQSink("localhost:9876", "dipper_test_write_merge4"); + RocketMQSink sink = new RocketMQSink("11.166.49.226:9876", "dipper_test_write_merge4"); @Override public void foreach(IMessage message, AbstractContext context) { if (messageCache == null) { @@ -126,8 +126,8 @@ public void testWrite() { @Test public void testConsumer() { - DataStreamSource.create("tmp", "tmp") - .fromRocketmq("dipper_test_write_merge", "dipper_group", true, "localhost:9876") + StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp") + .fromRocketmq("dipper_test_write_merge", "dipper_group", true, "11.166.49.226:9876") .flatMap(new FlatMapFunction() { @Override public List flatMap(JSONObject message) throws Exception { JSONArray jsonArray = message.getJSONArray("data"); @@ -193,4 +193,19 @@ public void testConsumer() { }) .start(); } + + @Test + public void testSink() throws InterruptedException { + StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp").fromFile("window_msg_10.txt") + .toPrint().toRocketmq("tmp_topic", "11.166.49.226:9876").start(); + Thread.sleep(5000); + } + + @Test + public void testSource() { + String groupName = "group6"; + System.out.println(groupName + " start time is " + DateUtil.getCurrentTimeString()); + StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp").fromRocketmq("tmp_topic", groupName, "*", true, "11.166.49.226:9876", true) + .toPrint().start(); + } } diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ScriptTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ScriptTest.java index 2ea9f2cd..56c64c82 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ScriptTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ScriptTest.java @@ -25,12 +25,12 @@ public class ScriptTest { @Test - public void testFloor(){ - JSONObject msg=new JSONObject(); - msg.put("_input",3233223.434334); - IMessage message=new Message(msg); - Context context=new Context(message); - ScriptComponent.getInstance().getService().executeScript(msg,"a=division(json_field(_input, 'lastTime'), 1000)"); + public void testFloor() { + JSONObject msg = new JSONObject(); + msg.put("_input", 3233223.434334); + IMessage message = new Message(msg); + Context context = new Context(message); + ScriptComponent.getInstance().getService().executeScript(msg, "a=division(json_field(_input, 'lastTime'), 1000)"); System.out.println(msg); } } diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SourceMergerTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SourceMergerTest.java new file mode 100644 index 00000000..c37897a4 --- /dev/null +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SourceMergerTest.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.client; + +import com.alibaba.fastjson.JSONObject; +import org.apache.rocketmq.streams.client.transform.DataStream; +import org.apache.rocketmq.streams.common.functions.MapFunction; +import org.junit.Test; + +/** + * /** + * + * @description + */ +public class SourceMergerTest { + + @Test + public void testUnion() { + // ComponentCreator.getProperties().put(ConfigureFileKey.CONNECT_TYPE, IConfigurableService.JOB_SERVICE_NAME); + DataStream formatStream = (StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "name") + .fromFile("window_msg_100.txt") + .map(new MapFunction() { + @Override public JSONObject map(JSONObject message) throws Exception { + message.put("left", "1"); + return message; + } + })); + + DataStream wendu = formatStream + .map(new MapFunction() { + @Override public JSONObject map(JSONObject message) throws Exception { + message.put("wendu", true); + return message; + } + }); + + DataStream shidu = formatStream + .map(new MapFunction() { + @Override public JSONObject map(JSONObject message) throws Exception { + message.put("shidu", true); + return message; + } + }); + + DataStream liangdu = formatStream + .map(new MapFunction() { + @Override public JSONObject map(JSONObject message) throws Exception { + message.put("liangdu", true); + return message; + } + }); + + wendu.union(shidu).union(liangdu).toPrint().start(); + + } + +} diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SourceTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SourceTest.java index 10d144e1..9234a104 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SourceTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SourceTest.java @@ -18,7 +18,6 @@ import java.io.Serializable; import java.util.concurrent.atomic.AtomicLong; -import org.apache.rocketmq.streams.client.source.DataStreamSource; import org.apache.rocketmq.streams.common.channel.impl.file.FileSource; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; @@ -46,7 +45,7 @@ public Object doMessage(IMessage message, AbstractContext context) { public void testoutputdata() { AtomicLong count = new AtomicLong(0); AtomicLong startTime = new AtomicLong(System.currentTimeMillis()); - DataStreamSource.create("tmp", "tmp") + StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp") .fromFile("/tmp/aegis_proc_public.txt") .toRocketmq("dipper_test_1", "dipper_group", "localhost:9876") .forEachMessage(new ForEachMessageFunction() { @@ -70,7 +69,7 @@ public void testoutputdata() { public void testImportMsgFromFile() { AtomicLong count = new AtomicLong(0); AtomicLong startTime = new AtomicLong(System.currentTimeMillis()); - DataStreamSource.create("tmp", "tmp") + StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp") .fromRocketmq("dipper_test_1", "dipper_group1", true, "localhost:9876") // .window(TumblingWindow.of(Time.seconds(10))) // .count("c") @@ -96,7 +95,7 @@ public void testImportMsgFromFile() { @Test public void testImportMsgFromSource() { - DataStreamSource.create("tmp", "tmp") + StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp") .fromRocketmq("TOPIC_AEGIS_DETECT_MSG", "chris_test", "T_MSG_PROC", true, null) .toFile("/tmp/aegis_proc.txt", true) .start(); @@ -104,7 +103,7 @@ public void testImportMsgFromSource() { @Test public void testImportMsgFromNet() { - DataStreamSource.create("tmp", "tmp") + StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp") .fromRocketmq("TOPIC_AEGIS_DETECT_MSG", "chris_test", "T_MSG_NETSTAT", true, null) .toFile("/tmp/aegis_net.txt", true) .start(); diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SplitTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SplitTest.java index ff5e860a..6067c263 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SplitTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SplitTest.java @@ -34,7 +34,7 @@ public class SplitTest implements Serializable { @Test public void testOperator() throws InterruptedException { - DataStream stream = (StreamBuilder.dataStream("namespace", "name").fromFile("/Users/yuanxiaodong/chris/sls_1000.txt").flatMap(new FlatMapFunction() { + DataStream stream = (StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "name").fromFile("/Users/yuanxiaodong/chris/sls_1000.txt").flatMap(new FlatMapFunction() { @Override public List flatMap(String message) throws Exception { List msgs = new ArrayList<>(); @@ -61,7 +61,7 @@ public String split(JSONObject o) { DataStream children = splitStream.select("children"); DataStream adult = splitStream.select("adult"); - children.union(adult).join("dburl", "dbUserName", "dbPassowrd", "tableNameOrSQL", 5).setCondition("(name,==,name)").toDataSteam().window(TumblingWindow.of(Time.seconds(5))).groupBy("ProjectName", "LogStore").setLocalStorageOnly(true).count("total").sum("OutFlow", "OutFlow").sum("InFlow", "InFlow").toDataSteam().toPrint().asyncStart(); + children.union(adult).join("dburl", "dbUserName", "dbPassowrd", "tableNameOrSQL", 5).setCondition("(name,==,name)").toDataSteam().window(TumblingWindow.of(Time.seconds(5))).groupBy("ProjectName", "LogStore").setLocalStorageOnly(true).count("total").sum("OutFlow", "OutFlow").sum("InFlow", "InFlow").toDataSteam().toPrint().start(); while (true) { Thread.sleep(1000); } @@ -70,7 +70,7 @@ public String split(JSONObject o) { @Test public void testDim() { - DataStream stream = (StreamBuilder.dataStream("namespace", "name").fromFile("/Users/yuanxiaodong/chris/sls_1000.txt").filter(new FilterFunction() { + DataStream stream = (StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "name").fromFile("/Users/yuanxiaodong/chris/sls_1000.txt").filter(new FilterFunction() { @Override public boolean filter(JSONObject value) throws Exception { diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/UnionTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/UnionTest.java index 2bef585d..ce2b325d 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/UnionTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/UnionTest.java @@ -21,35 +21,28 @@ import java.io.Serializable; import org.apache.rocketmq.streams.client.transform.DataStream; import org.apache.rocketmq.streams.common.functions.FilterFunction; +import org.apache.rocketmq.streams.common.functions.MapFunction; import org.junit.Test; public class UnionTest implements Serializable { @Test public void testUnion() { - DataStream leftStream = (StreamBuilder.dataStream("namespace", "name") + DataStream leftStream = (StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "name") .fromFile("/Users/yuanxiaodong/chris/sls_1000.txt") - .filter(new FilterFunction() { - - @Override - public boolean filter(JSONObject value) throws Exception { - if (value.getString("ProjectName") == null || value.getString("LogStore") == null) { - return true; - } - return false; + .map(new MapFunction() { + @Override public JSONObject map(JSONObject message) throws Exception { + message.put("left", true); + return message; } })); - DataStream rightStream = (StreamBuilder.dataStream("namespace", "name2") + DataStream rightStream = (StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "name2") .fromFile("/Users/yuanxiaodong/chris/sls_1000.txt") - .filter(new FilterFunction() { - - @Override - public boolean filter(JSONObject value) throws Exception { - if (value.getString("ProjectName") == null || value.getString("LogStore") == null) { - return true; - } - return false; + .map(new MapFunction() { + @Override public JSONObject map(JSONObject message) throws Exception { + message.put("rigth", true); + return message; } })); @@ -58,7 +51,7 @@ public boolean filter(JSONObject value) throws Exception { @Test public void testDim() { - DataStream stream = (StreamBuilder.dataStream("namespace", "name") + DataStream stream = (StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "name") .fromFile("/Users/yuanxiaodong/chris/sls_1000.txt") .filter(new FilterFunction() { diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/UserDefinedSourceTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/UserDefinedSourceTest.java index a7ad61b0..cd5e49a1 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/UserDefinedSourceTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/UserDefinedSourceTest.java @@ -18,5 +18,4 @@ public class UserDefinedSourceTest { - } diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/WindowTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/WindowTest.java index 90cf905f..3306b471 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/WindowTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/WindowTest.java @@ -24,16 +24,15 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Properties; import org.apache.commons.io.FileUtils; -import org.apache.commons.lang3.tuple.Pair; -import org.apache.rocketmq.streams.client.transform.window.SessionWindow; +import org.apache.rocketmq.streams.client.transform.window.HoppingWindow; import org.apache.rocketmq.streams.client.transform.window.Time; import org.apache.rocketmq.streams.client.transform.window.TumblingWindow; -import org.apache.rocketmq.streams.common.functions.ForEachFunction; import org.apache.rocketmq.streams.common.functions.MapFunction; -import org.apache.rocketmq.streams.common.functions.ReduceFunction; +import org.apache.rocketmq.streams.common.model.Pair; import org.apache.rocketmq.streams.common.utils.DateUtil; -import org.apache.rocketmq.streams.script.service.IAccumulator; +import org.apache.rocketmq.streams.window.model.WindowInstance; import org.junit.Assert; import org.junit.Test; @@ -41,33 +40,49 @@ public class WindowTest implements Serializable { @Test public void testWindow() { - StreamBuilder.dataStream("namespace", "name") - .fromFile("/Users/duheng/project/opensource/sls_100.txt", false) - .map((MapFunction) message -> JSONObject.parseObject(message)) + /** + * 把with中key,value放到properties中 + */ + Properties properties = new Properties(); + properties.put("topic", ""); + properties.put("bootstrap.servers", ""); + properties.put("group.id", ""); + properties.put("batchSize", 500); + + StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "name") + .fromFile("window_msg_100.txt", true) .window(TumblingWindow.of(Time.seconds(5))) - .setTimeField("时间字段") - .waterMark(10) .groupBy("ProjectName", "LogStore") - .setLocalStorageOnly(true) - .reduce(new ReduceFunction() { - - @Override public IAccumulator reduce(IAccumulator acccumulator, JSONObject msg) { - return null; + .setLocalStorageOnly(false) + .saveWindowMsg(new MapFunction>() { + @Override public JSONObject map(Pair message) throws Exception { + WindowInstance windowInstance = message.getKey(); + JSONObject msg = message.getValue(); + JSONObject newMsg = new JSONObject(); + newMsg.putAll(msg); + newMsg.put("window_start_time", windowInstance.getStartTime()); + newMsg.put("window_end_time", windowInstance.getEndTime()); + + return newMsg; } - }) - .forEach(new ForEachFunction() { - protected int sum = 0; + }, "kafka", properties) - @Override - public void foreach(JSONObject o) { - int total = o.getInteger("total"); - sum = sum + total; - o.put("sum(total)", sum); - } - }).toPrint().start(); + .toDataSteam() + .toPrint().start(); } + @Test + public void testHopWindow() { +// ComponentCreator.setProperties(""); + StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp") + .fromFile("window_msg_100.txt", true).window(HoppingWindow.of(Time.seconds(10), Time.seconds(5))) + .count("asCount") + .setLocalStorageOnly(false) + .toDataSteam() + .toPrint().start(); + } + // @Test // public void testWindowFromMetaq() throws InterruptedException { // String topic = "TOPIC_DIPPER_SYSTEM_MSG_4"; @@ -97,102 +112,102 @@ public void foreach(JSONObject o) { // }).toPrint().start(); // } - @Test - public void testSession() { - //dataset - List behaviorList = new ArrayList<>(); - - JSONObject userA = new JSONObject(); - userA.put("time", DateUtil.parse("2021-09-09 10:00:00")); - userA.put("user", "userA"); - userA.put("movie", "movie001"); - userA.put("flag", 1); - behaviorList.add(userA.toJSONString()); - - userA = new JSONObject(); - userA.put("time", DateUtil.parse("2021-09-09 10:00:01")); - userA.put("user", "userA"); - userA.put("movie", "movie002"); - userA.put("flag", 1); - behaviorList.add(userA.toJSONString()); - - JSONObject userB = new JSONObject(); - userB.put("time", DateUtil.parse("2021-09-09 10:00:00")); - userB.put("user", "userB"); - userB.put("movie", "movie003"); - userB.put("flag", 1); - behaviorList.add(userB.toJSONString()); - - JSONObject userC = new JSONObject(); - userC.put("time", DateUtil.parse("2021-09-09 10:00:00")); - userC.put("user", "userC"); - userC.put("movie", "movie004"); - userC.put("flag", 1); - behaviorList.add(userC.toJSONString()); - - userC = new JSONObject(); - userC.put("time", DateUtil.parse("2021-09-09 10:00:06")); - userC.put("user", "userC"); - userC.put("movie", "movie005"); - userC.put("flag", 1); - behaviorList.add(userC.toJSONString()); - - File dataFile = null; - try { - dataFile = File.createTempFile("behavior", ".txt"); - FileUtils.writeLines(dataFile, behaviorList); - } catch (Exception e) { - e.printStackTrace(); - } - - File resultFile = null; - try { - resultFile = File.createTempFile("behavior.txt", ".session"); - } catch (Exception e) { - e.printStackTrace(); - } - - StreamBuilder.dataStream("namespace", "session_test") - .fromFile(dataFile.getAbsolutePath(), false) - .map((MapFunction) message -> JSONObject.parseObject(message)) - .window(SessionWindow.of(Time.seconds(5), "time")) - .groupBy("user") - .setLocalStorageOnly(true) - .sum("flag", "count") - .toDataSteam() - .toFile(resultFile.getAbsolutePath()).start(true); - - try { - Thread.sleep(1 * 60 * 1000); - List sessionList = FileUtils.readLines(resultFile, "UTF-8"); - Map, Integer>>> sessionMap = new HashMap<>(4); - for (String line : sessionList) { - JSONObject object = JSONObject.parseObject(line); - String user = object.getString("user"); - String startTime = object.getString("start_time"); - String endTime = object.getString("end_time"); - Integer value = object.getIntValue("count"); - if (!sessionMap.containsKey(user)) { - sessionMap.put(user, new ArrayList<>()); - } - sessionMap.get(user).add(Pair.of(Pair.of(startTime, endTime), value)); - } - Assert.assertEquals(3, sessionMap.size()); - Assert.assertEquals(1, sessionMap.get("userA").size()); - Assert.assertEquals("2021-09-09 10:00:06", sessionMap.get("userA").get(0).getLeft().getRight()); - Assert.assertEquals(2, sessionMap.get("userC").size()); - Assert.assertEquals("2021-09-09 10:00:05", sessionMap.get("userC").get(0).getLeft().getRight()); - Assert.assertEquals("2021-09-09 10:00:06", sessionMap.get("userC").get(1).getLeft().getLeft()); - Assert.assertEquals(1, sessionMap.get("userB").size()); - - } catch (Exception e) { - e.printStackTrace(); - } finally { - dataFile.deleteOnExit(); - resultFile.deleteOnExit(); - } - - } +// @Test +// public void testSession() { +// //dataset +// List behaviorList = new ArrayList<>(); +// +// JSONObject userA = new JSONObject(); +// userA.put("time", DateUtil.parse("2021-09-09 10:00:00")); +// userA.put("user", "userA"); +// userA.put("movie", "movie001"); +// userA.put("flag", 1); +// behaviorList.add(userA.toJSONString()); +// +// userA = new JSONObject(); +// userA.put("time", DateUtil.parse("2021-09-09 10:00:01")); +// userA.put("user", "userA"); +// userA.put("movie", "movie002"); +// userA.put("flag", 1); +// behaviorList.add(userA.toJSONString()); +// +// JSONObject userB = new JSONObject(); +// userB.put("time", DateUtil.parse("2021-09-09 10:00:00")); +// userB.put("user", "userB"); +// userB.put("movie", "movie003"); +// userB.put("flag", 1); +// behaviorList.add(userB.toJSONString()); +// +// JSONObject userC = new JSONObject(); +// userC.put("time", DateUtil.parse("2021-09-09 10:00:00")); +// userC.put("user", "userC"); +// userC.put("movie", "movie004"); +// userC.put("flag", 1); +// behaviorList.add(userC.toJSONString()); +// +// userC = new JSONObject(); +// userC.put("time", DateUtil.parse("2021-09-09 10:00:06")); +// userC.put("user", "userC"); +// userC.put("movie", "movie005"); +// userC.put("flag", 1); +// behaviorList.add(userC.toJSONString()); +// +// File dataFile = null; +// try { +// dataFile = File.createTempFile("behavior", ".txt"); +// FileUtils.writeLines(dataFile, behaviorList); +// } catch (Exception e) { +// e.printStackTrace(); +// } +// +// File resultFile = null; +// try { +// resultFile = File.createTempFile("behavior.txt", ".session"); +// } catch (Exception e) { +// e.printStackTrace(); +// } +// +// StreamBuilder.dataStream("namespace", "session_test") +// .fromFile(dataFile.getAbsolutePath(), false) +// .map((MapFunction) message -> JSONObject.parseObject(message)) +// .window(SessionWindow.of(Time.seconds(5), "time")) +// .groupBy("user") +// .setLocalStorageOnly(true) +// .sum("flag", "count") +// .toDataSteam() +// .toFile(resultFile.getAbsolutePath()).start(true); +// +// try { +// Thread.sleep(1 * 60 * 1000); +// List sessionList = FileUtils.readLines(resultFile, "UTF-8"); +// Map, Integer>>> sessionMap = new HashMap<>(4); +// for (String line : sessionList) { +// JSONObject object = JSONObject.parseObject(line); +// String user = object.getString("user"); +// String startTime = object.getString("start_time"); +// String endTime = object.getString("end_time"); +// Integer value = object.getIntValue("count"); +// if (!sessionMap.containsKey(user)) { +// sessionMap.put(user, new ArrayList<>()); +// } +// sessionMap.get(user).add(Pair.of(Pair.of(startTime, endTime), value)); +// } +// Assert.assertEquals(3, sessionMap.size()); +// Assert.assertEquals(1, sessionMap.get("userA").size()); +// Assert.assertEquals("2021-09-09 10:00:06", sessionMap.get("userA").get(0).getLeft().getRight()); +// Assert.assertEquals(2, sessionMap.get("userC").size()); +// Assert.assertEquals("2021-09-09 10:00:05", sessionMap.get("userC").get(0).getLeft().getRight()); +// Assert.assertEquals("2021-09-09 10:00:06", sessionMap.get("userC").get(1).getLeft().getLeft()); +// Assert.assertEquals(1, sessionMap.get("userB").size()); +// +// } catch (Exception e) { +// e.printStackTrace(); +// } finally { +// dataFile.deleteOnExit(); +// resultFile.deleteOnExit(); +// } +// +// } @Test public void testCountDistinct() { @@ -250,7 +265,7 @@ public void testCountDistinct() { e.printStackTrace(); } - StreamBuilder.dataStream("namespace", "count_distinct_test") + StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "count_distinct_test") .fromFile(dataFile.getAbsolutePath(), false) .map((MapFunction) message -> JSONObject.parseObject(message)) .window(TumblingWindow.of(Time.minutes(5), "time")) @@ -260,7 +275,7 @@ public void testCountDistinct() { .count_distinct_large("page", "uv_large") .count_distinct_2("page", "uv_2") .toDataSteam() - .toFile(resultFile.getAbsolutePath()).start(true); + .toFile(resultFile.getAbsolutePath()).start(); try { Thread.sleep(6 * 60 * 1000); diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/CountFromFileTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/CountFromFileTest.java index 606de357..7e1cf167 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/CountFromFileTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/CountFromFileTest.java @@ -16,10 +16,9 @@ */ package org.apache.rocketmq.streams.client.example; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.transform.window.HoppingWindow; import org.apache.rocketmq.streams.client.transform.window.Time; -import org.apache.rocketmq.streams.client.transform.window.TumblingWindow; import org.junit.Test; public class CountFromFileTest { @@ -27,22 +26,20 @@ public class CountFromFileTest { /** * 窗口10分钟,触发前,每一分钟发送一次数据,过了触发时间,每5秒发一次数据 */ - @Test - public void testCountTumplFromFile() { - StreamBuilder.dataStream("tmp", "tmp") + @Test public void testCountTumplFromFile() { + + StreamExecutionEnvironment evn = StreamExecutionEnvironment.getExecutionEnvironment(); + + evn.create("", "") .fromFile("window_msg_88121.txt", true) - .window(TumblingWindow.of(Time.seconds(5))) - .count("count_result") - .toDataSteam() + .count() .toPrint() .start(); + } - @Test - public void testHopCountFromFile() { - StreamBuilder.dataStream("tmp", "tmp") - // filepath can be either a classpath file or an absolute path file address - //filepath can set window_msg_10 or window_msg_100 window_msg_1000 or window_msg_10000 + @Test public void testHopCountFromFile() { + StreamExecutionEnvironment.getExecutionEnvironment().create("", "") .fromFile("window_msg_10000.txt", true) .window(HoppingWindow.of(Time.seconds(10), Time.seconds(5))) .count("count_result") diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/CountSupportEmitTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/CountSupportEmitTest.java index 26bc9c15..fbc40981 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/CountSupportEmitTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/CountSupportEmitTest.java @@ -16,10 +16,9 @@ */ package org.apache.rocketmq.streams.client.example; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.transform.window.HoppingWindow; import org.apache.rocketmq.streams.client.transform.window.Time; -import org.apache.rocketmq.streams.client.transform.window.TumblingWindow; import org.junit.Test; public class CountSupportEmitTest { @@ -29,12 +28,9 @@ public class CountSupportEmitTest { */ @Test public void testCountTumplFromFile() { - StreamBuilder.dataStream("tmp", "tmp") + StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp") .fromFile("window_msg_88121.txt", true) - .window(TumblingWindow.of(Time.minutes(10))) - .emitBeforeFire(Time.seconds(5)) - .count("count_result") - .toDataSteam() + .count(5) .toPrint() .start(); ; @@ -42,7 +38,7 @@ public void testCountTumplFromFile() { @Test public void testHopCountFromFile() { - StreamBuilder.dataStream("tmp", "tmp") + StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp") // filepath can be either a classpath file or an absolute path file address //filepath can set window_msg_10 or window_msg_100 window_msg_1000 or window_msg_10000 .fromFile("window_msg_10000.txt", true) diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/DimTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/DimTest.java index 1004925c..3e39d42e 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/DimTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/DimTest.java @@ -16,7 +16,7 @@ */ package org.apache.rocketmq.streams.client.example; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.transform.JoinStream; import org.junit.Test; @@ -24,7 +24,7 @@ public class DimTest { @Test public void testInnerJoinDim() { - StreamBuilder.dataStream("tmp", "tmp") + StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp") .fromFile("window_msg_10.txt", true) .join("classpath://dim.txt", 10000) .setJoinType(JoinStream.JoinType.INNER_JOIN) @@ -36,7 +36,7 @@ public void testInnerJoinDim() { @Test public void testLeftDim() { - StreamBuilder.dataStream("tmp", "tmp") + StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp") .fromFile("window_msg_10.txt", true) .join("classpath://dim.txt", 10000) .setJoinType(JoinStream.JoinType.LEFT_JOIN) diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/JoinTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/JoinTest.java index 1a472db3..fc17c9bc 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/JoinTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/JoinTest.java @@ -24,7 +24,7 @@ import org.apache.rocketmq.client.producer.SendResult; import org.apache.rocketmq.common.message.Message; import org.apache.rocketmq.remoting.common.RemotingHelper; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.transform.DataStream; import org.junit.Test; @@ -60,9 +60,9 @@ public void testProduce() { @Test public void testInnerJoin() { - DataStream left = StreamBuilder.dataStream("tmp", "tmp").fromFile("window_msg_10.txt", true); + DataStream left = StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp").fromFile("window_msg_10.txt", true); - DataStream right = StreamBuilder.dataStream("tmp", "tmp2").fromFile("dim.txt", true); + DataStream right = StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp2").fromFile("dim.txt", true); left.join(right).on("(ProjectName,=,project)").toDataSteam().toPrint().start(); @@ -70,9 +70,9 @@ public void testInnerJoin() { @Test public void testLeftDim() { - DataStream left = StreamBuilder.dataStream("tmp", "tmp").fromFile("window_msg_10.txt", true); + DataStream left = StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp").fromFile("window_msg_10.txt", true); - DataStream right = StreamBuilder.dataStream("tmp", "tmp2").fromFile("dim.txt", true); + DataStream right = StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp2").fromFile("dim.txt", true); left.leftJoin(right).on("(ProjectName,=,project)").toDataSteam().toPrint().start(); } @@ -80,7 +80,7 @@ public void testLeftDim() { @Test public void testRocketmqJoin() { - DataStream left = StreamBuilder.dataStream("tmp", "tmp").fromRocketmq("TopicTest", "groupA", true, "localhost:9876"); + DataStream left = StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp").fromRocketmq("TopicTest", "groupA", true, "localhost:9876"); left.toPrint().start(); //DataStream right = StreamBuilder.dataStream("tmp", "tmp2").fromFile("dim.txt", true); // DataStream right = StreamBuilder.dataStream("tmp", "tmp").fromRocketmq("TopicTest", "groupB", true, "localhost:9876"); diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/SplitTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/SplitTest.java index 5138be14..f720a37f 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/SplitTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/SplitTest.java @@ -17,16 +17,14 @@ package org.apache.rocketmq.streams.client.example; import com.alibaba.fastjson.JSONObject; -import org.apache.rocketmq.streams.client.StreamBuilder; -import org.apache.rocketmq.streams.client.source.DataStreamSource; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.transform.SplitStream; import org.apache.rocketmq.streams.common.functions.SplitFunction; public class SplitTest { public static void main(String[] args) throws InterruptedException { - DataStreamSource dataStream = DataStreamSource.create("namespace", "name"); - SplitStream ds = StreamBuilder.dataStream("tmp", "tmp") + SplitStream ds = StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp") .fromFile("window_msg_10.txt", true) .split(new SplitFunction() { @Override public String split(JSONObject o) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IAfterConfigurableRefreshListener.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/model/JobModel.java similarity index 72% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IAfterConfigurableRefreshListener.java rename to rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/model/JobModel.java index d9f06c1d..858f8eaa 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IAfterConfigurableRefreshListener.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/model/JobModel.java @@ -14,12 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.common.configurable; +package org.apache.rocketmq.streams.client.model; -public interface IAfterConfigurableRefreshListener { +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; - /** - * 当configurable数据全部加载完成时,调用实现这个接口的configurable对象 - */ - void doProcessAfterRefreshConfigurable(IConfigurableService configurableService); +public class JobModel extends BasedConfigurable { + public JobModel() { + setType("jobModel"); + } } diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/model/RuleContainer.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/model/RuleContainer.java new file mode 100644 index 00000000..3f4e297d --- /dev/null +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/model/RuleContainer.java @@ -0,0 +1,51 @@ +/* + * 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.client.model; + +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; +import org.apache.rocketmq.streams.common.configurable.annotation.ConfigurableReference; +import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; + +public class RuleContainer extends BasedConfigurable { + @ConfigurableReference protected TestRule rule; + @ENVDependence protected String ak; + + public RuleContainer() { + setType("container"); + } + + @Override protected boolean initConfigurable() { + System.out.println("finish initConfigurable"); + return super.initConfigurable(); + } + + public TestRule getRule() { + return rule; + } + + public void setRule(TestRule rule) { + this.rule = rule; + } + + public String getAk() { + return ak; + } + + public void setAk(String ak) { + this.ak = ak; + } +} diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/model/TestRule.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/model/TestRule.java new file mode 100644 index 00000000..30d8a7ca --- /dev/null +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/model/TestRule.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.client.model; + +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; + +public class TestRule extends BasedConfigurable { + + public TestRule() { + setType("testRule"); + } +} diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/source/MqttStreamsTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/source/MqttStreamsTest.java index 9f272304..0eb4e1a4 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/source/MqttStreamsTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/source/MqttStreamsTest.java @@ -17,7 +17,7 @@ package org.apache.rocketmq.streams.client.source; import com.alibaba.fastjson.JSONObject; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.strategy.ShuffleStrategy; import org.apache.rocketmq.streams.client.transform.window.Time; import org.apache.rocketmq.streams.client.transform.window.TumblingWindow; @@ -29,7 +29,7 @@ public class MqttStreamsTest { @Before public void init() { - dataStream = StreamBuilder.dataStream("test_namespace", "graph_pipeline"); + dataStream = StreamExecutionEnvironment.getExecutionEnvironment().create("test_namespace", "graph_pipeline"); } @Test diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/source/UserDefinedSource.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/source/UserDefinedSource.java index d8932aa0..87ec7dd0 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/source/UserDefinedSource.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/source/UserDefinedSource.java @@ -16,7 +16,9 @@ */ package org.apache.rocketmq.streams.client.source; +import java.util.List; import org.apache.rocketmq.streams.common.channel.source.AbstractSource; +import org.apache.rocketmq.streams.common.channel.split.ISplit; public class UserDefinedSource extends AbstractSource { @@ -24,11 +26,11 @@ public class UserDefinedSource extends AbstractSource { return false; } - @Override public boolean supportRemoveSplitFind() { - return false; + @Override protected void destroySource() { + } - @Override protected boolean isNotDataSplit(String queueId) { - return false; + @Override public List> fetchAllSplits() { + return null; } } diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/AbstractWindowFireModeTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/AbstractWindowFireModeTest.java index 39c04ee8..2842496c 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/AbstractWindowFireModeTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/AbstractWindowFireModeTest.java @@ -29,12 +29,12 @@ import org.apache.rocketmq.streams.common.functions.MapFunction; import org.apache.rocketmq.streams.common.utils.DateUtil; - public abstract class AbstractWindowFireModeTest implements Serializable { - protected Date date=new Date(); - public AbstractWindowFireModeTest(){ + protected Date date = new Date(); + + public AbstractWindowFireModeTest() { - date.setYear(2021-1900); + date.setYear(2021 - 1900); date.setMonth(6); date.setDate(14); date.setHours(12); @@ -44,26 +44,27 @@ public AbstractWindowFireModeTest(){ } public void testWindowFireMode0(boolean isLocalOnly) throws InterruptedException { - testWindowFireMode1(isLocalOnly,5); + testWindowFireMode1(isLocalOnly, 5); } - public void testWindowFireMode0(boolean isLocalOnly,int windowSize) throws InterruptedException { + public void testWindowFireMode0(boolean isLocalOnly, int windowSize) throws InterruptedException { createSourceDataStream().map(new MapFunction() { - int count=0; - Long time=null; + int count = 0; + Long time = null; + @Override public JSONObject map(String message) throws Exception { - if(time==null){ - time=date.getTime(); - }else { - time+=count; + if (time == null) { + time = date.getTime(); + } else { + time += count; } count++; - JSONObject msg=JSONObject.parseObject(message); + JSONObject msg = JSONObject.parseObject(message); - msg.put("logTime",time); + msg.put("logTime", time); return msg; } @@ -71,27 +72,30 @@ public JSONObject map(String message) throws Exception { .window(TumblingWindow.of(Time.seconds(windowSize))) .groupBy("ProjectName", "LogStore") .setLocalStorageOnly(isLocalOnly) - .setMaxMsgGap(isLocalOnly?10L:20L) + .setMaxMsgGap(isLocalOnly ? 10L : 20L) .setTimeField("logTime") .count("total") .sum("OutFlow", "OutFlow") .sum("InFlow", "inflow") .toDataSteam() .forEach(new ForEachFunction() { - AtomicInteger sum = new AtomicInteger(0) ; + AtomicInteger sum = new AtomicInteger(0); + @Override public synchronized void foreach(JSONObject o) { int total = o.getInteger("total"); - o.put("sum(total)", sum.addAndGet(total)); + o.put("sum(total)", sum.addAndGet(total)); } }).toPrint().start(); } + public void testWindowFireMode1(boolean isLocalOnly) throws InterruptedException { - testWindowFireMode1(isLocalOnly,5); + testWindowFireMode1(isLocalOnly, 5); } - public void testWindowFireMode1(boolean isLocalOnly,int windowSize) throws InterruptedException { - AtomicInteger sum = new AtomicInteger(0) ; - createSourceDataStream() + + public void testWindowFireMode1(boolean isLocalOnly, int windowSize) throws InterruptedException { + AtomicInteger sum = new AtomicInteger(0); + createSourceDataStream() //.map(new MapFunction() { // AtomicInteger COUNT=new AtomicInteger(0); // Long time; @@ -113,7 +117,7 @@ public void testWindowFireMode1(boolean isLocalOnly,int windowSize) throws Inter .window(TumblingWindow.of(Time.seconds(windowSize))) .setTimeField("logTime") .fireMode(1) - .setMaxMsgGap(isLocalOnly?20L:20L) + .setMaxMsgGap(isLocalOnly ? 20L : 20L) .waterMark(100000000) .groupBy("ProjectName", "LogStore") .setLocalStorageOnly(isLocalOnly) @@ -123,37 +127,37 @@ public void testWindowFireMode1(boolean isLocalOnly,int windowSize) throws Inter .toDataSteam() .forEach(new ForEachFunction() { - @Override public synchronized void foreach(JSONObject o) { int total = o.getInteger("total"); - o.put("sum(total)", sum.addAndGet(total)); + o.put("sum(total)", sum.addAndGet(total)); } }).toPrint().start(); } - public void testWindowFireMode2(boolean isLocalOnly){ - long time=new Date().getTime(); + public void testWindowFireMode2(boolean isLocalOnly) { + long time = new Date().getTime(); System.out.println(DateUtil.getCurrentTimeString()); createSourceDataStream() .map(new MapFunction() { - int count=0; + int count = 0; + @Override public JSONObject map(String message) throws Exception { - JSONObject msg=JSONObject.parseObject(message); - long time= msg.getLong("logTime"); - Date date=new Date(time); - date.setYear(2021-1900); + JSONObject msg = JSONObject.parseObject(message); + long time = msg.getLong("logTime"); + Date date = new Date(time); + date.setYear(2021 - 1900); date.setMonth(6); date.setDate(14); - msg.put("logTime",date.getTime()+count++); + msg.put("logTime", date.getTime() + count++); return msg; } }) .window(TumblingWindow.of(Time.seconds(5))) .setTimeField("logTime") - .setMaxMsgGap(isLocalOnly?5L:20L) + .setMaxMsgGap(isLocalOnly ? 5L : 20L) .fireMode(1) .waterMark(100000000) .groupBy("ProjectName", "LogStore") @@ -163,11 +167,12 @@ public JSONObject map(String message) throws Exception { .sum("InFlow", "InFlow") .toDataSteam() .map(new MapFunction() { - long time=new Date().getTime(); + long time = new Date().getTime(); + @Override public JSONObject map(JSONObject message) throws Exception { - message.put("name","chris"); - message.put("time",time++); + message.put("name", "chris"); + message.put("time", time++); return message; } }) @@ -176,28 +181,28 @@ public JSONObject map(JSONObject message) throws Exception { .setMaxMsgGap(80L) .groupBy("name") .setTimeField("time") - .sum("total","sum_total") + .sum("total", "sum_total") .setLocalStorageOnly(true) .toDataSteam() .forEach(new ForEachFunction() { - AtomicInteger sum = new AtomicInteger(0) ; - Map map=new HashMap<>(); + AtomicInteger sum = new AtomicInteger(0); + Map map = new HashMap<>(); + @Override public synchronized void foreach(JSONObject o) { - String windowInstanceId=o.getString("windowInstanceId"); - Integer oldValue=map.get(windowInstanceId); + String windowInstanceId = o.getString("windowInstanceId"); + Integer oldValue = map.get(windowInstanceId); int total = o.getInteger("sum_total"); - if(oldValue!=null){ - total=total-oldValue; + if (oldValue != null) { + total = total - oldValue; } - int nowValue=sum.addAndGet(total); - map.put(windowInstanceId,total); - o.put("sum(total)", nowValue); + int nowValue = sum.addAndGet(total); + map.put(windowInstanceId, total); + o.put("sum(total)", nowValue); } }).toPrint().start(); } - protected abstract DataStream createSourceDataStream(); } diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/AbstractWindowTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/AbstractWindowTest.java index 5a5c346b..c4e17b1f 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/AbstractWindowTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/AbstractWindowTest.java @@ -98,9 +98,9 @@ public synchronized void foreach(JSONObject o) { /** * execute window data stream , need set parameters before execute * - * @param isLocalOnly true: hign performace model ,not support exactly once + * @param isLocalOnly true: hign performace types ,not support exactly once * @param windowSize Tumbling window size - * @param fireMode fire mode ,support three model + * @param fireMode fire mode ,support three types * @param waterMark * @param maxMsgGap * @throws InterruptedException diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/MultiSplitTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/MultiSplitTest.java index a628f282..6efb2413 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/MultiSplitTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/MultiSplitTest.java @@ -18,10 +18,10 @@ import com.alibaba.fastjson.JSONObject; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.transform.DataStream; import org.apache.rocketmq.streams.common.functions.MapFunction; -import org.apache.rocketmq.streams.common.topology.model.IWindow; +import org.apache.rocketmq.streams.common.topology.IWindow; import org.junit.Test; public class MultiSplitTest extends SingleSplitTest { @@ -31,7 +31,7 @@ public class MultiSplitTest extends SingleSplitTest { protected DataStream createSourceDataStream() { - return StreamBuilder.dataStream("namespace", "name1") + return StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "name1") .fromRocketmq(topic, "window_test", true, null); } @@ -81,7 +81,7 @@ public JSONObject map(JSONObject message) throws Exception { @Test public void testInsertWindowMsg() { - StreamBuilder.dataStream("namespace", "name1") + StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "name1") .fromFile(filePath, true).toRocketmq(topic, group, nameServerAddress).start(); } diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/ShuffleOverWindowTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/ShuffleOverWindowTest.java index 8794a909..b8f6860f 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/ShuffleOverWindowTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/ShuffleOverWindowTest.java @@ -19,7 +19,7 @@ import com.alibaba.fastjson.JSONObject; import java.util.ArrayList; import java.util.List; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.common.utils.FileUtil; import org.junit.Test; @@ -28,7 +28,7 @@ public class ShuffleOverWindowTest { @Test public void testShuffleWindow() { - StreamBuilder.dataStream("namespace", "name1") + StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "name1") .fromFile(filePath, true) .topN("rowNum", 10000, "city") .addOrderByFieldName("name", true) diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/SimpleWindowTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/SimpleWindowTest.java new file mode 100644 index 00000000..fc5643f4 --- /dev/null +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/SimpleWindowTest.java @@ -0,0 +1,54 @@ +/* + * 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.client.windows; + +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; +import org.apache.rocketmq.streams.client.transform.window.Time; +import org.apache.rocketmq.streams.client.transform.window.TumblingWindow; +import org.apache.rocketmq.streams.common.context.AbstractContext; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.functions.ForEachMessageFunction; +import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.junit.Test; + +public class SimpleWindowTest { + @Test + public void testCountWindow() { + int msgCount = 88121; + StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "name") + .fromFile("window_msg_" + msgCount + ".txt", true) + .window(TumblingWindow.of(Time.seconds(5))) + .groupBy("ProjectName") + .count("c") + .toDataSteam() + .forEachMessage(new ForEachMessageFunction() { + AtomicInteger count = new AtomicInteger(0); + + @Override public void foreach(IMessage message, AbstractContext context) { + int c = count.incrementAndGet(); + message.getMessageBody().put("total", c); + message.getMessageBody().put("current_time", DateUtil.getCurrentTimeString()); + if (c > (msgCount - 10)) { + System.out.println(message.getMessageBody()); + } + + } + }) + .start(); + } +} diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/SingleSplitTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/SingleSplitTest.java index 88909b6d..1953c0d6 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/SingleSplitTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/SingleSplitTest.java @@ -16,23 +16,22 @@ */ package org.apache.rocketmq.streams.client.windows; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.transform.DataStream; -import org.apache.rocketmq.streams.common.component.ComponentCreator; import org.apache.rocketmq.streams.common.functions.MapFunction; -import org.apache.rocketmq.streams.common.topology.model.IWindow; +import org.apache.rocketmq.streams.common.topology.IWindow; import org.junit.Test; public class SingleSplitTest extends AbstractWindowTest { protected DataStream createSourceDataStream() { - return StreamBuilder.dataStream("namespace", "name1") + return StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "name1") .fromFile(filePath, true); } @Test public void testFile() { - StreamBuilder.dataStream("namespace", "name1") + StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "name1") .fromFile(filePath, false) .map(new MapFunction() { @@ -45,7 +44,7 @@ public void testFile() { } protected int getSourceCount() { - return 88121; + return 10; } /** @@ -62,7 +61,7 @@ public void testWindowResult() { @Test public void testFireMode0() throws InterruptedException { // ComponentCreator.getProperties().setProperty("window.fire.isTest","true"); - ComponentCreator.getProperties().setProperty("dipper.configurable.polling.time", "-1"); +// ComponentCreator.getProperties().setProperty("dipper.configurable.polling.time", "-1"); super.executeWindowStream(true, 5, IWindow.DEFAULTFIRE_MODE, 0, 200l); } diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/WindowFireResultCompareTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/WindowFireResultCompareTest.java index 4457f04a..a63706cd 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/WindowFireResultCompareTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/WindowFireResultCompareTest.java @@ -32,50 +32,49 @@ public class WindowFireResultCompareTest { @Test - public void testCompareResult(){ - List first= FileUtil.loadFileLine("/tmp/rockstmq-streams/window_calculate/msg.txt"); - List second= FileUtil.loadFileLine("/tmp/rockstmq-streams-1/window_calculate/msg.txt"); - Map instanceCounts=new HashMap<>(); - int firstTotal=0; - Map firstMsgs=new HashMap<>(); - for(int i=0;i first = FileUtil.loadFileLine("/tmp/rockstmq-streams/window_calculate/msg.txt"); + List second = FileUtil.loadFileLine("/tmp/rockstmq-streams-1/window_calculate/msg.txt"); + Map instanceCounts = new HashMap<>(); + int firstTotal = 0; + Map firstMsgs = new HashMap<>(); + for (int i = 0; i < first.size(); i++) { JSONObject jsonObject = JSONObject.parseObject(first.get(i)); String msgKey = jsonObject.getString("msgKey"); - firstMsgs.put(msgKey,jsonObject); + firstMsgs.put(msgKey, jsonObject); } - List firstMsgList=new ArrayList<>(firstMsgs.values()); - for(int i=0;i firstMsgList = new ArrayList<>(firstMsgs.values()); + for (int i = 0; i < firstMsgList.size(); i++) { + JSONObject firstMsg = firstMsgList.get(i); String windowInstanceId = firstMsg.getString("windowInstanceId"); String project = firstMsg.getString("ProjectName"); String logstore = firstMsg.getString("LogStore"); - Long currentOffset= firstMsg.getLong("offset"); - Integer count=instanceCounts.get(windowInstanceId); - if(count==null){ - count=0; + Long currentOffset = firstMsg.getLong("offset"); + Integer count = instanceCounts.get(windowInstanceId); + if (count == null) { + count = 0; } count++; - instanceCounts.put(windowInstanceId,count); - + instanceCounts.put(windowInstanceId, count); } - for(Integer count:instanceCounts.values()){ - firstTotal+=count; + for (Integer count : instanceCounts.values()) { + firstTotal += count; } - Map secondMsgs=new HashMap<>(); - for(int i=0;i secondMsgs = new HashMap<>(); + for (int i = 0; i < second.size(); i++) { JSONObject jsonObject = JSONObject.parseObject(second.get(i)); String msgKey = jsonObject.getString("msgKey"); - secondMsgs.put(msgKey,jsonObject); + secondMsgs.put(msgKey, jsonObject); } - Map secondInstanceCounts=new HashMap<>(); - List secondMsgList=new ArrayList<>(secondMsgs.values()); - for(int i=0;i secondInstanceCounts = new HashMap<>(); + List secondMsgList = new ArrayList<>(secondMsgs.values()); + for (int i = 0; i < secondMsgList.size(); i++) { JSONObject msg = secondMsgList.get(i); String windowInstanceId = msg.getString("windowInstanceId"); @@ -89,88 +88,86 @@ public void testCompareResult(){ count++; secondInstanceCounts.put(windowInstanceId, count); - } - int secondTotal=0; - for(Integer count:secondInstanceCounts.values()){ - secondTotal+=count; + int secondTotal = 0; + for (Integer count : secondInstanceCounts.values()) { + secondTotal += count; } - System.out.println(instanceCounts.size()==secondInstanceCounts.size()); - System.out.println(firstTotal+" "+secondTotal); - for(String windowInstanceId:instanceCounts.keySet()){ - Integer firstCount=instanceCounts.get(windowInstanceId); - Integer secondCount=secondInstanceCounts.get(windowInstanceId); - if(firstCount.longValue()!=secondCount.longValue()){ - System.out.println(windowInstanceId+" "+firstCount+" "+secondCount); + System.out.println(instanceCounts.size() == secondInstanceCounts.size()); + System.out.println(firstTotal + " " + secondTotal); + for (String windowInstanceId : instanceCounts.keySet()) { + Integer firstCount = instanceCounts.get(windowInstanceId); + Integer secondCount = secondInstanceCounts.get(windowInstanceId); + if (firstCount.longValue() != secondCount.longValue()) { + System.out.println(windowInstanceId + " " + firstCount + " " + secondCount); } } } /** - * msg.put("offset",message.getHeader().getOffset()); - * msg.put("queueid",message.getMessageBody().getString(message.getHeader().getQueueId())); - * msg.put("windowInstaceId",instance.createWindowInstanceId()); + * msg.put("offset",message.getHeader().getOffset()); + * msg.put("queueid",message.getMessageBody().getString(message.getHeader().getQueueId())); + * msg.put("windowInstaceId",instance.createWindowInstanceId()); */ @Test - public void testResult(){ - String filePath="/tmp/rocketmq-streams/result.txt"; - List msgs=FileUtil.loadFileLine(filePath); - Map windowIntanceId2Count=new HashMap<>(); - for(String line:msgs){ - JSONObject msg=JSONObject.parseObject(line); + public void testResult() { + String filePath = "/tmp/rocketmq-streams/result.txt"; + List msgs = FileUtil.loadFileLine(filePath); + Map windowIntanceId2Count = new HashMap<>(); + for (String line : msgs) { + JSONObject msg = JSONObject.parseObject(line); String windowInstanceId = msg.getString("windowInstanceId"); String project = msg.getString("ProjectName"); String logstore = msg.getString("LogStore"); - Long currentOffset= msg.getLong("offset"); - if(StringUtil.isEmpty(project)){ - project=""; + Long currentOffset = msg.getLong("offset"); + if (StringUtil.isEmpty(project)) { + project = ""; } - if(StringUtil.isEmpty(logstore)){ - logstore=""; + if (StringUtil.isEmpty(logstore)) { + logstore = ""; } - String key = MapKeyUtil.createKey(windowInstanceId,project,logstore,currentOffset+""); - Integer count= windowIntanceId2Count.get(key); - if(count==null){ - count=0; + String key = MapKeyUtil.createKey(windowInstanceId, project, logstore, currentOffset + ""); + Integer count = windowIntanceId2Count.get(key); + if (count == null) { + count = 0; } count++; - windowIntanceId2Count.put(key,count); + windowIntanceId2Count.put(key, count); } - - filePath="/tmp/rocketmq-streams/result.txt.1"; - msgs=FileUtil.loadFileLine(filePath); - Map windowIntanceId2Count_2=new HashMap<>(); - for(String line:msgs){ - JSONObject msg=JSONObject.parseObject(line); + filePath = "/tmp/rocketmq-streams/result.txt.1"; + msgs = FileUtil.loadFileLine(filePath); + Map windowIntanceId2Count_2 = new HashMap<>(); + for (String line : msgs) { + JSONObject msg = JSONObject.parseObject(line); String windowInstanceId = msg.getString("windowInstanceId"); String project = msg.getString("ProjectName"); String logstore = msg.getString("LogStore"); - Long currentOffset= msg.getLong("offset"); - if(StringUtil.isEmpty(project)){ - project=""; + Long currentOffset = msg.getLong("offset"); + if (StringUtil.isEmpty(project)) { + project = ""; } - if(StringUtil.isEmpty(logstore)){ - logstore=""; + if (StringUtil.isEmpty(logstore)) { + logstore = ""; } - String key = MapKeyUtil.createKey(windowInstanceId,project,logstore,currentOffset+""); - Integer count= windowIntanceId2Count_2.get(key); - if(count==null){ - count=0; + String key = MapKeyUtil.createKey(windowInstanceId, project, logstore, currentOffset + ""); + Integer count = windowIntanceId2Count_2.get(key); + if (count == null) { + count = 0; } count++; - windowIntanceId2Count_2.put(key,count); + windowIntanceId2Count_2.put(key, count); } - System.out.println(windowIntanceId2Count.size()+" "+windowIntanceId2Count_2.size()); + System.out.println(windowIntanceId2Count.size() + " " + windowIntanceId2Count_2.size()); //assertTrue(windowIntanceId2Count.size()==windowIntanceId2Count_2.size()); System.out.println(windowIntanceId2Count_2.get("zPeZp6w6VHqzlKpAn1nHag==;e9+26U3f8cCAoSQnrj6jPg==;1Pd1bWZSiHpOd9JU383tuw==;49550465100000055")); - for(String key:windowIntanceId2Count.keySet()){ - Integer count=windowIntanceId2Count.get(key); - Integer count2=windowIntanceId2Count_2.get(key); - if(count.intValue()!=count2.intValue()){ + for (String key : windowIntanceId2Count.keySet()) { + Integer count = windowIntanceId2Count.get(key); + Integer count2 = windowIntanceId2Count_2.get(key); + if (count.intValue() != count2.intValue()) { System.out.println("result is not match"); assertTrue(false); } diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/WindowMsgManager.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/WindowMsgManager.java index a510c016..69695a22 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/WindowMsgManager.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/WindowMsgManager.java @@ -30,44 +30,43 @@ public class WindowMsgManager { protected String fileName; - protected Long initEventTime=1627358460609L; + protected Long initEventTime = 1627358460609L; @Test - public void testCreateMsgFiles(){ - String filePath=SingleSplitTest.class.getClassLoader().getResource(fileName).getFile(); - createFile(filePath,""); + public void testCreateMsgFiles() { + String filePath = SingleSplitTest.class.getClassLoader().getResource(fileName).getFile(); + createFile(filePath, ""); } - - protected void createFile(String filePath,String outFileName){ - File file=new File(filePath); - File dir=file.getParentFile(); - String outPath=FileUtil.concatFilePath(dir.getAbsolutePath(),outFileName); - Long time=null; - List lines= FileUtil.loadFileLine(filePath); - List msgs=new ArrayList<>(); - for(String line:lines){ - JSONObject jsonObject=JSONObject.parseObject(line); - JSONObject msg=new JSONObject(); - msg.put("ProjectName",jsonObject.getString("ProjectName")); - msg.put("LogStore",jsonObject.getString("LogStore")); - msg.put("OutFlow",jsonObject.getString("OutFlow")); - msg.put("InFlow",jsonObject.getString("InFlow")); - if(time==null){ - time=initEventTime; - }else { - time=time+1; + protected void createFile(String filePath, String outFileName) { + File file = new File(filePath); + File dir = file.getParentFile(); + String outPath = FileUtil.concatFilePath(dir.getAbsolutePath(), outFileName); + Long time = null; + List lines = FileUtil.loadFileLine(filePath); + List msgs = new ArrayList<>(); + for (String line : lines) { + JSONObject jsonObject = JSONObject.parseObject(line); + JSONObject msg = new JSONObject(); + msg.put("ProjectName", jsonObject.getString("ProjectName")); + msg.put("LogStore", jsonObject.getString("LogStore")); + msg.put("OutFlow", jsonObject.getString("OutFlow")); + msg.put("InFlow", jsonObject.getString("InFlow")); + if (time == null) { + time = initEventTime; + } else { + time = time + 1; } - msg.put("logTime",time); + msg.put("logTime", time); msg.put("currentTime", DateUtil.format(new Date(time))); - AbstractWindow window=new WindowOperator(); + AbstractWindow window = new WindowOperator(); window.setSizeInterval(5); window.setTimeUnitAdjust(1); window.setTimeFieldName("logTime"); msgs.add(msg.toJSONString()); } - FileUtil.write("outPath",msgs,false); + FileUtil.write("outPath", msgs, false); } } diff --git a/rocketmq-streams-clients/src/test/resources/log4j.xml b/rocketmq-streams-clients/src/test/resources/log4j.xml old mode 100755 new mode 100644 diff --git a/rocketmq-streams-commons/pom.xml b/rocketmq-streams-commons/pom.xml index 72af956e..a942dcf3 100755 --- a/rocketmq-streams-commons/pom.xml +++ b/rocketmq-streams-commons/pom.xml @@ -15,70 +15,78 @@ See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 org.apache.rocketmq rocketmq-streams - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT rocketmq-streams-commons ROCKETMQ STREAMS :: commons jar - - org.slf4j - slf4j-log4j12 - ${slf4j-log4j12.version} - - + org.apache.commons commons-lang3 - 3.11 - - - net.agkn - hll - 1.6.0 + commons-codec + commons-codec - - - junit - junit + commons-io + commons-io + + com.alibaba fastjson + + + - org.apache.httpcomponents - httpclient + com.google.re2j + re2j - commons-io - commons-io + com.google.guava + guava + + + com.google.auto.service + auto-service com.google.code.gson gson - com.google.auto.service - auto-service + com.univocity + univocity-parsers + + + - com.google.guava - guava + com.esotericsoftware + kryo + + + org.apache.httpcomponents + httpclient + + + de.ruedigermoeller + fst - com.lmax disruptor @@ -86,51 +94,50 @@ com.gliwka.hyperscan hyperscan - 5.4.0-2.0.0 net.java.dev.jna platform - 3.5.2 - - com.google.re2j - re2j - 1.6 + net.agkn + hll + + + + junit + junit + + + + org.apache.rocketmq + rocketmq-streams-local-cache + + + + org.slf4j + slf4j-log4j12 + org.apache.rocketmq rocketmq-client - - - ch.qos.logback - logback-classic - - - ch.qos.logback - logback-core - - - de.ruedigermoeller - fst + org.apache.rocketmq + rocketmq-acl - com.esotericsoftware - kryo - 5.3.0 + org.apache.rocketmq + rocketmq-tools - - commons-codec - commons-codec - 1.13 + org.springframework.boot + spring-boot-loader diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/batchsystem/BatchFinishMessage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/batchsystem/BatchFinishMessage.java index 041b0e01..a0da1473 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/batchsystem/BatchFinishMessage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/batchsystem/BatchFinishMessage.java @@ -20,18 +20,36 @@ import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.Message; import org.apache.rocketmq.streams.common.interfaces.ISystemMessage; +import org.apache.rocketmq.streams.common.topology.model.MessageFinishCallBack; +import org.apache.rocketmq.streams.common.utils.StringUtil; +/** + * 系统消息,当确定后续数据了,可以发这个消息,这个消息会告诉所有组件,完成flush和计算,会在整个拓扑传递 + * 主要在于测试场景,测试一小批次数据,或批处理场景 + */ public class BatchFinishMessage implements ISystemMessage { public static String PRIMARY_KEY = "__Primary_KEY"; + public static String CALLBACK = "__callback"; + public static String MESSAGE_ID = "__message_id"; protected IMessage message; public BatchFinishMessage(IMessage message) { this.message = message; + + } + + public static IMessage create(MessageFinishCallBack messageFinishCallBack) { + JSONObject msg = new JSONObject(); + msg.put(PRIMARY_KEY, true); + msg.put(CALLBACK, messageFinishCallBack); + msg.put(MESSAGE_ID, StringUtil.getUUID()); + return new Message(msg); } public static IMessage create() { JSONObject msg = new JSONObject(); msg.put(PRIMARY_KEY, true); + msg.put(MESSAGE_ID, StringUtil.getUUID()); return new Message(msg); } @@ -43,7 +61,19 @@ public static boolean isBatchFinishMessage(IMessage message) { } } + public MessageFinishCallBack getMessageFinishCallBack(IMessage msg) { + return (MessageFinishCallBack) msg.getMessageBody().get(CALLBACK); + } + + public String getMessageId() { + return this.message.getMessageBody().getString(MESSAGE_ID); + } + public IMessage getMsg() { return message; } + + @Override public int getSystemMessageType() { + return ISystemMessage.BATCH_FINISH; + } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/FileBasedTable.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/FileBasedTable.java index da603f18..923dadc5 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/FileBasedTable.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/FileBasedTable.java @@ -33,16 +33,14 @@ */ public abstract class FileBasedTable extends AbstractMemoryTable { - static transient String mappedFilePrefix = "/tmp/dipper_"; protected static final ByteOrder order = ByteOrder.nativeOrder(); - + static transient String mappedFilePrefix = "/tmp/dipper_"; + protected transient long fileOffset = 0; // unit -> byte + protected transient int fileRowCount = 0; //总的行数 String fileName; int columnsCount = -1; TableSchema tableSchema; - protected transient long fileOffset = 0; // unit -> byte - protected transient int fileRowCount = 0; //总的行数 - protected FileBasedTable() { } @@ -60,22 +58,34 @@ public FileBasedTable(String fileName, int columnsCount, TableSchema schema) { String fieldName = tableSchema.getField(i); String fieldType = tableSchema.getFieldType(i); DataType dataType = DataTypeUtil.getDataType(fieldType); - cloumnName2Index.put(fieldName, i); + columnName2Index.put(fieldName, i); index2ColumnName.put(i, fieldName); - cloumnName2DatatType.put(fieldName, dataType); + columnName2DataType.put(fieldName, dataType); } } } + public static String createRealFilePath(String fileName, String cyclePeriod, String index) { + return mappedFilePrefix + fileName + "_" + cyclePeriod + "_" + index; + } + + public static String createLockFilePath(String fileName, String cyclePeriod) { + return mappedFilePrefix + fileName + "_" + cyclePeriod + "_lock"; + } + + public static String createDoneFilePath(String fileName, String cyclePeriod) { + return mappedFilePrefix + fileName + "_" + cyclePeriod + "_done"; + } + // row id @Override public Iterator newIterator() { return new Iterator() { - protected long nextCursor = 0; private final long totalByteCount = fileOffset; + protected long nextCursor = 0; @Override public boolean hasNext() { @@ -294,14 +304,6 @@ private byte[] loadRowLengthByte(long offset) { return loadFromFile(offset, 2); } - abstract byte[] loadFromFile(long offset, int len); - - abstract boolean destroy(); - - int rowLenByte2Int(byte[] bytes) { - return NumberUtils.toInt(bytes); - } - // int lengthByte2Int(byte[] bytes){ // int bytesLength = bytes.length; // if(bytesLength == 1){ @@ -312,6 +314,14 @@ int rowLenByte2Int(byte[] bytes) { // } // } + abstract byte[] loadFromFile(long offset, int len); + + abstract boolean destroy(); + + int rowLenByte2Int(byte[] bytes) { + return NumberUtils.toInt(bytes); + } + private byte[] readColumnLen(byte[] bytes, int index) { byte b = bytes[index]; //第一位为1 @@ -390,16 +400,4 @@ public TableSchema getTableSchema() { public void setTableSchema(TableSchema tableSchema) { this.tableSchema = tableSchema; } - - public static String createRealFilePath(String fileName, String cyclePeriod, String index) { - return mappedFilePrefix + fileName + "_" + cyclePeriod + "_" + index; - } - - public static String createLockFilePath(String fileName, String cyclePeriod) { - return mappedFilePrefix + fileName + "_" + cyclePeriod + "_lock"; - } - - public static String createDoneFilePath(String fileName, String cyclePeriod) { - return mappedFilePrefix + fileName + "_" + cyclePeriod + "_done"; - } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/HybridByteBufferTable.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/HybridByteBufferTable.java index 0e642185..628bfc06 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/HybridByteBufferTable.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/HybridByteBufferTable.java @@ -30,63 +30,44 @@ import java.util.Map; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.cache.compress.AbstractMemoryTable; +import org.apache.rocketmq.streams.common.threadpool.ThreadPoolFactory; import org.apache.rocketmq.streams.common.utils.NumberUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class HybridByteBufferTable extends AbstractMemoryTable { - private static final Log logger = LogFactory.getLog(HybridByteBufferTable.class); - - public static int PAGE_SIZE = 1 << 12; //4k - protected static final ByteOrder order = ByteOrder.nativeOrder(); - - static String jobName; - static final String mappedFilePrefix = "/tmp/dipper_"; static final int MAX_FILE_SIZE = 1 << 35; //32 * 1024 * 1024 * 1024 = 32G; //文件的最大长度 static final long MAX_CACHE_SIZE = 1L << 35; // 32 * 1024 * 1024 * 1024 = 32G - - private transient long fileSizeLimit = -1; - - private transient long cacheSizeLimit = -1; - - //超过cache的部分,需要用内存映射方式读取 - private transient FileChannel channel; - - //row length固定位2个byte - private transient ByteBuffer rowLengthBuffer = ByteBuffer.allocateDirect(2); - - private transient ByteBuffer buffer256 = ByteBuffer.allocateDirect(256); - + private static final Logger logger = LoggerFactory.getLogger(HybridByteBufferTable.class); + public static int PAGE_SIZE = 1 << 12; //4k + static String jobName; + public ExecutorService executor; //cache的数据, cache会顺序写入 protected transient List cacheRows = new ArrayList<>(); - - //当前cache的行标 - private transient int curCacheRowIndex = 0; - //cache的字节数, 需要小于 cacheSize; protected transient long curCacheByteSize = 0; - //总的字节数 protected transient long curTotalByteSize = 0; - //总的行数 protected transient int curTotalRowCount; - protected transient MemoryPageCache pageCache; - protected transient ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue(); - - public ExecutorService executor = Executors.newFixedThreadPool(1); - //总的列的数量 protected int columnCount; - protected transient volatile boolean isFinishWrite = false; + private transient long fileSizeLimit = -1; + private transient long cacheSizeLimit = -1; + //超过cache的部分,需要用内存映射方式读取 + private transient FileChannel channel; + //row length固定位2个byte + private transient ByteBuffer rowLengthBuffer = ByteBuffer.allocateDirect(2); + private transient ByteBuffer buffer256 = ByteBuffer.allocateDirect(256); + //当前cache的行标 + private transient int curCacheRowIndex = 0; public HybridByteBufferTable(String jobName) { this.jobName = jobName; @@ -96,6 +77,7 @@ public HybridByteBufferTable(String jobName) { } catch (IOException e) { e.printStackTrace(); } + executor = ThreadPoolFactory.createFixedThreadPool(1, HybridByteBufferTable.class.getName() + "-" + jobName); executor.submit(new WriteFileTask()); } @@ -124,23 +106,20 @@ private final FileChannel createChannel() throws IOException { return fc; } - @Override - public Iterator newIterator() { + @Override public Iterator newIterator() { return new Iterator() { - protected long nextCursor = 0; private final long totalByteCount = curTotalByteSize; + protected long nextCursor = 0; // private long count = 0; - @Override - public boolean hasNext() { + @Override public boolean hasNext() { // check(getByteTotalSize()); return nextCursor < totalByteCount; } - @Override - public RowElement next() { + @Override public RowElement next() { // check(getByteTotalSize()); long current = nextCursor; Map row = new HashMap<>(); @@ -165,8 +144,7 @@ public Long saveRowByte(byte[][] values) { return saveRowByte(values, -1); } - @Override - protected Long saveRowByte(byte[][] values, int byteSize) { + @Override protected Long saveRowByte(byte[][] values, int byteSize) { int size = 0; byte[][] byteList = new byte[values.length + 1][]; int i = 0; @@ -212,8 +190,7 @@ protected Long saveRowByte(byte[][] values, int byteSize) { * @param index * @return */ - @Override - protected byte[][] loadRowByte(Long index) { + @Override protected byte[][] loadRowByte(Long index) { // boolean isCached = ((index & 0x8000000000000000L) == 0x8000000000000000L); // Long realIndex = index & 0x7fffffffffffffffL; //去掉第一位 @@ -604,8 +581,7 @@ public MemoryPageCache deepCopy() { public class WriteFileTask implements Runnable { - @Override - public void run() { + @Override public void run() { MemoryPageCache cache = null; double count = 0.0; while (true) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/MappedByteBufferTable.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/MappedByteBufferTable.java index c7166571..7413a48b 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/MappedByteBufferTable.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/MappedByteBufferTable.java @@ -33,16 +33,15 @@ import java.util.Map; import java.util.Properties; import org.apache.commons.io.FileUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.datatype.DataType; import org.apache.rocketmq.streams.common.utils.DataTypeUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class MappedByteBufferTable extends FileBasedTable { - private static final Log logger = LogFactory.getLog(MappedByteBufferTable.class); - protected static final int DEFAULT_SIZE = 1024 * 1024 * 1024; + private static final Logger logger = LoggerFactory.getLogger(MappedByteBufferTable.class); protected transient List caches = new ArrayList<>(); protected transient List channels = new ArrayList<>(); protected transient List files = new ArrayList<>(); @@ -325,6 +324,12 @@ public void setFiles(List files) { this.files = files; } + public interface DateLoader { + + void load(MappedByteBufferTable table); + + } + public static class FileMeta implements Serializable { private static final long serialVersionUID = -248063270830962898L; @@ -459,12 +464,6 @@ public void setGlobalCursor(long globalCursor) { } } - public interface DateLoader { - - void load(MappedByteBufferTable table); - - } - public static class Creator { Properties properties = new Properties(); @@ -569,9 +568,9 @@ private void loadFromFile(MappedByteBufferTable table) throws IOException { table.setFileOffset(meta.totalByteSize); table.setColumnsCount(meta.columnsCount); table.setFileRowCount(meta.totalRowCount); - table.setCloumnName2Index(meta.columnName2Index); + table.setColumnName2Index(meta.columnName2Index); table.setIndex2ColumnName(meta.index2ColumnName); - table.setCloumnName2DatatType(meta.columns2DataType); + table.setColumnName2DatatType(meta.columns2DataType); for (int i = 0; i < meta.filePaths.length; i++) { String path = meta.filePaths[i]; int limit = meta.limits[i]; @@ -583,15 +582,18 @@ private void release() { } - private String getCycleStr(Date date, int pollingTimeMinute) { + private String getCycleStr(Date date, int pollingTimeSecond) { String cycleStr = ""; - if (pollingTimeMinute >= 1 && pollingTimeMinute < 60) { + if (pollingTimeSecond >= 1 && pollingTimeSecond < 60) { + SimpleDateFormat format = new SimpleDateFormat("yyyyMMddHHmmss"); + cycleStr = format.format(date); + } else if (pollingTimeSecond >= 60 && pollingTimeSecond < (60 * 60)) { SimpleDateFormat format = new SimpleDateFormat("yyyyMMddHHmm"); cycleStr = format.format(date); - } else if (pollingTimeMinute >= 60 && pollingTimeMinute < (60 * 24)) { + } else if (pollingTimeSecond >= 60 * 60 && pollingTimeSecond < (60 * 60 * 24)) { SimpleDateFormat format = new SimpleDateFormat("yyyyMMddHH"); cycleStr = format.format(date); - } else if (pollingTimeMinute >= (60 * 24)) { + } else if (pollingTimeSecond >= (60 * 60 * 24)) { SimpleDateFormat format = new SimpleDateFormat("yyyyMMdd"); cycleStr = format.format(date); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/MappedByteBufferTableWithPrimaryIndex.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/MappedByteBufferTableWithPrimaryIndex.java deleted file mode 100644 index 72f2f892..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/MappedByteBufferTableWithPrimaryIndex.java +++ /dev/null @@ -1,482 +0,0 @@ -///* -// * 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 java.io.File; -//import java.io.IOException; -//import java.io.RandomAccessFile; -//import java.io.Serializable; -//import java.nio.ByteOrder; -//import java.nio.MappedByteBuffer; -//import java.nio.channels.FileChannel; -//import java.util.*; -//import java.util.concurrent.atomic.AtomicInteger; -// -//import org.apache.commons.logging.Log; -//import org.apache.commons.logging.LogFactory; -//import org.apache.rocketmq.streams.common.cache.compress.AbstractMemoryTable; -//import org.apache.rocketmq.streams.common.utils.NumberUtils; -// -//@Deprecated -//public class MappedByteBufferTableWithPrimaryIndex extends AbstractMemoryTable { -// -// private static final Log logger = LogFactory.getLog(MappedByteBufferTableWithPrimaryIndex.class); -// -// protected String jobName; -// -// protected static final String mappedFilePrefix = "/tmp/dipper_"; -// -// protected static final int SIZE = 1024 * 1024 * 1024; -// -// protected int fileSize = -1; -// -// //1g 1个文件,超过1g,可以多个文件,每个文件对应一个MappedByteBuffer -// protected transient List caches = new ArrayList<>(); -// -// //当一个文件写满时,把文件的索引号和起始位置map起来,方便查找 -// //key = rowId, value为全局索引号, 把二维索引变成一维索引 -// protected transient Map maxOffsets = new HashMap<>(); -// -// //row id -// protected transient AtomicInteger currentIndex = new AtomicInteger(0); -// -// //全局游标 -// protected transient volatile long cursor = 0; -// -// protected transient volatile int currentFileIndex = 0; -// -// private static final int MAX_FILE_COUNT = 2; -// -// protected long totalMargin = 0; -// -// protected static final ByteOrder order = ByteOrder.nativeOrder(); -// -// public MappedByteBufferTableWithPrimaryIndex(){ -// -// } -// -// public MappedByteBufferTableWithPrimaryIndex(String jobName){ -// this.jobName = jobName; -// this.fileSize = SIZE; -// } -// -// public MappedByteBufferTableWithPrimaryIndex(String jobName, int fileSize){ -// this.jobName = jobName; -// this.fileSize = fileSize; -// } -// -// -// //todo file lock -// private final String createMappedFile(int fileIndex) throws IOException { -// String path = mappedFilePrefix + jobName + "_" + fileIndex; -// File file = new File(path); -// boolean isSuccess = file.exists(); -// if(isSuccess){ -// return path; -// }else{ -// isSuccess = file.createNewFile(); -// } -// if(isSuccess){ -// return path; -// }else{ -// logger.error(String.format("create mapped file error, file path is %s", path)); -// return null; -// } -// } -// -// private final MappedByteBuffer createMappedByteBuffer(int fileIndex) throws IOException { -// String filePath = createMappedFile(fileIndex); -// RandomAccessFile raf = new RandomAccessFile(filePath, "rw"); -// FileChannel fc = raf.getChannel(); -// MappedByteBuffer mbb = fc.map(FileChannel.MapMode.READ_WRITE, 0, getFileSizeOrDefault()); -// return mbb; -// } -// -// private final MappedByteBuffer getOrCreateMappedByteBuffer(int index) throws IOException { -// MappedByteBuffer buffer = null; -// if(index < caches.size()){ -// buffer = caches.get(index); -// } -// if(buffer == null){ -// buffer = createMappedByteBuffer(index); -// caches.add(buffer); -// } -// return buffer; -// } -// -// private final int calCurrentFileIndex(long cursor){ -// return Long.valueOf((cursor/getFileSizeOrDefault())).intValue(); -// } -// -// private final int calCurrentFileIndex(){ -// return calCurrentFileIndex(cursor); -// } -// -// private final FilePosition seek(long startIndex){ -// -// int fileIndex = Long.valueOf(startIndex / getFileSizeOrDefault()).intValue(); -// long bufferPosition = startIndex % getFileSizeOrDefault(); -// return new FilePosition(fileIndex, bufferPosition); -// -// } -// -//// 1. 在 tmp目录创建文件,一个文件1g,最大4g(2g?) -//// 2. 一台物理机/虚拟机可能运行多个taskmanager,一个taskmanager可能会有多个并发,文件目录 为 tmp/${jobName}/00 ~ 03 -//// 3. 初始化加写锁? -//// 4. index是什么意思? -// -// @Override -// public Iterator newIterator() { -// return new Iterator() { -// -// protected int rowIndex = 0; -// private final int count = maxOffsets.size(); -// -// @Override -// public boolean hasNext() { -// check(maxOffsets.size()); -// return rowIndex < count; -// } -// -// @Override -// public RowElement next() { -// check(maxOffsets.size()); -// Map row = getRow(rowIndex); -// return new RowElement(row, rowIndex++); -// } -// -// private final void check(int count){ -// if(this.count != count){ -// throw new ConcurrentModificationException("unsupported modified. " + this.count + " != " + count); -// } -// } -// }; -// } -// -// /** -// * 保存row到文件,整个数据的构成:行长度(2个字节)+ 行字节数组 -// * 每行多列:每一列前面加一个长度,标识这个列代表的字节数组的长度 -// * 长度用1-2个字节表示,根据第一位是0还是1,决定是1个字节还是两个字节 -// * -// * @param values -// * @return row id -// */ -// @Override -// protected Integer saveRowByte(byte[][] values, int byteSize) { -// int size = 0; -// List byteList = new ArrayList(); -// for(byte[] bytes : values){ -// byte[] lenBytes = createLenByte(bytes); -// byte[] columnBytes = new byte[lenBytes.length + bytes.length]; -// int index = 0; -// for(byte b : lenBytes){ -// columnBytes[index] = b; -// index++; -// } -// for(byte b : bytes){ -// columnBytes[index] = b; -// index++; -// } -// byteList.add(columnBytes); -// size += columnBytes.length; -// } -//// size = size + 2; -// //计算整行的长度 -// byte[] rowLength = createLenByte(size); -// int startIndex = currentIndex.getAndIncrement(); -// //指向下一个 -// byteList.add(0, rowLength);//加上长度 -// size = size + rowLength.length; -// try { -// //写入成功之后再加size -// save2File(startIndex, byteList, size); -// } catch (IOException e) { -// e.printStackTrace(); -// } -// return startIndex; -// } -// -// /** -// * 从文件中加载行 -// * -// * @param startIndex -// * @return -// */ -// @Override -// protected byte[][] loadRowByte(Integer startIndex) { -// -// List bytes = null; -// try { -// bytes = getFromFile(startIndex); -// } catch (IOException e) { -// e.printStackTrace(); -// } -// byte[][] byteRows = new byte[bytes.size()][]; -// int i = 0; -// for(byte[] byteArray : bytes){ -// byteRows[i] = byteArray; -// i++; -// } -// return byteRows; -// -// } -// -// /** -// * startIndex 为row id -// * @param startIndex -// * @return -// * @throws IOException -// */ -// protected List getFromFile(int startIndex) throws IOException { -// -// long cursor = maxOffsets.get(startIndex); -// FilePosition position = seek(cursor); -// int fileIndex = position.fileIndex; -// long bufferCursor = position.bufferPosition; -// MappedByteBuffer buffer = caches.get(fileIndex); -// //解析整行的长度 -// byte[] rowLenBytes = readLen((int)bufferCursor, buffer); -// //计算整行的长度, 不包含行长度的元数据 -// int length = readLenFromBytes(rowLenBytes); -// //解析出整行的数据 -// byte[] bytes = readByteFromFile(buffer, bufferCursor + rowLenBytes.length, length); -// Listresult = new ArrayList<>(); -// int index = 0; -// while (index < length){ -// byte[] lenBytes = readLen(bytes, index); -// index = index + lenBytes.length; -// int len = readLenFromBytes(lenBytes); -// byte[] column = readByteFromBytes(bytes, index, len); -// result.add(column); -// index = index + column.length; -// } -// return result; -// } -// -// -// private byte[] readByteFromBytes(byte[] bytes,int index, int len){ -//// if(len == 0){ -//// return new byte[0]; -//// } -// byte[] ret = new byte[len]; -// for(int i = 0; i < len; i++){ -// ret[i] = bytes[index++]; -// } -// return ret; -// } -// -// /** -// * 从字节数组解析长度 -// * @param bytes -// * @return -// */ -// private int readLenFromBytes(byte[] bytes){ -// int bytesLength = bytes.length; -// assert (bytesLength == 1 || bytesLength == 2) : "bytes length must be 1 or 2, but " + Arrays.toString(bytes); -// if(bytesLength == 1){ -// return NumberUtils.toInt(bytes[0]); -// }else{ -// bytes[1] = (byte)(bytes[1] & 0x7f); -// return NumberUtils.toInt(bytes); -// } -// } -// -// private byte[] readLen(int start, MappedByteBuffer buffer){ -// buffer.position(0); -// byte b = buffer.get(start); -// int length = 1; -// if((b & 0x80) == 0x80){ -// length = 2; -// } -// byte[] ret = new byte[length]; -// ret[0] = b; -// if(length == 2){ -// ret[1] = buffer.get(); -// } -//// buffer.position(0); -//// System.out.println("start - " + start + ", " + "remaing - " + buffer.remaining() + ", " + "position - " + buffer.position()); -//// buffer.get(ret, start, length); -// return ret; -// } -// -// private byte[] readLen(byte[] bytes, int index) { -// byte b = bytes[index]; -// //第一位为1 -// if((b & 0x80) == 0x80){ -// byte[] ret = new byte[2]; -// ret[0] = b; -// ret[1] = bytes[++index]; -// return ret; -// }else{ -// byte[] ret = new byte[1]; -// ret[0] = b; -// return ret; -// } -// } -// -// private byte[] readByteFromFile(MappedByteBuffer buffer, long startIndex, int len) throws IOException { -// byte[] dst = new byte[len]; -// int index = Long.valueOf(startIndex).intValue(); -// buffer.position(index); -// buffer.get(dst); -// return dst; -// } -// -// /** -// * startIndex 为row id -// * @param startIndex -// * @param list -// * @param byteSize -// * @throws IOException -// */ -// private void save2File(int startIndex, List list, int byteSize) throws IOException { -// MappedByteBuffer mappedByteBuffer = getOrCreateMappedByteBufferWithCapacity(byteSize); -// if(byteSize > getFileSizeOrDefault()){ -// throw new RuntimeException("row is too long, size is " + byteSize); -// } -// for(byte[] bytes : list){ -// mappedByteBuffer.put(bytes); -// } -// maxOffsets.put(startIndex, cursor); -//// if((maxOffsets.size() % 100000) == 1){ -//// System.out.println(maxOffsets.size()); -//// } -// cursor = cursor + byteSize; -// } -// -// /** -// * 获取数组的长度, 大于等于128的用两个byte表示, 小于等于127的用一个byte表示 -// * @param bytes -// * @return -// */ -// private byte[] createLenByte(byte[] bytes) { -// //todo 如果长度小于127,用一个字节表示长度,第一位为0 -// //如果长度大于127,用两个字节表示长度,第一位为1 -// //返回标识长度的字节数组 -// return createLenByte(bytes.length); -// } -// -// private byte[] createLenByte(int size) { -// byte[] ret; -// int length = size; -// //max length 65535/2 -// if(length >= 0 && length <= 127){ -// ret = NumberUtils.toByteArray(length, 0xff); -// }else if(length >= 128 && length <= 32767){ -// ret = NumberUtils.toByteArray(length, 0xffff); -// //第一位置成1 -// ret[1] = (byte)((ret[1]) ^ ((byte)0x80)); -// }else{ -// throw new RuntimeException("length must be 1 ~ 32767, but " + size); -// } -// return ret; -// } -// -// /** -// * -// * @param fileIndex -// * @return -// * @throws IOException -// */ -// private MappedByteBuffer selectFile(int fileIndex) throws IOException { -//// Long cursor = maxOffsets.get(startIndex); -//// int fileIndex = calCurrentFileIndex(cursor.longValue()); -// MappedByteBuffer mappedByteBuffer = caches.get(fileIndex); -// return mappedByteBuffer; -// } -// -// /** -// * 检查当前的mappedByteBuffer 是否有足够的容量 -// * @param byteSize -// * @return -// * @throws IOException -// */ -// private MappedByteBuffer getOrCreateMappedByteBufferWithCapacity(int byteSize) throws IOException { -// -// int tmpFileIndex = calCurrentFileIndex(); -// MappedByteBuffer buffer = getOrCreateMappedByteBuffer(tmpFileIndex); -// int leftCapacity = getFileSizeOrDefault() - (buffer.position() + 1); -// if(leftCapacity <= byteSize){ -// tmpFileIndex += 1; -// totalMargin = totalMargin + leftCapacity; -// cursor = cursor + leftCapacity; -// buffer = getOrCreateMappedByteBuffer(tmpFileIndex); -// } -// return buffer; -// } -// -// public String getJobName() { -// return jobName; -// } -// -// public void setJobName(String jobName) { -// this.jobName = jobName; -// } -// -// public int getFileSize() { -// return fileSize; -// } -// -// public void setFileSize(int fileSize) { -// this.fileSize = fileSize; -// } -// -// private final int getFileSizeOrDefault(){ -// if(fileSize == -1){ -// fileSize = SIZE; -// } -// return fileSize; -// } -// -// public List getCaches() { -// return caches; -// } -// -// public void setCaches(List caches) { -// this.caches = caches; -// } -// -// public static class FilePosition implements Serializable{ -// -// private static final long serialVersionUID = 9128119471875329716L; -// -// public int fileIndex; -// public long bufferPosition; -// -// public FilePosition(int a, long b){ -// fileIndex = a; -// bufferPosition = b; -// } -// -// public int getFileIndex() { -// return fileIndex; -// } -// -// public void setFileIndex(int fileIndex) { -// this.fileIndex = fileIndex; -// } -// -// public long getBufferPosition() { -// return bufferPosition; -// } -// -// public void setBufferPosition(long bufferPosition) { -// this.bufferPosition = bufferPosition; -// } -// -// } -//} \ No newline at end of file diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/AbstractMemoryTable.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/AbstractMemoryTable.java index 41e21fb4..2f2e04c4 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/AbstractMemoryTable.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/AbstractMemoryTable.java @@ -25,49 +25,47 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; 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 org.apache.rocketmq.streams.common.utils.NumberUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * 压缩表,行数据以byte[][]存放 */ public abstract class AbstractMemoryTable { - private static final Log logger = LogFactory.getLog(AbstractMemoryTable.class); - + private static final Logger LOGGER = LoggerFactory.getLogger(AbstractMemoryTable.class); + static long compressCount = 0; /** * 列名和位置,根据列名生成一个位置 */ - protected Map cloumnName2Index = new HashMap<>(); - + protected Map columnName2Index = new HashMap<>(); /** * 位置和列名,根据位置获取列名 */ protected Map index2ColumnName = new HashMap<>(); - /** * 列名和类型的映射关系 */ - protected Map cloumnName2DatatType = new HashMap<>(); - + protected Map columnName2DataType = new HashMap<>(); /** * 列名当前索引值。根据获取的列的先后顺序,建立序号 */ protected AtomicInteger index = new AtomicInteger(0); - /** * 总字节数 */ protected AtomicLong byteCount = new AtomicLong(0); protected AtomicInteger rowCount = new AtomicInteger(0); - //需要压缩的列名 Set compressFieldNames = new HashSet<>(1); + transient long compressCompareCount = 0; + transient long compressByteLength = 0; + transient long deCompressByteLength = 0; public Set getCompressFieldNames() { return compressFieldNames; @@ -77,37 +75,6 @@ public void setCompressFieldNames(Set compressFieldNames) { this.compressFieldNames = compressFieldNames; } - static long compressCount = 0; - transient long compressCompareCount = 0; - transient long compressByteLength = 0; - transient long deCompressByteLength = 0; - - public static class RowElement { - protected Map row; - protected Long rowIndex; - - public RowElement(Map row, Long rowIndex) { - this.row = row; - this.rowIndex = rowIndex; - } - - public Map getRow() { - return row; - } - - public long getRowIndex() { - return rowIndex; - } - - @Override - public String toString() { - return "RowElement{" + - "row=" + row + - ", rowIndex=" + rowIndex + - '}'; - } - } - /** * 创建迭代器,可以循环获取全部数据,一行数据是Map * @@ -119,13 +86,11 @@ public Iterator> rowIterator() { return new Iterator>() { Iterator it = newIterator(); - @Override - public boolean hasNext() { + @Override public boolean hasNext() { return it.hasNext(); } - @Override - public Map next() { + @Override public Map next() { RowElement rowElement = it.next(); return rowElement.getRow(); } @@ -186,10 +151,6 @@ public byte[][] row2Byte(Map row) { return row2Byte(row, null); } - protected class CountHolder { - int count = 0; - } - /** * 把一个row行转换成byte[][]数组 * @@ -236,7 +197,7 @@ protected byte[][] row2Byte(Map row, CountHolder countHolder) { byteRows[i] = temp; } if ((compressCount++) % 100000 == 0) { - logger.info(this.hashCode() + " builder compress table continue..." + (compressCount - 1) + ", compressCompareCount is " + compressCompareCount + ", compressByteLength is " + compressByteLength + ", deCompressByteLength is " + deCompressByteLength); + LOGGER.info(this.hashCode() + " builder compress table continue..." + (compressCount - 1) + ", compressCompareCount is " + compressCompareCount + ", compressByteLength is " + compressByteLength + ", deCompressByteLength is " + deCompressByteLength); } return byteRows; } @@ -253,7 +214,7 @@ public Map byte2Row(byte[][] bytes) { byte[] columnValue = bytes[i]; String columnName = index2ColumnName.get(i); boolean isCompress = compressFieldNames.contains(columnName); - DataType dataType = cloumnName2DatatType.get(columnName); + DataType dataType = columnName2DataType.get(columnName); Object object = null; if (isCompress) { @@ -263,7 +224,6 @@ public Map byte2Row(byte[][] bytes) { object = dataType.byteToValue(decompressValue); } catch (Exception e) { e.printStackTrace(); - logger.error(e); } } else { object = dataType.byteToValue(columnValue); @@ -286,14 +246,14 @@ private byte[] createBytes(String key, Object value) { return null; } Object tmp = value; - DataType dataType = cloumnName2DatatType.get(key); + DataType dataType = columnName2DataType.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); + columnName2DataType.put(key, dataType); } Object object = dataType.convert(tmp); return dataType.toBytes(object, true); @@ -306,37 +266,37 @@ private byte[] createBytes(String key, Object value) { * @return */ public int getColumnIndex(String key) { - Integer columnIndex = cloumnName2Index.get(key); + Integer columnIndex = columnName2Index.get(key); if (columnIndex == null) { columnIndex = index.incrementAndGet() - 1; - cloumnName2Index.put(key, columnIndex); + columnName2Index.put(key, columnIndex); index2ColumnName.put(columnIndex, key); } return columnIndex; } public Map getCloumnName2Index() { - return cloumnName2Index; + return columnName2Index; } public Map getIndex2ColumnName() { return index2ColumnName; } - public Map getCloumnName2DatatType() { - return cloumnName2DatatType; + public void setIndex2ColumnName(Map index2ColumnName) { + this.index2ColumnName = index2ColumnName; } - public void setCloumnName2Index(Map cloumnName2Index) { - this.cloumnName2Index = cloumnName2Index; + public Map getCloumnName2DatatType() { + return columnName2DataType; } - public void setIndex2ColumnName(Map index2ColumnName) { - this.index2ColumnName = index2ColumnName; + public void setColumnName2Index(Map columnName2Index) { + this.columnName2Index = columnName2Index; } - public void setCloumnName2DatatType(Map cloumnName2DatatType) { - this.cloumnName2DatatType = cloumnName2DatatType; + public void setColumnName2DatatType(Map columnName2DatatType) { + this.columnName2DataType = columnName2DatatType; } public long getByteCount() { @@ -347,4 +307,30 @@ public int getRowCount() { return rowCount.get(); } + public static class RowElement { + protected Map row; + protected Long rowIndex; + + public RowElement(Map row, Long rowIndex) { + this.row = row; + this.rowIndex = rowIndex; + } + + public Map getRow() { + return row; + } + + public long getRowIndex() { + return rowIndex; + } + + @Override public String toString() { + return "RowElement{" + "row=" + row + ", rowIndex=" + rowIndex + '}'; + } + } + + protected class CountHolder { + int count = 0; + } + } 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 deleted file mode 100644 index ef6f8022..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/IntValueKV.java +++ /dev/null @@ -1,185 +0,0 @@ -/* - * 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.io.BufferedReader; -import java.io.FileInputStream; -import java.io.InputStreamReader; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Random; -import java.util.Set; -import java.util.UUID; -import org.apache.rocketmq.streams.common.cache.compress.ByteArray; -import org.apache.rocketmq.streams.common.cache.compress.ByteStore; -import org.apache.rocketmq.streams.common.cache.compress.CacheKV; -import org.apache.rocketmq.streams.common.utils.NumberUtils; -import org.junit.Assert; - -/** - * 支持key是string,value是int的场景,支持size不大于10000000.只支持int,long,boolean,string类型 - */ -public class IntValueKV extends CacheKV { - - protected ByteStore conflicts = new ByteStore(4); - - @Override - public Integer get(String key) { - ByteArray byteArray = super.getInner(key); - if (byteArray == null) { - return null; - } - int value = byteArray.castInt(0, 4); - return value; - } - - @Override - public void put(String key, Integer value) { - byte[] bytes = NumberUtils.toByte(value, 5); - bytes[4] = (byte) 0; - super.putInner(key, new ByteArray(bytes), true); - } - - @Override - public boolean contains(String key) { - Integer value = get(key); - if (value == null) { - return false; - } - return true; - - } - - @Override - public int calMemory() { - return super.calMemory() + (this.conflicts.getConflictIndex() + 1) * this.conflicts.getBlockSize(); - } - - /** - * TODO remove the key from the sinkcache and return the removed value - * - * @return - */ - public IntValueKV(int capacity) { - super(capacity, true); - } - - 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 dataMap = new HashMap<>(size); - Set whiteSet = new HashSet<>(1024); - Map sample1Map = new HashMap<>(1024); - Map 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> iterator = dataMap.entrySet().iterator(); - while (iterator.hasNext()) { - Entry 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> iterator1 = sample1Map.entrySet().iterator(); - Iterator> iterator2 = sample2Map.entrySet().iterator(); - while (iterator1.hasNext() && iterator2.hasNext()) { - Entry entry1 = iterator1.next(); - String key1 = entry1.getKey(); - Integer value1 = entry1.getValue(); - Entry 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/StringValueKV.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/StringValueKV.java deleted file mode 100644 index 814c8fb0..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/StringValueKV.java +++ /dev/null @@ -1,136 +0,0 @@ -/* - * 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.io.UnsupportedEncodingException; -import java.util.HashMap; -import java.util.Map; -import org.apache.rocketmq.streams.common.cache.compress.ByteArrayValueKV; -import org.apache.rocketmq.streams.common.cache.compress.ICacheKV; -import org.junit.Assert; - -/** - * 支持key是string,value是int的场景,支持size不大于10000000.只支持int,long,boolean,string类型 只能一次行load,不能进行更新 - */ -public class StringValueKV implements ICacheKV { - - protected final static String CODE = "UTF-8"; - protected ByteArrayValueKV values; - - public StringValueKV(int capacity) { - values = new ByteArrayValueKV(capacity); - } - - @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); - 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 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/impl/SoftReferenceCache.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/softreference/impl/SoftReferenceCache.java index e74acf14..f79cc5ff 100644 --- 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 @@ -29,6 +29,9 @@ public class SoftReferenceCache implements ICache { protected RebuildCacheElement rebuildCacheElement; + // 缓存,用软引用记录 + private ConcurrentHashMap> cache = new ConcurrentHashMap>(); + private ReferenceQueue refQueue = new ReferenceQueue(); public SoftReferenceCache(RebuildCacheElement rebuildCacheElement) { this.rebuildCacheElement = rebuildCacheElement; @@ -43,24 +46,6 @@ public V create(K k) { }; } - protected class ExtraInfoReference extends SoftReference { - - private Object info; - - public ExtraInfoReference(Object info, T t, ReferenceQueue refQueue) { - super(t, refQueue); - this.info = info; - } - - public Object getExtraInfo() { - return this.info; - } - } - - // 缓存,用软引用记录 - private ConcurrentHashMap> cache = new ConcurrentHashMap>(); - private ReferenceQueue refQueue = new ReferenceQueue(); - @Override public V get(K key) { V value = null; @@ -117,10 +102,24 @@ public void clear() { @SuppressWarnings("unchecked") protected void clearRefQueue() { ExtraInfoReference refValue = null; - while ((refValue = (ExtraInfoReference)refQueue.poll()) != null) { - K key = (K)refValue.getExtraInfo(); + while ((refValue = (ExtraInfoReference) refQueue.poll()) != null) { + K key = (K) refValue.getExtraInfo(); cache.remove(key); } } + protected class ExtraInfoReference extends SoftReference { + + private Object info; + + public ExtraInfoReference(Object info, T t, ReferenceQueue refQueue) { + super(t, refQueue); + this.info = info; + } + + public Object getExtraInfo() { + return this.info; + } + } + } 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 index 7591cd09..84b88564 100644 --- 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 @@ -25,44 +25,69 @@ import java.util.ArrayList; import java.util.Enumeration; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Set; import java.util.jar.JarEntry; import java.util.jar.JarFile; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.classloader.IsolationClassLoader; import org.apache.rocketmq.streams.common.utils.FileUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; - +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.boot.loader.archive.Archive; +import org.springframework.boot.loader.archive.JarFileArchive; +import org.springframework.boot.loader.jar.Handler; + +/** + * 可以扫描指定位置的类,支持目录和package扫描 + */ public abstract class AbstractScan { - private static final Log LOG = LogFactory.getLog(AbstractScan.class); + private static final Logger LOG = LoggerFactory.getLogger(AbstractScan.class); private static final String CLASS_REAR = ".class"; protected Set scanDirs = new HashSet<>(); + public static void main(String[] args) throws IOException, ClassNotFoundException { + AbstractScan scan = new AbstractScan() { + @Override + protected void doProcessor(Class clazz, String functionName) { + System.out.println(clazz.getName()); + } + }; + scan.scanClassInJar(new URL("jar:file:/Users/fgm0129/Desktop/stream-engine.jar!/BOOT-INF/lib/rsqldb-parser-1.0.3_siem.industry-SNAPSHOT.jar!/com/alibaba/rsqldb/parser/parser/sqlnode"), "com/alibaba/rsqldb/parser/parser/sqlnode".replace("/", "."), AbstractScan.class.getClassLoader(), null); +// scan.scanClassInJar(new URL("jar:file:/Users/yuanxiaodong/Downloads/tmp/et-industry-stream-engine.jar!/BOOT-INF/lib/rsqldb-parser-1.0.3_siem.industry-SNAPSHOT.jar!/com/alibaba/rsqldb/parser/parser/sqlnode"),"com/alibaba/rsqldb/parser/parser/sqlnode".replace("/","."),AbstractScan.class.getClassLoader(),"xxx"); + } + public void scanJarsFromDir(String dir, String packageName) { IsolationClassLoader classLoader = new IsolationClassLoader(dir); File file = new File(dir); - if (file.exists() == false) { + if (!file.exists()) { return; } - if (file.isDirectory() == false) { + if (!file.isDirectory()) { return; } File[] jars = file.listFiles(); - for (File jar : jars) { - if (!jar.getName().endsWith(".jar")) { - continue; + if (jars != null) { + for (File jar : jars) { + if (!jar.getName().endsWith(".jar")) { + continue; + } + scanClassDir(jar, packageName, classLoader, null); } - scanClassDir(jar, packageName, classLoader, null); } } public void scanClassDir(File jarFile, String packageName, ClassLoader classLoader, String functionName) { - scanClassInJar(jarFile.getAbsolutePath(), packageName, classLoader, functionName); + try { + scanClassInJar(new URL("jar:file:" + jarFile.getAbsolutePath()), packageName, classLoader, functionName); + } catch (Exception e) { + throw new RuntimeException("url parse error " + jarFile.getAbsolutePath(), e); + } + } public void scanClassDir(String dir, String packageName, ClassLoader classLoader) { @@ -84,19 +109,20 @@ public void scanClassDir(String dir, String packageName, ClassLoader classLoader } 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, null); + if (files != null) { + 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, null); + } + } catch (ClassNotFoundException e) { + LOG.error("load class error " + file.getName(), e); } - } catch (ClassNotFoundException e) { - LOG.error("load class error " + file.getName(), e); - continue; } } } @@ -139,7 +165,6 @@ public void scanPackage(String packageName) { scanDir(dir); hasScan.add(dir); } - } } @@ -197,39 +222,85 @@ protected void scanClassInJar(String dirName) { } 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(), null); + scanClassInJar(url, packageName, this.getClass().getClassLoader(), null); + } + protected void scanClassInJar(String packageName, ClassLoader classLoader, String functionName) { + try { + if (classLoader == null) { + classLoader = this.getClass().getClassLoader(); + } + doRegisterFunction(functionName, packageName, classLoader); + } catch (Exception e) { + LOG.error("ScanFunctionService scanClassInJar JarFile error", e); + } } - protected void scanClassInJar(String jarPath, String packageName, ClassLoader classLoader, String functionName) { + protected void scanClassInJar(URL url, String packageName, ClassLoader classLoader, String functionName) { try { if (classLoader == null) { classLoader = this.getClass().getClassLoader(); } - JarFile jarFile = new JarFile(jarPath); - Enumeration 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(functionName, className, classLoader); + if (isNestJar(url)) { + Handler handler = new Handler(); + org.springframework.boot.loader.jar.JarFile root = handler.getRootJarFileFromUrl(url); + int startIndex = url.toString().indexOf("!/"); + int endIndex = url.toString().lastIndexOf("!/"); + String jarPath = url.toString().substring(startIndex + 2, endIndex); + JarFileArchive jarFileArchive = new JarFileArchive(root); + //过滤Jar包 + Iterator jarFileIterator = jarFileArchive.getNestedArchives(entry -> { + if (entry.getName().equals(jarPath)) { + System.out.println(jarPath); + return true; + } + return false; + }, null); + final ClassLoader jarClassLoader = classLoader; + while (jarFileIterator.hasNext()) { + JarFileArchive archive = (JarFileArchive) jarFileIterator.next(); + //过滤class + archive.iterator().forEachRemaining(entry -> { + String className = entry.getName().replace("/", "."); + if (className.startsWith(packageName) && className.endsWith(".class")) { + className = className.replace(CLASS_REAR, ""); + doRegisterFunction(functionName, className, jarClassLoader); + } + }); } + } else { + String jarPath = url.toString().replace("jar:file:", ""); + int index = jarPath.indexOf("!/"); + jarPath = jarPath.substring(0, index); + JarFile jarFile = new JarFile(jarPath); + Enumeration 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(functionName, className, classLoader); + } + } } + } catch (Exception e) { - LOG.error("ScanFunctionService scanClassInJar JarFile error", e); + LOG.error("ScanFunctionService scanClassInJar JarFile url:{},packageName:{},functionName:{},error:{}", url.getPath(), packageName, functionName, e); + throw new RuntimeException("ScanFunctionService scanClassInJar JarFile error", e); } } + protected boolean isNestJar(URL url) { + String urlStr = url.toString(); + int startIndex = urlStr.indexOf("!/"); + int endIndex = urlStr.lastIndexOf("!/"); + if (startIndex == endIndex) { + return false; + } + return true; + } + protected void scanClassInDir(String dirName) { InputStream in = this.getClass().getResourceAsStream(dirName); @@ -257,8 +328,7 @@ protected void scanClassInDir(String dirName) { protected String createPackageName(String dirName) { if (dirName.startsWith("/")) { - String packageName = dirName.substring(1).replace("/", ".") + "."; - return packageName; + return dirName.substring(1).replace("/", ".") + "."; } else { return this.getClass().getPackage().getName() + "." + dirName + "."; } @@ -269,15 +339,15 @@ protected void doRegisterFunction(String className) { } protected void doRegisterFunction(String functionName, String className, ClassLoader classLoader) { - Class clazz = null; + Class clazz = null; try { clazz = Class.forName(className, true, classLoader); doProcessor(clazz, functionName); } catch (Exception e) { - e.printStackTrace(); LOG.error("初始化类错误" + e.getMessage(), e); } } protected abstract void doProcessor(Class clazz, String functionName); + } 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 index 35107071..1f34ad36 100644 --- 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 @@ -24,10 +24,10 @@ 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.source.SplitProgress; 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.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.MessageOffset; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; @@ -42,13 +42,12 @@ public abstract class AbstractChannel extends BasedConfigurable implements IChannel { protected transient ISink sink; protected transient ISource source; + protected transient AtomicBoolean hasCreated = new AtomicBoolean(false); protected abstract ISink createSink(); protected abstract ISource createSource(); - protected transient AtomicBoolean hasCreated = new AtomicBoolean(false); - @Override protected boolean initConfigurable() { create(); @@ -74,14 +73,14 @@ public void getJsonObject(JSONObject jsonObject) { create(); if (sourceValue != null) { source = InstantiationUtil.deserializeObject(Base64Utils.decode(sourceValue)); - if(source!=null){ + if (source != null) { source.init(); } } if (sinkValue != null) { sink = InstantiationUtil.deserializeObject(Base64Utils.decode(sinkValue)); - if(sink!=null){ + if (sink != null) { sink.init(); } } @@ -109,22 +108,30 @@ protected void setJsonObject(JSONObject jsonObject) { } public void removeSplit(Set splitIds) { - if(source instanceof AbstractSource){ - (( AbstractSource)source).removeSplit(splitIds); + if (source instanceof AbstractSource) { + ((AbstractSource) source).removeSplit(splitIds); } } public void addNewSplit(Set splitIds) { - if(source instanceof AbstractSource){ - (( AbstractSource)source).addNewSplit(splitIds); + if (source instanceof AbstractSource) { + ((AbstractSource) source).addNewSplit(splitIds); } } + @Override public Map getSplitProgress() { + return source.getSplitProgress(); + } + @Override public Map getFinishedQueueIdAndOffsets(CheckPointMessage checkPointMessage) { return sink.getFinishedQueueIdAndOffsets(checkPointMessage); } + @Override public List> fetchAllSplits() { + return source.fetchAllSplits(); + } + @Override public boolean flush(Set splitIds) { return sink.flush(splitIds); @@ -156,7 +163,7 @@ public ISource getSource() { } @Override - public boolean batchAdd(IMessage fieldName2Value, ISplit split) { + public boolean batchAdd(IMessage fieldName2Value, ISplit split) { return sink.batchAdd(fieldName2Value, split); } @@ -186,13 +193,13 @@ public void closeAutoFlush() { } @Override - public void setBatchSize(int batchSize) { - sink.setBatchSize(batchSize); + public int getBatchSize() { + return sink.getBatchSize(); } @Override - public int getBatchSize() { - return sink.getBatchSize(); + public void setBatchSize(int batchSize) { + sink.setBatchSize(batchSize); } @Override @@ -232,24 +239,22 @@ public long getCheckpointTime() { public void setJsonData(Boolean isJsonData) { create(); - ((AbstractSource)source).setJsonData(isJsonData); + ((AbstractSource) source).setJsonData(isJsonData); } @Override - public String getTopic(){ + public String getTopic() { return source.getTopic(); } @Override - public void setTopic(String topic){ + public void setTopic(String topic) { source.setTopic(topic); } @Override - public void atomicSink(ISystemMessage message){ + public void atomicSink(ISystemMessage message) { } - - } 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 index be374750..7c5dc79a 100644 --- 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 @@ -23,11 +23,10 @@ /** * source和sink的结合,dipper框架中,不单独区分source和sink,统一用channel 接口表示 */ -public interface IChannel - extends ISink, ISource, Serializable { +public interface IChannel extends ISink, ISource, Serializable { String TYPE = "channel"; - ISink getSink(); + ISink getSink(); - ISource getSource(); + ISource getSource(); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/AbstractChannelSQLParser.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/AbstractChannelSQLParser.java index 9991ef85..f982803d 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/AbstractChannelSQLParser.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/AbstractChannelSQLParser.java @@ -19,12 +19,10 @@ import org.apache.rocketmq.streams.common.channel.sink.ISink; import org.apache.rocketmq.streams.common.channel.source.ISource; -public abstract class AbstractChannelSQLParser extends AbstractSupportShuffleChannelBuilder{ +public abstract class AbstractChannelSQLParser extends AbstractSupportShuffleChannelBuilder { @Override public ISink createBySource(ISource pipelineSource) { throw new RuntimeException("can not this method "); } - - } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/AbstractChannelSQLSupportShuffleSQLParser.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/AbstractChannelSQLSupportShuffleSQLParser.java index 31a2fe1e..13d2c681 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/AbstractChannelSQLSupportShuffleSQLParser.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/AbstractChannelSQLSupportShuffleSQLParser.java @@ -22,15 +22,14 @@ import org.apache.rocketmq.streams.common.metadata.MetaData; import org.apache.rocketmq.streams.common.utils.ConfigurableUtil; -public abstract class AbstractChannelSQLSupportShuffleSQLParser extends AbstractSupportShuffleChannelBuilder{ - - +public abstract class AbstractChannelSQLSupportShuffleSQLParser extends AbstractSupportShuffleChannelBuilder { @Override public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) { return (ISource) ConfigurableUtil.create(getSourceClass(), namespace, name, createFormatProperty(properties), null); } protected abstract String getSourceClass(); + protected abstract String getSinkClass(); @Override public String getType() { 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 index cc149b8c..da45e18c 100644 --- 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 @@ -26,7 +26,7 @@ public abstract class AbstractSupportShuffleChannelBuilder implements IChannelBu @Override public ISource copy(ISource pipelineSource) { JSONObject jsonObject = JSONObject.parseObject(pipelineSource.toJson()); - return ConfigurableUtil.create(pipelineSource.getNameSpace(), pipelineSource.getConfigureName(), jsonObject, pipelineSource.getClass().getName()); + return ConfigurableUtil.create(pipelineSource.getNameSpace(), pipelineSource.getName(), jsonObject, pipelineSource.getClass().getName()); } protected JSONObject createFormatProperty(Properties properties) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/CollectionSource.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/CollectionSource.java index 47e5810f..ca103a0c 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/CollectionSource.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/CollectionSource.java @@ -22,28 +22,22 @@ import java.util.List; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicLong; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.channel.source.AbstractSource; +import org.apache.rocketmq.streams.common.channel.source.AbstractSingleSplitSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * @description for test checkpoint */ -public class CollectionSource extends AbstractSource implements Serializable { - - private static final Log logger = LogFactory.getLog(CollectionSource.class); +public class CollectionSource extends AbstractSingleSplitSource implements Serializable { + private static final Logger logger = LoggerFactory.getLogger(CollectionSource.class); + private static final int checkpointIntervalMs = 10 * 1000; transient ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); - transient AtomicLong offset = new AtomicLong(0); - long maxOffset; - //must be json string List elements; - - private static final int checkpointIntervalMs = 10 * 1000; - long lastCheckpointTime = System.currentTimeMillis(); transient volatile long currentOffset; @@ -115,24 +109,8 @@ public void run() { return true; } - @Override - public boolean supportNewSplitFind() { - return false; - } - - @Override - public boolean supportRemoveSplitFind() { - return false; - } + @Override protected void destroySource() { - @Override - public boolean supportOffsetRest() { - return true; - } - - @Override - protected boolean isNotDataSplit(String queueId) { - return false; } public long getMaxOffset() { 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 index eb846890..557f4924 100644 --- 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 @@ -21,7 +21,8 @@ import java.util.Properties; 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; @@ -84,11 +85,11 @@ protected JSONObject createFormatProperty(Properties properties) { @Override public ISource copy(ISource pipelineSource) { - return new MemoryChannel(); + return new MemorySource(); } @Override public ISink createBySource(ISource pipelineSource) { - return new MemoryChannel(); + return new MemorySink(); } } 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 index 93f727a4..48db2545 100644 --- 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 @@ -34,18 +34,17 @@ */ public class FileSink extends AbstractSupportShuffleSink { + private static final String PREFIX = "dipper.upgrade.channel.file.envkey"; + protected transient BufferedWriter writer; /** * 文件全路径。如果目录是环境变量,可以写成evnDir/文件名 */ @ENVDependence private String filePath; - /** * 文件写入时,是否追加 */ private boolean needAppend = true; - - protected transient BufferedWriter writer; /** * FileChannel 中 writer 完成 初始化 标识 */ @@ -63,6 +62,10 @@ public FileSink(String filePath, boolean needAppend) { } + public FileSink() { + setType(ISink.TYPE); + } + @Override public String getShuffleTopicFieldName() { return "filePath"; @@ -74,9 +77,9 @@ protected void createTopicIfNotExist(int splitNum) { } @Override - public List> getSplitList() { + public List> getSplitList() { File file = new File(filePath); - List> splits = new ArrayList<>(); + List> splits = new ArrayList<>(); splits.add(new FileSplit(file)); return splits; } @@ -105,24 +108,16 @@ protected boolean batchInsert(List messages) { @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 防止文件不存在异常 */ @@ -131,12 +126,12 @@ private void initWrite() { synchronized (this) { if (!writerInitFlag) { try { - File file=new File(filePath); - File dir=file.getParentFile(); - if(dir.exists()==false){ + File file = new File(filePath); + File dir = file.getParentFile(); + if (dir.exists() == false) { dir.mkdirs(); } - if(file.exists()==false){ + if (file.exists() == false) { file.createNewFile(); } writer = new BufferedWriter(new FileWriter(filePath, 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 index 2843abff..1070a111 100644 --- 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 @@ -16,53 +16,87 @@ */ package org.apache.rocketmq.streams.common.channel.impl.file; +import com.alibaba.fastjson.JSONObject; +import com.univocity.parsers.common.processor.RowListProcessor; +import com.univocity.parsers.csv.CsvParser; +import com.univocity.parsers.csv.CsvParserSettings; import java.io.BufferedReader; import java.io.File; import java.io.FileNotFoundException; import java.io.FileReader; import java.io.IOException; import java.net.URL; +import java.util.ArrayList; import java.util.Iterator; +import java.util.List; 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 org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; -import org.apache.rocketmq.streams.common.channel.source.AbstractBatchSource; +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.configurable.annotation.ENVDependence; import org.apache.rocketmq.streams.common.context.Message; +import org.apache.rocketmq.streams.common.threadpool.ThreadPoolFactory; import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; /** * 文件的输入输出,source是把指定的文件数据加载到内存,for循环输出到后续节点 sink,把内容写入文件,可以配置写入模式,是追加还是覆盖 */ -public class FileSource extends AbstractBatchSource { +public class FileSource extends AbstractSource { + protected String namePattern;//可以配置正则过滤需要的文件 + protected transient BufferedReader reader; + protected transient ExecutorService executorService; + protected boolean isCSV = false; + protected transient String[] fieldNames; + protected transient Object locker = new Object(); /** * 文件全路径。如果目录是环境变量,可以写成evnDir/文件名 */ @ENVDependence private String filePath; - protected transient BufferedReader reader; - protected transient ExecutorService executorService; - public FileSource(String filePath) { this(); this.filePath = filePath; } + public FileSource() { + setType(ISource.TYPE); + } + + public static void main(String[] args) { + FileSource fileSource = new FileSource("/Users/yuanxiaodong/Downloads/sample.csv"); + fileSource.setCSV(true); + fileSource.init(); + fileSource.start((message, context) -> { + if (message.getHeader().isSystemMessage()) { + return null; + } + System.out.println(message.getMessageBody()); + return null; + }); + } + @Override protected boolean initConfigurable() { super.initConfigurable(); File file = getFile(filePath); if (file.exists() && file.isDirectory()) { - executorService = new ThreadPoolExecutor(maxThread, maxThread, 0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue(1000)); + if (executorService == null) { + executorService = ThreadPoolFactory.createFixedThreadPool(maxThread, FileSource.class.getName() + "-" + getName()); + } } - return true; + if (file.exists() && !file.isDirectory()) { + if (executorService == null) { + executorService = ThreadPoolFactory.createFixedThreadPool(1, FileSource.class.getName() + "-" + getName()); + } + } + return super.initConfigurable(); } private File getFile(String filePath) { @@ -83,13 +117,13 @@ private File getFile(String filePath) { @Override protected boolean startSource() { - LinkedBlockingQueue 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) { @@ -112,11 +146,11 @@ protected boolean startSource() { /** * 如果是目录,每个文件一个iterator,如果是文件只生成一个iterator */ - protected LinkedBlockingQueue createIteratorList() { + public LinkedBlockingQueue createIteratorList() { LinkedBlockingQueue iterators = new LinkedBlockingQueue<>(1000); File file = getFile(filePath); if (!file.exists()) { - return null; + throw new RuntimeException("The file[" + filePath + "] does not exists"); } try { if (!file.isDirectory()) { @@ -125,7 +159,13 @@ protected LinkedBlockingQueue createIteratorList() { } File[] files = file.listFiles(); for (File subFile : files) { - iterators.add(new FileIterator(subFile)); + if (namePattern == null) { + iterators.add(new FileIterator(subFile)); + } else { + if (StringUtil.matchRegex(subFile.getAbsolutePath(), namePattern)) { + iterators.add(new FileIterator(subFile)); + } + } } return iterators; } catch (Exception e) { @@ -134,14 +174,98 @@ protected LinkedBlockingQueue createIteratorList() { } + @Override + public void destroySource() { + + try { + if (reader != null) { + reader.close(); + } + if (this.executorService != null) { + this.executorService.shutdown(); + this.executorService = null; + } + } catch (IOException e) { + String realFilePath = filePath; + throw new RuntimeException("close error " + realFilePath, e); + } + } + + @Override public List> fetchAllSplits() { + File file = getFile(filePath); + ISplit split = new FileSplit(file); + List> splits = new ArrayList<>(); + splits.add(split); + return splits; + } + + public String getFilePath() { + return filePath; + } + + public void setFilePath(String filePath) { + this.filePath = filePath; + } + + public List parseCSV(File file) { + //创建一个配置选项,用来提供多种配置选项 + CsvParserSettings parserSettings = new CsvParserSettings(); + //打开解析器的自动检测功能,让它自动检测输入中包含的分隔符 + parserSettings.setLineSeparatorDetectionEnabled(true); + + //创建RowListProcessor对象,用来把每个解析的行存储在列表中 + RowListProcessor rowListProcessor = new RowListProcessor(); + parserSettings.setProcessor(rowListProcessor); //配置解析器 + //待解析的CSV文件包含标题头,把第一个解析行看作文件中每个列的标题 + parserSettings.setHeaderExtractionEnabled(true); + parserSettings.setLineSeparatorDetectionEnabled(true); + + //创建CsvParser对象,用于解析文件 + CsvParser parser = new CsvParser(parserSettings); + parser.parse(file); + + //如果解析中包含标题,用于获取标题 + String[] headers = rowListProcessor.getHeaders(); + //获取行值,并遍历打印 + List rows = rowListProcessor.getRows(); + List messages = new ArrayList<>(); + for (String[] row : rows) { + JSONObject message = new JSONObject(); + for (int i = 0; i < headers.length; i++) { + message.put(headers[i], row[i]); + } + messages.add(message); + } + /*for(int i = 0; i < rows.size(); i++){ + System.out.println(Arrays.asList(rows.get(i))); + }*/ + return messages; + } + + public String getNamePattern() { + return namePattern; + } + + public void setNamePattern(String namePattern) { + this.namePattern = namePattern; + } + + public boolean isCSV() { + return isCSV; + } + + public void setCSV(boolean CSV) { + isCSV = CSV; + } + /** * 一个文件生成一个Iterator,每次加载一行数据 */ public static class FileIterator implements Iterator { protected File file; - private String line; protected int index = 0; protected BufferedReader reader = null; + private String line; public FileIterator(File file) throws FileNotFoundException { this.file = file; @@ -179,6 +303,10 @@ public void close() { public String next() { return line; } + + public File getFile() { + return file; + } } /** @@ -199,11 +327,21 @@ public ReadTask(FileIterator fileIterator, AtomicInteger count, CountDownLatch c 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(); + if (isCSV) { + List messages = parseCSV(fileIterator.file); + for (JSONObject msg : messages) { + doReceiveMessage(msg, false, fileIterator.file.getName(), offset + ""); + offset++; + count.incrementAndGet(); + } + } else { + + while (fileIterator.hasNext()) { + String line = fileIterator.next(); + doReceiveMessage(line, false, fileIterator.file.getName(), offset + ""); + offset++; + count.incrementAndGet(); + } } sendCheckpoint(fileIterator.file.getName()); @@ -214,33 +352,6 @@ public void run() { } } - - @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 index 0a1b11e5..167dd809 100644 --- 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 @@ -23,12 +23,18 @@ public class FileSplit extends BasedConfigurable implements ISplit { protected transient File file; + protected String splitId; private String filePath; - protected int splitId; + + public FileSplit(File file) { + this.filePath = file.getAbsolutePath(); + splitId = this.filePath; + this.file = file; + } @Override public String getQueueId() { - return filePath + "_" + splitId; + return filePath; } @Override @@ -38,20 +44,17 @@ public File getQueue() { @Override public int compareTo(FileSplit o) { - return splitId - o.splitId; + return splitId.compareTo(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; + public File getFile() { + return file; } } 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 deleted file mode 100644 index e6d61e9b..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemoryChannel.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * 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.List; -import java.util.concurrent.atomic.AtomicLong; -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; - -/** - * 消息产生的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 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(); - } - }; - } - - @Override - public String createCheckPointName() { - return "memory-source"; - } - -} 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 index 720e3e17..cdadd1a1 100644 --- 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 @@ -54,11 +54,11 @@ public ISink createSink(String namespace, String name, Properties properties, Me @Override public ISource copy(ISource pipelineSource) { - return new MemoryChannel(); + return new MemorySource(); } @Override public ISink createBySource(ISource pipelineSource) { - return new MemoryChannel(); + return new MemorySink(); } } 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 index 618dbb83..8ace8ff3 100644 --- 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 @@ -21,11 +21,10 @@ 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.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.configurable.annotation.ConfigurableReference; import org.apache.rocketmq.streams.common.context.IMessage; -public class MemorySink extends AbstractSupportShuffleSink implements IAfterConfigurableRefreshListener { +public class MemorySink extends AbstractSupportShuffleSink { /** * 是否启动qps的统计 */ @@ -38,8 +37,7 @@ public class MemorySink extends AbstractSupportShuffleSink implements IAfterConf * 最早的处理时间 */ protected transient long firstReceiveTime = System.currentTimeMillis(); - protected String cacheName; - protected transient MemoryCache memoryCache; + @ConfigurableReference protected MemoryCache memoryCache; public MemorySink() { } @@ -74,8 +72,8 @@ protected void createTopicIfNotExist(int splitNum) { } @Override - public List> getSplitList() { - List> splits = new ArrayList<>(); + public List> getSplitList() { + List> splits = new ArrayList<>(); splits.add(new MemorySplit()); return splits; } @@ -85,22 +83,8 @@ 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 index c59c0477..799759cb 100644 --- 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 @@ -16,14 +16,13 @@ */ package org.apache.rocketmq.streams.common.channel.impl.memory; -import org.apache.rocketmq.streams.common.channel.source.AbstractBatchSource; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.channel.source.AbstractSingleSplitSource; +import org.apache.rocketmq.streams.common.configurable.annotation.ConfigurableReference; -public class MemorySource extends AbstractBatchSource implements IAfterConfigurableRefreshListener { +public class MemorySource extends AbstractSingleSplitSource { - protected String cacheName; - protected transient MemoryCache memoryCache; + @ConfigurableReference protected transient MemoryCache memoryCache; + protected transient volatile boolean isClosed = false; public MemorySource() { @@ -36,23 +35,29 @@ protected boolean initConfigurable() { @Override protected boolean startSource() { - boolean success = super.startSource(); + isClosed = false; Thread thread = new Thread(new Runnable() { @Override public void run() { - try { - while (true) { - Object message = memoryCache.queue.poll(); - while (message != null) { - doReceiveMessage(createJson(message)); - message = memoryCache.queue.poll(); + Long startTime = System.currentTimeMillis(); + while (!isClosed) { + Object message = memoryCache.queue.poll(); + while (message != null) { + doReceiveMessage(createJson(message)); + if (memoryCache.queue != null && memoryCache.queue.size() > 10) { + System.out.println("memory source queues count is " + memoryCache.queue.size()); } + message = memoryCache.queue.poll(); + if (System.currentTimeMillis() - startTime > getCheckpointTime()) { + sendCheckpoint(getQueueId()); + startTime = System.currentTimeMillis(); + } + } + if (System.currentTimeMillis() - startTime > getCheckpointTime()) { sendCheckpoint(getQueueId()); - Thread.sleep(1000); + startTime = System.currentTimeMillis(); } - - } catch (Exception e) { - throw new RuntimeException(e); + sleepThread(100); } } @@ -61,31 +66,29 @@ public void run() { return true; } - @Override - public String getQueueId() { - return "1"; + private void sleepThread(int time) { + try { + Thread.sleep(time); + } catch (Exception e) { + e.printStackTrace(); + } } - public String getCacheName() { - return cacheName; + @Override protected void destroySource() { + this.isClosed = true; } - public void setCacheName(String cacheName) { - this.cacheName = cacheName; + @Override + public String getQueueId() { + return "1"; } - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - memoryCache = configurableService.queryConfigurable(MemoryCache.TYPE, cacheName); + public MemoryCache getMemoryCache() { + return memoryCache; } 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 index 0a5247f8..183312d9 100644 --- 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 @@ -21,6 +21,9 @@ import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; public class MemorySplit extends BasedConfigurable implements ISplit { + public MemorySplit() { + } + @Override public String getQueueId() { return 1 + ""; @@ -41,7 +44,4 @@ 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/impl/memory/RowListChannelBuilder.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/RowListChannelBuilder.java new file mode 100644 index 00000000..e1f9007b --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/RowListChannelBuilder.java @@ -0,0 +1,87 @@ +/* + * 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 com.google.auto.service.AutoService; +import java.util.Properties; +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.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 = RowListChannelBuilder.TYPE, aliasName = "RowListSource") +public class RowListChannelBuilder extends AbstractSupportShuffleChannelBuilder { + + public static final String TYPE = "row_list"; + + /** + * @param namespace + * @param name + * @param properties + * @return + */ + @Override + public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) { + return (ISource) ConfigurableUtil.create(RowListSource.class.getName(), namespace, name, createFormatProperty(properties), null); + } + + @Override + public String getType() { + return TYPE; + } + + @Override + public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { + throw new RuntimeException("can not support this method"); + } + + /** + * 创建标准的属性文件 + * + * @param properties + * @return + */ + @Override + 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.get(key)); + } + IChannelBuilder.formatPropertiesName(formatProperties, properties, "rows", "list"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "rows", "data"); + return formatProperties; + } + + @Override + public ISource copy(ISource pipelineSource) { + return new RowListSource(); + } + + @Override + public ISink createBySource(ISource pipelineSource) { + throw new RuntimeException("can not support this method"); + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/RowListSource.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/RowListSource.java new file mode 100644 index 00000000..49ecddf9 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/RowListSource.java @@ -0,0 +1,51 @@ +/* + * 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 org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; +import org.apache.rocketmq.streams.common.channel.source.AbstractSingleSplitSource; +import org.apache.rocketmq.streams.common.context.Message; + +public class RowListSource extends AbstractSingleSplitSource { + + protected List rows = new ArrayList<>(); + + @Override + protected boolean startSource() { + + for (String row : rows) { + doReceiveMessage(createJson(row)); + } + sendCheckpoint(getQueueId()); + executeMessage((Message) BatchFinishMessage.create()); + return true; + } + + @Override protected void destroySource() { + + } + + public List getRows() { + return rows; + } + + public void setRows(List rows) { + this.rows = rows; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewSink.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewSink.java index 82265d47..c47ddfa8 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewSink.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewSink.java @@ -18,15 +18,18 @@ package org.apache.rocketmq.streams.common.channel.impl.view; import java.util.List; -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.context.IMessage; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ViewSink extends AbstractSink { - private static final Log LOG = LogFactory.getLog(ViewSink.class); + private static final Logger LOG = LoggerFactory.getLogger(ViewSink.class); protected String viewTableName; + protected int parallelTasks = 4; + protected int pollingTimeSecond = 10; + @Override protected boolean batchInsert(List messages) { return false; } @@ -38,4 +41,20 @@ public String getViewTableName() { public void setViewTableName(String viewTableName) { this.viewTableName = viewTableName; } + + public int getParallelTasks() { + return parallelTasks; + } + + public void setParallelTasks(int parallelTasks) { + this.parallelTasks = parallelTasks; + } + + public int getPollingTimeSecond() { + return pollingTimeSecond; + } + + public void setPollingTimeSecond(int pollingTimeSecond) { + this.pollingTimeSecond = pollingTimeSecond; + } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewSource.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewSource.java index 710fefed..1f1a4c26 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewSource.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewSource.java @@ -17,43 +17,30 @@ package org.apache.rocketmq.streams.common.channel.impl.view; +import java.util.List; import org.apache.rocketmq.streams.common.channel.source.AbstractSource; +import org.apache.rocketmq.streams.common.channel.split.ISplit; import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; -import org.apache.rocketmq.streams.common.topology.task.TaskAssigner; public class ViewSource extends AbstractSource { protected String tableName; - + protected AbstractSource rootSource; @Override public void addConfigurables(PipelineBuilder pipelineBuilder) { - TaskAssigner taskAssigner=new TaskAssigner(); - taskAssigner.setTaskName(tableName); - taskAssigner.setPipelineName(pipelineBuilder.getPipelineName());; - pipelineBuilder.addConfigurables(taskAssigner); pipelineBuilder.addConfigurables(this); } - - @Override protected boolean startSource() { return true; } - @Override public boolean supportNewSplitFind() { - return false; - } - - @Override public boolean supportRemoveSplitFind() { - return false; + @Override public List> fetchAllSplits() { + return rootSource.fetchAllSplits(); } - @Override public boolean supportOffsetRest() { - return false; - } + @Override protected void destroySource() { - @Override protected boolean isNotDataSplit(String queueId) { - return false; } public String getTableName() { @@ -63,4 +50,12 @@ public String getTableName() { public void setTableName(String tableName) { this.tableName = tableName; } + + public AbstractSource getRootSource() { + return rootSource; + } + + public void setRootSource(AbstractSource rootSource) { + this.rootSource = rootSource; + } } 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 index 3b4c8317..eb8af7e0 100644 --- 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 @@ -23,8 +23,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -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.MessageCache; import org.apache.rocketmq.streams.common.channel.sinkcache.impl.MultiSplitMessageCache; @@ -35,34 +33,36 @@ import org.apache.rocketmq.streams.common.checkpoint.SourceState; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; import org.apache.rocketmq.streams.common.configurable.IConfigurableIdentification; -import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.MessageOffset; import org.apache.rocketmq.streams.common.interfaces.ILifeCycle; import org.apache.rocketmq.streams.common.interfaces.ISystemMessage; +import org.apache.rocketmq.streams.common.metadata.MetaData; import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * 输出的接口抽象,针对json消息的场景 */ public abstract class AbstractSink extends BasedConfigurable implements ISink, ILifeCycle { - private static final Log logger = LogFactory.getLog(AbstractSink.class); - public static String TARGET_QUEUE = "target_queue";//指定发送queue public static final int DEFAULT_BATCH_SIZE = 3000; + private static final Logger logger = LoggerFactory.getLogger(AbstractSink.class); + public static String TARGET_QUEUE = "target_queue";//指定发送queue protected transient IMessageCache messageCache; protected volatile int batchSize = DEFAULT_BATCH_SIZE; protected transient volatile Map sourceName2State = new HashMap<>();//保存完成刷新的queueid和offset protected volatile int autoFlushSize = 300; protected volatile int autoFlushTimeGap = 1000; - + protected MetaData metaData;//可以指定meta data,和insertSQL二选一 public AbstractSink() { setType(TYPE); + } - @Override - protected boolean initConfigurable() { + @Override protected boolean initConfigurable() { messageCache = new MultiSplitMessageCache(this); ((MessageCache) messageCache).setAutoFlushTimeGap(autoFlushTimeGap); ((MessageCache) messageCache).setAutoFlushSize(autoFlushSize); @@ -72,13 +72,13 @@ protected boolean initConfigurable() { } @Override - public boolean batchAdd(IMessage message, ISplit split) { + public boolean batchAdd(IMessage message, ISplit split) { message.getMessageBody().put(TARGET_QUEUE, split); return batchAdd(message); } - public ISplit getSplit(IMessage message) { - return (ISplit) message.getMessageBody().get(TARGET_QUEUE); + public ISplit getSplit(IMessage message) { + return (ISplit) message.getMessageBody().get(TARGET_QUEUE); } @Override @@ -94,6 +94,7 @@ public void openAutoFlush() { @Override public boolean batchSave(List messages) { + init(); if (messages == null || messages.size() == 0) { //LOG.warn("has empty data to insert"); return true; @@ -150,7 +151,9 @@ public boolean flush(String... splitIds) { @Override public void closeAutoFlush() { - messageCache.closeAutoFlush(); + if (messageCache != null) { + messageCache.closeAutoFlush(); + } } @Override @@ -196,7 +199,7 @@ public boolean checkpoint(String... splitIds) { @Override public boolean flush() { - String name = getConfigureName(); + String name = getName(); if (StringUtil.isEmpty(name)) { name = getClass().getName(); } @@ -245,12 +248,16 @@ public IMessageCache getMessageCache() { return messageCache; } + public void setMessageCache(IMessageCache messageCache) { + this.messageCache = messageCache; + } + @Override public Map getFinishedQueueIdAndOffsets(CheckPointMessage checkPointMessage) { String pipelineName = null; if (checkPointMessage.getStreamOperator() instanceof IConfigurableIdentification) { IConfigurableIdentification configurable = (IConfigurableIdentification) checkPointMessage.getStreamOperator(); - pipelineName = configurable.getConfigureName(); + pipelineName = configurable.getName(); } SourceState sourceState = this.sourceName2State.get(CheckPointManager.createSourceName(checkPointMessage.getSource(), pipelineName)); if (sourceState != null) { @@ -259,10 +266,6 @@ public Map getFinishedQueueIdAndOffsets(CheckPointMessage return new HashMap<>(); } - public void setMessageCache(IMessageCache messageCache) { - this.messageCache = messageCache; - } - @Override public void atomicSink(ISystemMessage message) { @@ -278,6 +281,19 @@ public boolean isFinished() throws Exception { return false; } + @Override public void destroy() { + super.destroy(); + this.setHasInit(false); + } + + public MetaData getMetaData() { + return metaData; + } + + public void setMetaData(MetaData metaData) { + this.metaData = metaData; + } + public int getAutoFlushSize() { return autoFlushSize; } 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 index bb59f505..eb71a5cc 100644 --- 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 @@ -17,19 +17,18 @@ package org.apache.rocketmq.streams.common.channel.sink; import java.util.List; + import org.apache.rocketmq.streams.common.channel.split.ISplit; public abstract class AbstractSupportShuffleSink extends AbstractSink { - protected transient int splitNum;//分片个数 + protected transient int splitNum = 10;//分片个数 //sls对应的project和logstore初始化是否完成标志 protected volatile transient boolean hasCreated = false; /** * 获取sink的主题,在sls中是logStore,RocketMQ是topic - * - * @return */ public abstract String getShuffleTopicFieldName(); @@ -40,25 +39,18 @@ public abstract class AbstractSupportShuffleSink extends AbstractSink { /** * 获取所有的分片 - * - * @return */ public abstract List> getSplitList(); @Override protected boolean initConfigurable() { - boolean success = super.initConfigurable(); - hasCreated = false; - if (this.splitNum > 0) { - checkAndCreateTopic(); - } - return success; + return super.initConfigurable() && checkAndCreateTopic(); } /** * 创建主题,只创建一次 */ - protected void checkAndCreateTopic() { + protected boolean checkAndCreateTopic() { if (!hasCreated) { synchronized (this) { if (!hasCreated) { @@ -67,18 +59,16 @@ protected void checkAndCreateTopic() { } catch (Exception e) { e.printStackTrace(); } - hasCreated = true; } - } } - + return hasCreated; } + public abstract int getSplitNum(); + 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/AbstractUDFSink.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractUDFSink.java index 8aee4622..d657a6e6 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractUDFSink.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractUDFSink.java @@ -24,10 +24,6 @@ import org.apache.rocketmq.streams.common.context.IMessage; public abstract class AbstractUDFSink extends AbstractSink { - @Override protected boolean batchInsert(List messages) { - return batchInsert(messages, this); - } - public static boolean batchInsert(List messages, AbstractUDFSink sink) { if (messages == null) { @@ -69,6 +65,10 @@ public static boolean batchInsert(List messages, AbstractUDFSink sink) return true; } + @Override protected boolean batchInsert(List messages) { + return batchInsert(messages, this); + } + protected abstract void sendMessage2Store(List messageList); protected abstract void sendMessage2Store(ISplit split, List messageList); 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 index fb3d055d..0d85c1a6 100644 --- 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 @@ -23,29 +23,26 @@ 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.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.MessageOffset; import org.apache.rocketmq.streams.common.interfaces.ISystemMessage; -import org.apache.rocketmq.streams.common.topology.builder.IStageBuilder; +import org.apache.rocketmq.streams.common.topology.IStageBuilder; - -public interface ISink extends IConfigurable, IStageBuilder, IMessageFlushCallBack { +public interface ISink extends IConfigurable, IStageBuilder, IMessageFlushCallBack { String TYPE = "sink"; /** - * 根据channel推断 meta,或者不需要meta,如消息对垒 + * 写入缓存,根据channel推断 meta,或者不需要meta,如消息对垒 * * @param message * @return */ - boolean batchAdd(IMessage message, ISplit split); + boolean batchAdd(IMessage message, ISplit split); /** - * 根据channel推断 meta,或者不需要meta,如消息对垒 + * 写入缓存 * - * @param context * @return */ boolean batchAdd(IMessage message); @@ -59,19 +56,19 @@ public interface ISink extends IConfigurable, IStageBuilder, boolean batchSave(List messages); /** - * 刷新某个分片 + * 刷新某个分片,把缓存刷新到存储 * * @return */ boolean flush(Set splitId); - /** - * 刷新某个分片 + * 刷新某个分片,,把缓存刷新到存储 * * @return */ boolean flush(String... splitIds); + /** * 如果支持批量保存,此方法完成数据的全部写入 * @@ -80,27 +77,36 @@ public interface ISink extends IConfigurable, IStageBuilder, boolean flush(); /** - * 如果支持批量保存,此方法完成数据的全部写入 + * 等同与flush * * @return */ boolean checkpoint(Set splitIds); - - /** - * 如果支持批量保存,此方法完成数据的全部写入 + * 等同与flush,如果支持批量保存,此方法完成数据的全部写入 * * @return */ boolean checkpoint(String... splitIds); + /** - * 调用这个方法后,不必调用flush,由框架定时或定批完成刷新 + * 调用这个方法后,开启自动刷新缓存,由框架定时或定批完成刷新 */ void openAutoFlush(); + /** + * 关闭自动刷新 + */ void closeAutoFlush(); + /** + * 获取批次大小 + * + * @return + */ + int getBatchSize(); + /** * 设置缓存大小,超过条数,强制刷新 * @@ -109,15 +115,13 @@ public interface ISink extends IConfigurable, IStageBuilder, void setBatchSize(int batchSize); /** - * 获取批次大小 + * 返回已经刷新到存储的每个分片,最大的offset * + * @param checkPointMessage * @return */ - int getBatchSize(); - Map getFinishedQueueIdAndOffsets(CheckPointMessage checkPointMessage); void atomicSink(ISystemMessage message); - } 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 index 431dec12..7a81856f 100644 --- 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 @@ -16,26 +16,21 @@ */ 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.schedule.IScheduleCondition; - /** * 自动刷新缓存的任务,开始openAutoFlush后,可以由独立线程完成数据的flush,不必显式调用 */ -public class DataSourceAutoFlushTask implements Runnable, IScheduleCondition { - - private static final Log LOG = LogFactory.getLog(DataSourceAutoFlushTask.class); +public class DataSourceAutoFlushTask implements Runnable { - private volatile boolean isAutoFlush = false; private final IMessageCache messageCache; protected transient Long lastUpdateTime; protected volatile int autoFlushSize = 300; protected volatile int autoFlushTimeGap = 1000; + private volatile boolean isAutoFlush = false; public DataSourceAutoFlushTask(boolean isAutoFlush, IMessageCache messageCache) { this.isAutoFlush = isAutoFlush; this.messageCache = messageCache; + lastUpdateTime = System.currentTimeMillis(); } @Override @@ -76,8 +71,4 @@ public void setAutoFlushTimeGap(int autoFlushTimeGap) { this.autoFlushTimeGap = autoFlushTimeGap; } - @Override - public boolean canExecute() { - return messageCache.getMessageCount() >= autoFlushSize || (lastUpdateTime == null || (System.currentTimeMillis() - lastUpdateTime) >= autoFlushTimeGap); - } } 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 index d8f3fa60..10894802 100644 --- 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 @@ -62,16 +62,16 @@ public interface IMessageCache { void closeAutoFlush(); /** - * 设置缓存大小,超过条数,强制刷新 + * 获取批次大小 * - * @param batchSize + * @return */ - void setBatchSize(int batchSize); + int getBatchSize(); /** - * 获取批次大小 + * 设置缓存大小,超过条数,强制刷新 * - * @return + * @param batchSize */ - int getBatchSize(); + void setBatchSize(int batchSize); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/AbstractMultiSplitMessageCache.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/AbstractMultiSplitMessageCache.java index 136ab6d5..75c5633f 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/AbstractMultiSplitMessageCache.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/AbstractMultiSplitMessageCache.java @@ -22,13 +22,11 @@ import java.util.concurrent.ConcurrentHashMap; 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 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 org.apache.rocketmq.streams.common.threadpool.ThreadPoolFactory; import org.apache.rocketmq.streams.common.utils.StringUtil; public abstract class AbstractMultiSplitMessageCache extends MessageCache { @@ -41,7 +39,8 @@ public AbstractMultiSplitMessageCache( super(null); // this.executorService = new ThreadPoolExecutor(10, 10, // 0L, TimeUnit.MILLISECONDS, -// new LinkedBlockingQueue()); +// new LinkedBlockingQueue(), new ThreadPoolFactory.DipperThreadFactory("AbstractMultiSplitMessageCache")); + this.executorService = ThreadPoolFactory.createFixedThreadPool(10, AbstractMultiSplitMessageCache.class.getName() + "-message_cache"); this.flushCallBack = new MessageFlushCallBack(flushCallBack); } @@ -60,7 +59,6 @@ public int addCache(R msg) { if (this.isOpenAutoFlush) { messageCache.openAutoFlush(); } - messageCache.setAutoFlushExecutorService(this.autoFlushExecutorService); MessageCache existMessageCache = queueMessageCaches.putIfAbsent(queueId, messageCache); if (existMessageCache != null) { messageCache = existMessageCache; @@ -78,8 +76,8 @@ public int addCache(R msg) { return size; } - protected MessageCache createMessageCache(){ - return new MessageCache(flushCallBack); + protected MessageCache createMessageCache() { + return new MessageCache(flushCallBack); } protected abstract String createSplitId(R msg); @@ -104,7 +102,7 @@ public int flush(Set splitIds) { } if (splitIds.size() == 1) { String spiltId = splitIds.iterator().next(); - if(StringUtil.isEmpty(spiltId)){ + if (StringUtil.isEmpty(spiltId)) { return 0; } IMessageCache cache = queueMessageCaches.get(spiltId); @@ -117,7 +115,7 @@ public int flush(Set splitIds) { } CountDownLatch countDownLatch = new CountDownLatch(splitIds.size()); for (String splitId : splitIds) { - if(StringUtil.isEmpty(splitId)){ + if (StringUtil.isEmpty(splitId)) { continue; } executorService.execute(new Runnable() { @@ -174,13 +172,13 @@ public void closeAutoFlush() { } @Override - public void setBatchSize(int batchSize) { - this.batchSize = batchSize; + public int getBatchSize() { + return batchSize; } @Override - public int getBatchSize() { - return batchSize; + public void setBatchSize(int batchSize) { + this.batchSize = batchSize; } protected class MessageFlushCallBack implements IMessageFlushCallBack { 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 index dcb7abbb..7792ad79 100644 --- 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 @@ -20,31 +20,42 @@ import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; 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 org.apache.rocketmq.streams.common.schedule.ScheduleManager; -import org.apache.rocketmq.streams.common.schedule.ScheduleTask; +import org.apache.rocketmq.streams.common.threadpool.ScheduleFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * 消息缓存的实现,通过消息队列做本地缓存。目前多是用了这个实现 */ public class MessageCache implements IMessageCache { - + private static final Logger LOGGER = LoggerFactory.getLogger(MessageCache.class); protected IMessageFlushCallBack flushCallBack; - protected volatile AtomicInteger messageCount = new AtomicInteger(0);//缓存中的数据条数 - protected int batchSize = 1000;//最大缓存条数,超过后需要,刷新出去,做内存保护 - protected transient DataSourceAutoFlushTask autoFlushTask;//自动任务刷新,可以均衡实时性和吞吐率 - protected volatile transient ConcurrentLinkedQueue dataQueue = new ConcurrentLinkedQueue<>();//缓存数据的消息队列 + /** + * 当前缓存的消息数量 + */ + protected volatile AtomicInteger messageCount = new AtomicInteger(0); + /** + * 最大缓存条数,超过后需要,刷新出去,做内存保护 + */ + protected int batchSize = 1000; + /** + * 自动任务刷新,可以均衡实时性和吞吐率 + */ + protected transient DataSourceAutoFlushTask autoFlushTask; + /** + * 缓存数据的消息队列 + */ + protected volatile transient ConcurrentLinkedQueue dataQueue = new ConcurrentLinkedQueue<>(); protected AtomicBoolean openAutoFlushLock = new AtomicBoolean(false); protected volatile int autoFlushSize = 300; protected volatile int autoFlushTimeGap = 1000; - protected ExecutorService autoFlushExecutorService; - public MessageCache(IMessageFlushCallBack flushCallBack) { this.flushCallBack = flushCallBack; } @@ -60,6 +71,7 @@ public int addCache(R msg) { offerQueue(msg); int size = messageCount.incrementAndGet(); if (batchSize > 0 && size >= batchSize) { + LOGGER.debug("MessageCache must sync process message, beacuse the queue is full"); flush(); } return size; @@ -67,13 +79,11 @@ public int addCache(R msg) { @Override public void openAutoFlush() { - if (openAutoFlushLock.compareAndSet(false, true)) {//可重入锁 + if (openAutoFlushLock.compareAndSet(false, true)) { autoFlushTask = new DataSourceAutoFlushTask(true, this); autoFlushTask.setAutoFlushSize(this.autoFlushSize); autoFlushTask.setAutoFlushTimeGap(this.autoFlushTimeGap); - ScheduleTask scheduleTask = new ScheduleTask(autoFlushTask, autoFlushTask); - scheduleTask.setExecutorService(this.autoFlushExecutorService); - ScheduleManager.getInstance().regist(scheduleTask); + ScheduleFactory.getInstance().execute(MessageCache.class.getName() + "-message_cache_schedule", autoFlushTask, 0, 100, TimeUnit.MILLISECONDS); } } @@ -82,6 +92,7 @@ public void closeAutoFlush() { if (autoFlushTask != null) { autoFlushTask.setAutoFlush(false); openAutoFlushLock.set(false); + ScheduleFactory.getInstance().cancel(MessageCache.class.getName() + "-message_cache_schedule"); } } @@ -165,11 +176,4 @@ public IMessageFlushCallBack getFlushCallBack() { return flushCallBack; } - public ExecutorService getAutoFlushExecutorService() { - return autoFlushExecutorService; - } - - public void setAutoFlushExecutorService(ExecutorService autoFlushExecutorService) { - this.autoFlushExecutorService = autoFlushExecutorService; - } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractPushSource.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractPushSource.java new file mode 100644 index 00000000..5c2523fd --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractPushSource.java @@ -0,0 +1,27 @@ +/* + * 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 AbstractPushSource extends AbstractSource { + + /** + * 需要提供消息变动的监听,这块需要实现者根据消息队列类型自行实现: + * 1.当新增分片时,调用addNewSplit方法发送系统通知 + * 2.当分片移走时,removeSplit方法,发送系统通知 + */ + protected abstract boolean hasListenerSplitChanged(); +} 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/AbstractSingleSplitSource.java similarity index 69% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractBatchSource.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractSingleSplitSource.java index c50b09bc..e2c4502a 100644 --- 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/AbstractSingleSplitSource.java @@ -17,11 +17,11 @@ package org.apache.rocketmq.streams.common.channel.source; import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; 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 org.apache.rocketmq.streams.common.channel.split.CommonSplit; +import org.apache.rocketmq.streams.common.channel.split.ISplit; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.BatchMessageOffset; import org.apache.rocketmq.streams.common.context.Message; @@ -31,23 +31,7 @@ /** * 用批处理实现数据流 比如通过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; - +public abstract class AbstractSingleSplitSource extends AbstractSource { /** * 模拟offset生成,递增产生 */ @@ -60,34 +44,16 @@ public abstract class AbstractBatchSource extends AbstractSource { private transient BatchMessageOffset progress;//如果需要保存offset,通过这个对象保存 - public AbstractBatchSource() { - setBatchMessage(true); + public AbstractSingleSplitSource() { + } @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); } @@ -113,25 +79,6 @@ public AbstractContext doReceiveMessage(JSONObject message, boolean needFlush) { 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); @@ -153,11 +100,6 @@ public AbstractContext doReceiveMessage(JSONObject message, boolean needSetCheck return executeMessage(msg); } - @Override - protected boolean isNotDataSplit(String queueId) { - return false; - } - /** * 对于批量接入的消息,可以在消息中加入checkpoint,在这批消息执行完成后,flush所有的输出节点,确保消息至少被消费一次 * @@ -184,25 +126,17 @@ public AbstractContext doReceiveMessage(List messages, boolean needS 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(); } + @Override public List> fetchAllSplits() { + ISplit split = new CommonSplit("1"); + List> splits = new ArrayList<>(); + splits.add(split); + return splits; + } + 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 index 320cbd12..6b7746dd 100644 --- 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 @@ -19,15 +19,12 @@ import com.alibaba.fastjson.JSONArray; import com.alibaba.fastjson.JSONObject; import java.io.UnsupportedEncodingException; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; +import java.util.Date; 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 java.util.concurrent.atomic.AtomicBoolean; import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; @@ -36,27 +33,39 @@ import org.apache.rocketmq.streams.common.channel.split.ISplit; import org.apache.rocketmq.streams.common.checkpoint.CheckPointManager; import org.apache.rocketmq.streams.common.checkpoint.CheckPointMessage; +import org.apache.rocketmq.streams.common.checkpoint.CheckPointState; 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.MessageOffset; import org.apache.rocketmq.streams.common.context.UserDefinedMessage; import org.apache.rocketmq.streams.common.interfaces.ILifeCycle; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; import org.apache.rocketmq.streams.common.metadata.MetaData; import org.apache.rocketmq.streams.common.metadata.MetaDataField; +import org.apache.rocketmq.streams.common.model.NameCreator; +import org.apache.rocketmq.streams.common.topology.IWindow; import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; +import org.apache.rocketmq.streams.common.topology.metric.SourceMetric; +import org.apache.rocketmq.streams.common.utils.CollectionUtil; +import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.common.utils.IdUtil; +import org.apache.rocketmq.streams.common.utils.JsonableUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * channel的抽象,实现了消息的封装,发送等核心逻辑 */ public abstract class AbstractSource extends BasedConfigurable implements ISource, ILifeCycle { + protected static final Logger LOGGER = LoggerFactory.getLogger(AbstractSource.class); + public static String CHARSET = "UTF-8"; /** * 输入的消息是否为json @@ -67,21 +76,16 @@ public abstract class AbstractSource extends BasedConfigurable implements ISourc */ protected Boolean msgIsJsonArray = false; - @ENVDependence - protected String groupName; + @ENVDependence protected String groupName; - protected int maxThread = Runtime.getRuntime().availableProcessors(); + protected int maxThread = 2; - @ENVDependence - protected String topic = ""; + @ENVDependence protected String topic = ""; /** * 多长时间做一次checkpoint */ - protected long checkpointTime = 1000 * 60 * 2; - /** - * 是否是批量消息,批量消息会一批做一次checkpoint,比如通过数据库加载的批消息 - */ - protected boolean isBatchMessage = false; + protected long checkpointTime = 1000 * 30; + /** * 每次拉取的最大条数,多用于消息队列 */ @@ -104,115 +108,153 @@ public abstract class AbstractSource extends BasedConfigurable implements ISourc */ protected MetaData metaData; - protected List headerFieldNames; - /** - * if set the value,the data will be shuffled to a new topic + * 需要数据源支持 */ - protected int shuffleConcurrentCount; + protected List headerFieldNames; + + protected String sendLogTimeFieldName;//可以配置日志中,把日志写入消息队列的时间字段,通过这个字段可以判断队列有无延迟 + protected transient Map splitProgressMap = new HashMap<>();//每个分片的执行进度 + protected transient Map split2MaxTime = new HashMap<>();//每个分片收到的最大发送时间 /** * 数据源投递消息的算子,此算子用来接收source的数据,做处理 */ protected volatile transient IStreamOperator receiver; - /** - * 开启mock模式,则会收集mock数据,如果数据源没数据,则会发送mock数据 - */ - protected transient volatile Boolean openMock = false; protected transient AtomicBoolean hasStart = new AtomicBoolean(false); + protected transient Long msgLastReceiverTime = null; + + protected transient SourceMetric sourceMetric = new SourceMetric(); + /** * 做checkpoint的管理 */ protected transient CheckPointManager checkPointManager = null; - @Override - protected boolean initConfigurable() { + public AbstractSource() { + setType(ISource.TYPE); + } + + @Override protected boolean initConfigurable() { hasStart = new AtomicBoolean(false); - openMock = false; checkPointManager = new CheckPointManager(); + sourceMetric.setMetaData(this.metaData); + sourceMetric.setSource(this); + sourceMetric.setMetaData(metaData); return super.initConfigurable(); } - @Override - public boolean start(IStreamOperator receiver) { + @Override public boolean start(IStreamOperator receiver) { this.receiver = receiver; - boolean isStartSucess = true; - if (hasStart.compareAndSet(false, true)) { - isStartSucess = startSource(); + boolean isStartSuccess = false; + try { + if (hasStart.compareAndSet(false, true)) { + isStartSuccess = startSource(); + sourceMetric.setStartTime(System.currentTimeMillis()); + + } + LOGGER.info("[{}][{}] Source_Start_Success", IdUtil.instanceId(), this.getName()); + } catch (Exception e) { + hasStart.set(false); + LOGGER.error("[{}][{}] Source_Start_Error", IdUtil.instanceId(), this.getName(), e); + throw new RuntimeException("Source_Start_Error", e); } - return isStartSucess; + return isStartSuccess; } @Override public void destroy() { - if (hasStart.compareAndSet(true, false)) { - super.destroy(); + try { + hasStart.set(false); + destroySource(); + LOGGER.info("[{}][{}] Source_Stop_Success", IdUtil.instanceId(), this.getName()); + } catch (Exception e) { + hasStart.set(true); + LOGGER.error("[{}][{}] Source_Stop_Error", IdUtil.instanceId(), this.getName(), e); + throw new RuntimeException("Source_Stop_Error", e); } } /** * 启动 source - * - * @return */ protected abstract boolean startSource(); - public AbstractSource() { - setType(ISource.TYPE); - } + protected abstract void destroySource(); /** - * 提供单条消息的处理逻辑,默认不会加入checkpoint + * 处理消息,并且判断是否需要进行加入check point表识别 * * @param message + * @param needFlushState 是否需要处理的算子刷新状态 * @return */ - public AbstractContext doReceiveMessage(JSONObject message, boolean needSetCheckPoint, String queueId, String offset) { - Message msg = createMessage(message, queueId, offset, needSetCheckPoint); - return executeMessage(msg); + public AbstractContext doReceiveMessage(String message, boolean needFlushState, String queueId, String offset) { + return doReceiveMessage(message, needFlushState, queueId, offset, new HashMap<>()); } /** * 处理消息,并且判断是否需要进行加入check point表识别 * * @param message - * @param needSetCheckPoint + * @param needFlushState 是否需要处理的算子刷新状态 * @return */ - public AbstractContext doReceiveMessage(String message, boolean needSetCheckPoint, String queueId, String offset) { + public AbstractContext doReceiveMessage(String message, boolean needFlushState, String queueId, String offset, Map additionValues) { if (this.msgIsJsonArray) { JSONArray jsonArray = JSONObject.parseArray(message); - if (jsonArray == null || jsonArray.size() == 0) { + if (jsonArray == null || jsonArray.isEmpty()) { return null; } - AbstractContext context = 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; + if (CollectionUtil.isNotEmpty(additionValues)) { + msgBody.putAll(additionValues); } + boolean checkpoint = needFlushState && i == jsonArray.size() - 1; 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); + if (CollectionUtil.isNotEmpty(additionValues)) { + jsonObject.putAll(additionValues); + } + return doReceiveMessage(jsonObject, needFlushState, queueId, offset); } } /** - * 发送一个系统消息,执行组件不可见,告诉所有组件刷新存储 + * 提供单条消息的处理逻辑,默认不会加入checkpoint + * + * @param message + * @return + */ + public AbstractContext doReceiveMessage(JSONObject message, boolean needFlushState, String queueId, String offset) { + Message msg = createMessage(message, queueId, offset, needFlushState); + return executeMessage(msg); + } + + /** + * 发送一个系统消息,执行组件不可见,告诉所有组件刷新状态 * * @param queueId */ public void sendCheckpoint(String queueId) { - Set queues = new HashSet<>(); - queues.add(queueId); - sendCheckpoint(queues); + sendCheckpoint(queueId, null); + } + + public void sendCheckpoint(String queueId, MessageOffset offset) { + Set queueIds = new HashSet<>(); + queueIds.add(queueId); + if (offset != null) { + CheckPointState checkPointState = new CheckPointState(); + checkPointState.getQueueIdAndOffset().put(queueId, offset); + sendCheckpoint(queueIds, checkPointState); + } else { + sendCheckpoint(queueIds); + } } /** @@ -220,25 +262,29 @@ public void sendCheckpoint(String queueId) { * * @param queueIds */ - public void sendCheckpoint(Set queueIds) { + public void sendCheckpoint(Set queueIds, CheckPointState... checkPointStates) { 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(); + if (checkPointStates != null) { + for (CheckPointState checkPointState : checkPointStates) { + checkPointMessage.getCheckPointStates().add(checkPointState); + } + } checkPointMessage.setStreamOperator(this.receiver); checkPointMessage.setSource(this); message.setSystemMessage(checkPointMessage); executeMessage(message); - if (checkPointMessage.isValidate() && supportOffsetRest()) { + sourceMetric.setSplitProgresses(this.getSplitProgress()); + if (checkPointMessage.isValidate()) { saveCheckpoint(checkPointMessage); } + LOGGER.info("[{}][{}] Source_Heartbeat_On({})_At({})_LastMsgReceiveTime({})", IdUtil.instanceId(), getName(), String.join(",", queueIds), System.currentTimeMillis(), this.msgLastReceiverTime == null ? "-" : DateUtil.longToString(this.msgLastReceiverTime)); } protected void saveCheckpoint(CheckPointMessage checkPointMessage) { @@ -334,114 +380,118 @@ public JSONObject create(String message) { } + public AbstractContext executeMessage(Message channelMessage) { + AbstractContext context = new Context(channelMessage); + return executeMessage(channelMessage, context); + } + /** * 交给receiver执行后续逻辑 * * @param channelMessage * @return */ - public AbstractContext executeMessage(Message channelMessage) { + public AbstractContext executeMessage(Message channelMessage, AbstractContext context) { if (BatchFinishMessage.isBatchFinishMessage(channelMessage)) { - /** - * 可以通过真实信息发送,消息结束通知 + /* + * 系统消息:告诉系统后面没有消息了,需要尽快完成计算,尤其是窗口,不必等窗口触发,可以立即触发窗口 */ channelMessage.getHeader().setSystemMessage(true); channelMessage.setSystemMessage(new BatchFinishMessage(channelMessage)); } - AbstractContext context = new Context(channelMessage); + if (isSplitInRemoving(channelMessage)) { return context; } - if (!channelMessage.getHeader().isSystemMessage()) { - messageQueueChangedCheck(channelMessage.getHeader()); - } +// if (!channelMessage.getHeader().isSystemMessage()) { +// messageQueueChangedCheck(channelMessage.getHeader()); +// } - boolean needFlush = !channelMessage.getHeader().isSystemMessage() && channelMessage.getHeader().isNeedFlush(); + boolean needFlushState = !channelMessage.getHeader().isSystemMessage() && channelMessage.getHeader().isNeedFlush(); if (receiver != null) { - receiver.doMessage(channelMessage, context); + try { + this.msgLastReceiverTime = System.currentTimeMillis(); + + if (this.sendLogTimeFieldName != null && channelMessage.getMessageBody().containsKey(this.sendLogTimeFieldName)) { + updateSplitProgress(channelMessage.getHeader().getQueueId(), channelMessage.getMessageBody().get(this.sendLogTimeFieldName)); + } + sourceMetric.setMsgReceivTime(System.currentTimeMillis()); + long start = System.currentTimeMillis(); + receiver.doMessage(channelMessage, context); + sourceMetric.endCalculate(start); + } catch (Exception e) { + LOGGER.error("[{}][{}] Source_SendMsg_Error_On({})_Msg({})_ErrorMsg({})", IdUtil.instanceId(), NameCreator.getFirstPrefix(getName(), IWindow.TYPE, ISource.TYPE), this.getClass().getName(), channelMessage.getMessageBody(), e.getMessage(), e); + } + } - if (needFlush) { + if (needFlushState) { sendCheckpoint(channelMessage.getHeader().getQueueId()); } - executeMessageAfterReceiver(channelMessage, context); return context; } - protected boolean isSplitInRemoving(Message channelMessage) { - return this.checkPointManager.isRemovingSplit(channelMessage.getHeader().getQueueId()); - } - - /** - * source 能否自动返现新增的分片,如果不支持,系统将会模拟实现 - * - * @return - */ - public boolean supportNewSplitFind() { - return false; + protected void updateSplitProgress(String splitId, Object sendLogTime) { + if (sendLogTime == null) { + return; + } + Long sendLogTimeLong = convertTime2Long(sendLogTime); + if (sendLogTimeLong == null) { + return; + } + Long maxSendLogTime = this.split2MaxTime.get(splitId); + if (maxSendLogTime == null || sendLogTimeLong > maxSendLogTime) { + this.split2MaxTime.put(splitId, sendLogTimeLong); + this.splitProgressMap.put(splitId, new SplitProgress(splitId, System.currentTimeMillis() - sendLogTimeLong, true)); + } } /** - * 能否发现分片移走了,如果不支持,系统会模拟实现 + * 如果时间是long,直接返回,如果是string,判断是否是标准格式,如果是转换成long返回 * + * @param time * @return */ - public abstract boolean supportRemoveSplitFind(); - - /** - * 是否运行中,在分片发现时,自动设置分片的offset。必须支持supportNewSplitFind - * - * @return - */ - @Deprecated - public boolean supportOffsetRest() { - return false; - } - - /** - * 系统模拟新分片发现,把消息中的分片保存下来,如果第一次收到,认为是新分片 - * - * @param header - */ - protected void messageQueueChangedCheck(MessageHeader header) { - if (supportNewSplitFind() && supportRemoveSplitFind()) { - return; - } - Set queueIds = new HashSet<>(); - String msgQueueId = header.getQueueId(); - if (StringUtil.isNotEmpty(msgQueueId)) { - queueIds.add(msgQueueId); - } - Set checkpointQueueIds = header.getCheckpointQueueIds(); - if (checkpointQueueIds != null) { - queueIds.addAll(checkpointQueueIds); + protected Long convertTime2Long(Object time) { + if (time == null) { + return null; } - Set newQueueIds = new HashSet<>(); - Set 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); + Long timeStamp = null; + String LONG = "^[-\\+]?[\\d]*$"; // 整数 + if (time instanceof Long) { + timeStamp = (Long) time; + } else if (time instanceof String) { + String timeStr = (String) time; + if (StringUtil.matchRegex(timeStr, LONG)) { + timeStamp = Long.valueOf(timeStr); + } else { + Date date = DateUtil.parse(timeStr); + if (date == null) { + LOGGER.warn(timeStr + " can not match date format, expect 2022-09-03 12:12:12"); + return null; } + timeStamp = date.getTime(); } + } else { + LOGGER.warn(time.toString() + " can not match date format, expect 2022-09-03 12:12:12"); + return null; } - if (!supportNewSplitFind()) { - addNewSplit(newQueueIds); + if (timeStamp != null) { + int length = (timeStamp + "").length(); + if (length < 13) { + int x = 1; + for (int i = 0; i < 13 - length; i++) { + x = x * 10; + } + timeStamp = timeStamp * x; + } } - + return timeStamp; } - protected abstract boolean isNotDataSplit(String queueId); + protected boolean isSplitInRemoving(Message channelMessage) { + return this.checkPointManager.isRemovingSplit(channelMessage.getHeader().getQueueId()); + } /** * 当分片被移走前需要做的回调 @@ -457,19 +507,13 @@ public void removeSplit(Set splitIds) { //先保存所有的分片 sendCheckpoint(splitIds); this.checkPointManager.flush(); - synchronized (this) { - for (String splitId : splitIds) { - this.checkPointManager.removeSplit(splitId); - } - + for (String splitId : splitIds) { + this.checkPointManager.removeSplit(splitId); } - } - public List getAllSplits() { - return null; } - public Map> getWorkingSplitsGroupByInstances() { + public Map>> getWorkingSplitsGroupByInstances() { return new HashMap<>(); } @@ -486,7 +530,6 @@ public void addNewSplit(Set splitIds) { 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()); @@ -499,7 +542,7 @@ public void addNewSplit(Set splitIds) { /** * 发送系统消息,执行组件不可见,告诉所有组件刷新存储 * - * @param queueIds + * @param queueIds 队列 */ public void sendRemoveSplitSystemMessage(Set queueIds) { JSONObject msg = new JSONObject(); @@ -508,8 +551,7 @@ public void sendRemoveSplitSystemMessage(Set queueIds) { //message.getHeader().setCheckpointQueueIds(queueIds); message.getHeader().setNeedFlush(true); message.getHeader().setSystemMessage(true); - Set currentSplitIds = new HashSet<>(); - currentSplitIds.addAll(this.checkPointManager.getCurrentSplits()); + Set currentSplitIds = new HashSet<>(this.checkPointManager.getCurrentSplits()); for (String queueId : queueIds) { currentSplitIds.remove(queueId); } @@ -520,37 +562,11 @@ public void sendRemoveSplitSystemMessage(Set queueIds) { 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> it = msg.entrySet().iterator(); - JSONObject newMsg = new JSONObject(); - newMsg.putAll(msg); - while (it.hasNext()) { - Entry 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 + * @param msg 获取的消息 + * @return 返回dipper的message 实例 */ public Message createMessage(JSONObject msg, String queueId, String offset, boolean checkpoint) { Message channelMessage = new Message(msg); @@ -558,6 +574,7 @@ public Message createMessage(JSONObject msg, String queueId, String offset, bool channelMessage.getHeader().setOffset(offset); channelMessage.getHeader().setQueueId(queueId); channelMessage.getHeader().setNeedFlush(checkpoint); + channelMessage.getHeader().setPipelineName(getName()); //消息的header中存储Pipeline的名称 channelMessage.setJsonMessage(isJsonData); return channelMessage; } @@ -565,50 +582,39 @@ public Message createMessage(JSONObject msg, String queueId, String offset, bool /** * 每批次通过加小序号来区分offset的大小 * - * @param offset - * @param i - * @return + * @param offset offset + * @param i 序号 + * @return offset字符串 */ private String createBatchOffset(String offset, int i) { - String index = "" + i; + StringBuilder index = new StringBuilder("" + i); for (int j = index.length(); j < 5; j++) { - index = "0" + index; + index.insert(0, "0"); } return offset + index; } - @Override - public void setMaxFetchLogGroupSize(int size) { - this.maxFetchLogGroupSize = size; - } - - @Override - public AbstractSource createStageChain(PipelineBuilder pipelineBuilder) { + @Override public AbstractSource createStageChain(PipelineBuilder pipelineBuilder) { return this; } - @Override - public void addConfigurables(PipelineBuilder pipelineBuilder) { + @Override public void addConfigurables(PipelineBuilder pipelineBuilder) { pipelineBuilder.addConfigurables(this); } - @Override - public String getGroupName() { + @Override public String getGroupName() { return groupName; } - @Override - public void setGroupName(String groupName) { + @Override public void setGroupName(String groupName) { this.groupName = groupName; } - @Override - public int getMaxThread() { + @Override public int getMaxThread() { return maxThread; } - @Override - public void setMaxThread(int maxThread) { + @Override public void setMaxThread(int maxThread) { this.maxThread = maxThread; } @@ -636,24 +642,20 @@ 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; + @Override public void setMaxFetchLogGroupSize(int size) { + this.maxFetchLogGroupSize = size; } - public void setTopic(String topic) { - this.topic = topic; + @Override public String getTopic() { + return topic; } - public void setCheckpointTime(long checkpointTime) { - this.checkpointTime = checkpointTime; + @Override public void setTopic(String topic) { + this.topic = topic; } public List getLogFingerprintFields() { @@ -664,22 +666,20 @@ public void setLogFingerprintFields(List logFingerprintFields) { this.logFingerprintFields = logFingerprintFields; } - @Override - public long getCheckpointTime() { + @Override public long getCheckpointTime() { return checkpointTime; } - public boolean isBatchMessage() { - return isBatchMessage; + public void setCheckpointTime(long checkpointTime) { + this.checkpointTime = checkpointTime; } - @Override - public String createCheckPointName() { + @Override public String createCheckPointName() { ISource source = this; String namespace = source.getNameSpace(); - String name = source.getConfigureName(); + String name = source.getName(); String groupName = source.getGroupName(); if (StringUtil.isEmpty(namespace)) { @@ -694,28 +694,22 @@ public String createCheckPointName() { groupName = "default_groupName"; } String topic = source.getTopic(); - if (topic == null || topic.trim().length() == 0) { + if (topic == null || topic.trim().isEmpty()) { topic = "default_topic"; } return MapKeyUtil.createKey(namespace, groupName, topic, name); } - public int getShuffleConcurrentCount() { - return shuffleConcurrentCount; - } - - public void setShuffleConcurrentCount(int shuffleConcurrentCount) { - this.shuffleConcurrentCount = shuffleConcurrentCount; + @Override public Map getSplitProgress() { + return this.splitProgressMap; } - @Override - public boolean isFinished() { + @Override public boolean isFinished() { return false; } - @Override - public void finish() { + @Override public void finish() { checkPointManager.finish(); } @@ -751,5 +745,16 @@ public void setHeaderFieldNames(List headerFieldNames) { this.headerFieldNames = headerFieldNames; } + public String getSendLogTimeFieldName() { + return sendLogTimeFieldName; + } + + public void setSendLogTimeFieldName(String sendLogTimeFieldName) { + this.sendLogTimeFieldName = sendLogTimeFieldName; + } + + public SourceMetric getSourceMetric() { + return sourceMetric; + } } 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 index 4eab84f6..08024e52 100644 --- 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 @@ -25,39 +25,31 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -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 org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * 不可靠的消息源,如http,syslog,可以继承这个类。做了系统保护,如果消息发送太快,可能会出现丢失。 */ -public abstract class AbstractUnreliableSource extends AbstractBatchSource { - private static final Log LOG = LogFactory.getLog(AbstractUnreliableSource.class); +public abstract class AbstractUnreliableSource extends AbstractSingleSplitSource { + private static final Logger LOG = LoggerFactory.getLogger(AbstractUnreliableSource.class); protected Boolean enableAsyncReceive = false; protected boolean isSingleType = false;//是否只有单个生产者,如果是,则为true - + protected transient boolean discard = false;//如果过快,直接丢弃。只有enableAsyncReceive生效时使用 private transient ExecutorService cachedThreadPool = null; private transient int bufferSize = 1024; private transient Disruptor disruptor; private transient DisruptorProducer disruptorProducer; private transient BufferFullFunction bufferFullFunction; - protected transient boolean discard = false;//如果过快,直接丢弃。只有enableAsyncReceive生效时使用 private transient EventHandler eventEventHandler; - @Override - protected boolean initConfigurable() { - bufferSize = 1024; - boolean discard = false;//如果过快,直接丢弃。只有enableAsyncReceive生效时使用 - return super.initConfigurable(); - } - public AbstractUnreliableSource() { super(); if (!enableAsyncReceive) { @@ -85,6 +77,13 @@ public void process(Object data) { }; } + @Override + protected boolean initConfigurable() { + bufferSize = 1024; + boolean discard = false;//如果过快,直接丢弃。只有enableAsyncReceive生效时使用 + return super.initConfigurable(); + } + @Override public AbstractContext executeMessage(Message channelMessage) { if (enableAsyncReceive) { @@ -96,21 +95,6 @@ public AbstractContext executeMessage(Message channelMessage) { } - @Override - public boolean supportRemoveSplitFind() { - return false; - } - - @Override - public boolean supportOffsetRest() { - return false; - } - - @Override - protected boolean isNotDataSplit(String queueId) { - return false; - } - /** * 收到消息发送出去,因为是不可靠队列,如果对象不是json,则用UserDefinedMessage处理 * @@ -127,10 +111,6 @@ public AbstractContext doReceiveMessage(JSONObject message) { return doUnreliableReceiveMessage(message); } - @Override - protected void executeMessageAfterReceiver(Message channelMessage, AbstractContext context) { - } - public Boolean getEnableAsyncReceive() { return enableAsyncReceive; } @@ -144,15 +124,6 @@ public AbstractContext executeMessageBySupper(Message msg) { } - protected class MessageEventHandler implements EventHandler { - @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; } @@ -160,4 +131,13 @@ public boolean isSingleType() { public void setSingleType(boolean singleType) { isSingleType = singleType; } + + protected class MessageEventHandler implements EventHandler { + @Override + public void onEvent(DisruptorEvent event, long sequence, boolean endOfBatch) throws Exception { + // LOG.info("get event " + event); + Message msg = (Message) event.getData(); + executeMessageBySupper(msg); + } + } } 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 index 5dd99408..37640f10 100644 --- 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 @@ -16,11 +16,14 @@ */ package org.apache.rocketmq.streams.common.channel.source; +import java.util.List; +import java.util.Map; +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; +import org.apache.rocketmq.streams.common.topology.IStageBuilder; -public interface ISource extends IConfigurable, IStageBuilder { +public interface ISource extends IConfigurable, IStageBuilder { String TYPE = "source"; /** @@ -29,7 +32,7 @@ public interface ISource extends IConfigurable, IStageBuilder * @param receiver 处理流数据 * @return 是否正常启动 */ - boolean start(IStreamOperator receiver); + boolean start(IStreamOperator receiver); /** * 同一个group name共同消费一份数据,主要针对消息队列,如果实现的子类用到这个字段,需要保持语义 @@ -67,28 +70,41 @@ public interface ISource extends IConfigurable, IStageBuilder void setMaxFetchLogGroupSize(int size); /** - * 消息超过多长时间,会被checkpoint一次,对于批量消息无效 + * 消息超过多长时间,会被checkpoint一次,会强制刷新各个算子的状态 * * @return checkpoint时间 */ long getCheckpointTime(); /** - * - * @param topic + * @return topic */ - void setTopic(String topic); + String getTopic(); /** - * - * @return + * @param topic topic */ - String getTopic(); + void setTopic(String topic); /** - * 创建checkpoint名字 + * 创建checkpoint名字,主要用于保存状态 + * * @return checkpoint key name */ String createCheckPointName(); + /** + * get all split for the source + * + * @return all splits + */ + List> fetchAllSplits(); + + /** + * 获取数据源的消费进度 + * + * @return split progress + */ + Map getSplitProgress(); + } 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 index 05e5e2e3..559e82c6 100644 --- 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 @@ -16,25 +16,9 @@ */ 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.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/SplitProgress.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/SplitProgress.java new file mode 100644 index 00000000..b9a48d39 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/SplitProgress.java @@ -0,0 +1,68 @@ +/* + * 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 class SplitProgress { + protected String splitId; + protected Long progress; + protected boolean isTimeStamp; + protected boolean isLogCount; + + public SplitProgress(String splitId, int progressInt, boolean isTimeStamp) { + this(splitId, (progressInt < 0 ? 0 : Long.valueOf(progressInt)), isTimeStamp); + } + + public SplitProgress(String splitId, long progress, boolean isTimeStamp) { + this.splitId = splitId; + if (isTimeStamp) { + this.isTimeStamp = true; + this.isLogCount = false; + } else { + this.isTimeStamp = false; + this.isLogCount = true; + } + this.progress = progress; + } + + public Long getProgress() { + return progress; + } + + public void setProgress(Long progress) { + this.progress = progress; + } + + public boolean isTimeStamp() { + return isTimeStamp; + } + + public void setTimeStamp(boolean timeStamp) { + isTimeStamp = timeStamp; + } + + public boolean isLogCount() { + return isLogCount; + } + + public void setLogCount(boolean logCount) { + isLogCount = logCount; + } + + public String getSplitId() { + return splitId; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/systemmsg/ChangeTableNameMessage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/systemmsg/ChangeTableNameMessage.java index 95cfba8f..45c8a335 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/systemmsg/ChangeTableNameMessage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/systemmsg/ChangeTableNameMessage.java @@ -25,11 +25,11 @@ public class ChangeTableNameMessage implements ISystemMessage { String scheduleCycle; - public ChangeTableNameMessage(){ + public ChangeTableNameMessage() { } - public ChangeTableNameMessage(String scheduleCycle){ + public ChangeTableNameMessage(String scheduleCycle) { this.scheduleCycle = scheduleCycle; } @@ -40,4 +40,8 @@ public String getScheduleCycle() { public void setScheduleCycle(String scheduleCycle) { this.scheduleCycle = scheduleCycle; } + + @Override public int getSystemMessageType() { + return ISystemMessage.CHANGE_TABLE_NAME; + } } 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 index a3c6bc5b..28b3bc47 100644 --- 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 @@ -17,10 +17,18 @@ package org.apache.rocketmq.streams.common.channel.source.systemmsg; import java.util.Set; +import org.apache.rocketmq.streams.common.interfaces.ISystemMessage; +/** + * 系统消息,当数据源分片增加时发送消息,主要应用在窗口计算,提前加载窗口实例数据,加载新分片状态,不会跨shuffle + */ public class NewSplitMessage extends SplitChangedMessage { public NewSplitMessage(Set splitIds, Set currentSplitIds) { super(splitIds, currentSplitIds, false); } + + @Override public int getSystemMessageType() { + return ISystemMessage.SPLIT_ADD; + } } 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 index 59540bdd..37f7d694 100644 --- 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 @@ -17,10 +17,18 @@ package org.apache.rocketmq.streams.common.channel.source.systemmsg; import java.util.Set; +import org.apache.rocketmq.streams.common.interfaces.ISystemMessage; +/** + * 系统消息,当数据源分片减少时发送消息,主要应用在窗口计算,窗口清理这个分片的状态,加载新分片状态,不会跨shuffle + */ public class RemoveSplitMessage extends SplitChangedMessage { public RemoveSplitMessage(Set splitIds, Set currentSplitIds) { super(splitIds, currentSplitIds, true); } + + @Override public int getSystemMessageType() { + return ISystemMessage.SPLIT_REMOVE; + } } 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 index 667634f9..51803bc8 100644 --- 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 @@ -21,7 +21,10 @@ import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; import org.apache.rocketmq.streams.common.interfaces.ISystemMessage; -public class SplitChangedMessage implements ISystemMessage { +/** + * 系统消息,当数据源分片发生变化时发送消息,主要应用在窗口计算,提前加载窗口实例数据,加载新分片状态,不会跨shuffle + */ +public abstract class SplitChangedMessage implements ISystemMessage { protected Set splitIds; protected boolean needFlush;//需要同步刷新 protected ISource source;//数据源对象 diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/systemmsg/WaterMarkNotifyMessage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/systemmsg/WaterMarkNotifyMessage.java new file mode 100644 index 00000000..a590dd59 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/systemmsg/WaterMarkNotifyMessage.java @@ -0,0 +1,35 @@ +/* + * 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 org.apache.rocketmq.streams.common.interfaces.ISystemMessage; + +public class WaterMarkNotifyMessage implements ISystemMessage { + protected String sourceSplitId; + + public String getSourceSplitId() { + return sourceSplitId; + } + + public void setSourceSplitId(String sourceSplitId) { + this.sourceSplitId = sourceSplitId; + } + + @Override public int getSystemMessageType() { + return ISystemMessage.WATER_MARK; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/split/CommonSplit.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/split/CommonSplit.java new file mode 100644 index 00000000..50ea2ed1 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/split/CommonSplit.java @@ -0,0 +1,42 @@ +/* + * 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.configurable.BasedConfigurable; + +public class CommonSplit extends BasedConfigurable implements ISplit { + protected String splitId; + + public CommonSplit() { + } + + public CommonSplit(String splitId) { + this.splitId = splitId; + } + + @Override public String getQueueId() { + return splitId; + } + + @Override public String getQueue() { + return splitId; + } + + @Override public int compareTo(CommonSplit o) { + return splitId.compareTo(o.splitId); + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/split/DynamicSplit.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/split/DynamicSplit.java new file mode 100644 index 00000000..64b617fd --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/split/DynamicSplit.java @@ -0,0 +1,35 @@ +/* + * 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.configurable.BasedConfigurable; +import org.apache.rocketmq.streams.common.utils.RuntimeUtil; + +public class DynamicSplit extends BasedConfigurable implements ISplit { + @Override public String getQueueId() { + return RuntimeUtil.getDipperInstanceId(); + } + + @Override public String getQueue() { + return RuntimeUtil.getDipperInstanceId(); + } + + @Override public int compareTo(DynamicSplit o) { + return getQueueId().compareTo(o.getQueueId()); + } + +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/AbstractCheckPointStorage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/AbstractCheckPointStorage.java index 62d72c5e..21d0efb0 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/AbstractCheckPointStorage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/AbstractCheckPointStorage.java @@ -19,45 +19,41 @@ import java.util.ArrayList; import java.util.Date; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; -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.IMessageFlushCallBack; import org.apache.rocketmq.streams.common.channel.sinkcache.impl.MessageCache; import org.apache.rocketmq.streams.common.context.MessageOffset; +import org.apache.rocketmq.streams.common.utils.IdUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * @description */ public abstract class AbstractCheckPointStorage implements ICheckPointStorage { - static final Log logger = LogFactory.getLog(AbstractCheckPointStorage.class); + static final Logger LOGGER = LoggerFactory.getLogger(AbstractCheckPointStorage.class); protected transient IMessageCache messageCache; public AbstractCheckPointStorage() { - messageCache = new MessageCache<>(new IMessageFlushCallBack() { - @Override - public boolean flushMessage(List messages) { - //合并最近的checkpoint,只存储一次 - // key 为 sourceName, Value中的map, k : v = queueid : offset - Map sourceStateMap = mergeSourceState(messages); - - logger.info(String.format("flushMessage raw size %d, merge size %d", messages.size(), sourceStateMap.size())); - logger.info("flushMessage : " + messages.get(0).getCheckPointStates().get(0).getQueueIdAndOffset().toString()); - - saveCheckPoint(sourceStateMap); - return true; - } + messageCache = new MessageCache<>(messages -> { + //合并最近的checkpoint,只存储一次 + // key 为 sourceName, Value中的map, k : v = queueid : offset + Map sourceStateMap = mergeSourceState(messages); + + // LOGGER.info("[{}] flushMessage raw size {}, merge size {}", IdUtil.instanceId(), messages.size(), sourceStateMap.size()); + // LOGGER.info("[{}] flushMessage : {}", IdUtil.instanceId(), messages.get(0).getCheckPointStates().get(0).getQueueIdAndOffset().toString()); + + saveCheckPoint(sourceStateMap); + return true; }); - ((MessageCache) messageCache).setAutoFlushSize(50); - ((MessageCache) messageCache).setAutoFlushTimeGap(10 * 1000); + ((MessageCache) messageCache).setAutoFlushSize(50); + ((MessageCache) messageCache).setAutoFlushTimeGap(10 * 1000); messageCache.openAutoFlush(); } - public void flush() { + @Override public void flush() { messageCache.flush(); } @@ -93,17 +89,14 @@ protected Map mergeSourceState(List mess * @return */ protected SourceState merge(SourceState sourceState, SourceState existSourceState) { - Iterator> it = sourceState.getQueueId2Offsets().entrySet() - .iterator(); - while (it.hasNext()) { - Map.Entry entry = it.next(); + for (Map.Entry entry : sourceState.getQueueId2Offsets().entrySet()) { String queueId = entry.getKey(); MessageOffset offset = entry.getValue(); MessageOffset existOffset = existSourceState.getQueueId2Offsets().get(queueId); if (existOffset == null) { existSourceState.getQueueId2Offsets().put(queueId, offset); } else { - boolean isGreateThan = offset.greateThan(existOffset.getOffsetStr()); + boolean isGreateThan = offset.greaterThan(existOffset.getOffsetStr()); if (isGreateThan) { existSourceState.getQueueId2Offsets().put(queueId, offset); } @@ -134,18 +127,14 @@ protected SourceState createSourceState(CheckPointMessage checkPointMessage) { if (checkPointState.isReplyRefuse()) { return null; } - Iterator> it = checkPointState.getQueueIdAndOffset().entrySet() - .iterator(); - while (it.hasNext()) { - Map.Entry entry = it.next(); + for (Map.Entry 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) { + if (existOffset.greaterThan(offset.getOffsetStr())) { queueId2Offsets.put(queueId, offset); } else { queueId2Offsets.put(queueId, existOffset); @@ -168,7 +157,7 @@ protected void saveCheckPoint(Map sourceStateMap) { for (SourceState sourceState : sourceStateMap.values()) { for (Map.Entry entry : sourceState.getQueueId2Offsets().entrySet()) { - CheckPoint checkPoint = new CheckPoint(); + CheckPoint checkPoint = new CheckPoint<>(); checkPoint.setSourceName(sourceState.getSourceName()); checkPoint.setQueueId(entry.getKey()); checkPoint.setData(entry.getValue().getMainOffset()); @@ -182,16 +171,15 @@ protected void saveCheckPoint(Map sourceStateMap) { save(checkPoints); } - public void addCheckPointMessage(CheckPointMessage message) { + @Override public void addCheckPointMessage(CheckPointMessage message) { List states = message.getCheckPointStates(); for (CheckPointState state : states) { - logger.debug(String.format("addCheckPointMessage states %s", state.getQueueIdAndOffset().toString())); + LOGGER.debug("[{}] addCheckPointMessage states {}", IdUtil.instanceId(), state.getQueueIdAndOffset().toString()); } messageCache.addCache(message); } - @Override - public void finish() { + @Override public void finish() { this.messageCache.closeAutoFlush(); } 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 index f97f7fdc..fa144b0a 100644 --- 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 @@ -18,11 +18,12 @@ import com.alibaba.fastjson.JSONObject; import org.apache.rocketmq.streams.common.model.Entity; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; /** - * model for checkpoint,need save in store + * types for checkpoint,need save in store */ -public class CheckPoint extends Entity { +public class CheckPoint extends Entity implements ISplitOffset { protected String sourceNamespace; protected String pipelineName; @@ -34,6 +35,11 @@ public class CheckPoint extends Entity { protected JSONObject content; + @Override public String getName() { + return MapKeyUtil.createKey(sourceNamespace, sourceName); + } + + @Override public String getQueueId() { return queueId; } @@ -42,6 +48,10 @@ public void setQueueId(String queueId) { this.queueId = queueId; } + @Override public String getOffset() { + return data.toString(); + } + public String getSourceNamespace() { return sourceNamespace; } 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 index bf588dc5..0e3f15a0 100644 --- 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 @@ -22,16 +22,18 @@ import java.util.List; import java.util.Map; import java.util.Set; +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.component.ComponentCreator; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; import org.apache.rocketmq.streams.common.context.MessageOffset; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.common.utils.ReflectUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; +/** + * 管理分片和checkpoint + */ public class CheckPointManager extends BasedConfigurable { protected transient Map currentSplitAndLastUpdateTime = new HashMap<>();//保存这个实例处理的分片数 @@ -39,8 +41,62 @@ public class CheckPointManager extends BasedConfigurable { protected transient ICheckPointStorage iCheckPointStorage; public CheckPointManager() { - String name = ComponentCreator.getProperties().getProperty(ConfigureFileKey.CHECKPOINT_STORAGE_NAME); - iCheckPointStorage = CheckPointStorageFactory.getInstance().getStorage(name); + } + + /** + * 根据source进行划分,主要是针对双流join的场景 + * + * @param source + * @return + */ + public static String createSourceName(ISource source, String pipelineName) { + + if (StringUtil.isNotEmpty(pipelineName)) { + return MapKeyUtil.createKey(source.createCheckPointName(), pipelineName); + } + if (source == null) { + return null; + } + return source.createCheckPointName(); + } + + public static final String createCheckPointKey(String key, String queueId) { + return key + "^^^" + queueId; + } + + public static final String[] parseCheckPointKey(String checkPointKey) { + return checkPointKey.split("\\^\\^\\^"); + } + + public static final String getNameSpaceFromCheckPointKey(String checkPointKey) { + return parseCheckPointKey(checkPointKey)[0].split("\\;")[0]; + } + + public static final String getGroupNameFromCheckPointKey(String checkPointKey) { + return parseCheckPointKey(checkPointKey)[0].split("\\;")[1]; + } + + public static final String getNameFromCheckPointKey(String checkPointKey) { + return parseCheckPointKey(checkPointKey)[0].split("\\;")[2]; + } + + public static final String getTopicFromCheckPointKey(String checkPointKey) { + return parseCheckPointKey(checkPointKey)[0].split("\\;")[3]; + } + + public static final String getQueueIdFromCheckPointKey(String checkPointKey) { + return parseCheckPointKey(checkPointKey)[1]; + } + + public static void main(String[] args) { + SourceSnapShot snapShot = new SourceSnapShot(); + snapShot.setId(1L); + snapShot.setGmtCreate(new Date()); + snapShot.setGmtModified(new Date()); + snapShot.setKey("key"); + snapShot.setValue("value"); + System.out.println(ReflectUtil.serializeObject(snapShot)); + } public synchronized void addSplit(String splitId) { @@ -55,9 +111,9 @@ public boolean contains(String splitId) { return this.currentSplitAndLastUpdateTime.containsKey(splitId); } - private final List fromSourceState(Map sourceStateMap) { + private final List> fromSourceState(Map sourceStateMap) { - List checkPoints = new ArrayList<>(); + List> checkPoints = new ArrayList<>(); for (Map.Entry entry : sourceStateMap.entrySet()) { String key = entry.getKey(); SourceState value = entry.getValue(); @@ -66,7 +122,7 @@ private final List fromSourceState(Map sourceSt for (Map.Entry tmpEntry : value.getQueueId2Offsets().entrySet()) { String queueId = tmpEntry.getKey(); String offset = tmpEntry.getValue().getMainOffset(); - CheckPoint checkPoint = new CheckPoint(); + CheckPoint checkPoint = new CheckPoint<>(); checkPoint.setSourceNamespace(ss[0]); checkPoint.setPipelineName(ss[1]); checkPoint.setSourceName(ss[2]); @@ -87,12 +143,11 @@ public void addCheckPointMessage(CheckPointMessage message) { } - public CheckPoint recover(ISource iSource, ISplit iSplit) { + public ISplitOffset recover(ISource iSource, ISplit iSplit) { if (this.iCheckPointStorage == null) { return null; } - String isRecover = ComponentCreator.getProperties().getProperty(ConfigureFileKey.IS_RECOVER_MODE); - if (isRecover != null && Boolean.valueOf(isRecover)) { + if (iSource instanceof AbstractSource) { String queueId = iSplit.getQueueId(); return iCheckPointStorage.recover(iSource, queueId); } @@ -122,23 +177,6 @@ public void finish() { } - /** - * 根据source进行划分,主要是针对双流join的场景 - * - * @param source - * @return - */ - public static String createSourceName(ISource source, String pipelineName) { - - if (StringUtil.isNotEmpty(pipelineName)) { - return MapKeyUtil.createKey(source.createCheckPointName(), pipelineName); - } - if (source == null) { - return null; - } - return source.createCheckPointName(); - } - public Map getCurrentSplitAndLastUpdateTime() { return currentSplitAndLastUpdateTime; } @@ -170,42 +208,11 @@ public synchronized boolean isRemovingSplit(String splitId) { return true; } - public static final String createCheckPointKey(String key, String queueId) { - return key + "^^^" + queueId; + public ICheckPointStorage getiCheckPointStorage() { + return iCheckPointStorage; } - public static final String[] parseCheckPointKey(String checkPointKey) { - return checkPointKey.split("\\^\\^\\^"); - } - - public static final String getNameSpaceFromCheckPointKey(String checkPointKey) { - return parseCheckPointKey(checkPointKey)[0].split("\\;")[0]; - } - - public static final String getGroupNameFromCheckPointKey(String checkPointKey) { - return parseCheckPointKey(checkPointKey)[0].split("\\;")[1]; - } - - public static final String getNameFromCheckPointKey(String checkPointKey) { - return parseCheckPointKey(checkPointKey)[0].split("\\;")[2]; - } - - public static final String getTopicFromCheckPointKey(String checkPointKey) { - return parseCheckPointKey(checkPointKey)[0].split("\\;")[3]; - } - - public static final String getQueueIdFromCheckPointKey(String checkPointKey) { - return parseCheckPointKey(checkPointKey)[1]; - } - - public static void main(String[] args) { - SourceSnapShot snapShot = new SourceSnapShot(); - snapShot.setId(1L); - snapShot.setGmtCreate(new Date()); - snapShot.setGmtModified(new Date()); - snapShot.setKey("key"); - snapShot.setValue("value"); - System.out.println(ReflectUtil.serializeObject(snapShot)); - + public void setiCheckPointStorage(ICheckPointStorage iCheckPointStorage) { + this.iCheckPointStorage = iCheckPointStorage; } } 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 index ee91d9cf..ed928999 100644 --- 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 @@ -29,7 +29,6 @@ public class CheckPointMessage implements ISystemMessage { protected boolean isValidate = true; protected String pipelineName; - public ISource getSource() { return source; } @@ -78,4 +77,8 @@ public String getPipelineName() { public void setPipelineName(String pipelineName) { this.pipelineName = pipelineName; } + + @Override public int getSystemMessageType() { + return ISystemMessage.CHECK_POINT; + } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/CheckPointStorageFactory.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/CheckPointStorageFactory.java index 91afa326..f32b022b 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/CheckPointStorageFactory.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/CheckPointStorageFactory.java @@ -21,16 +21,16 @@ import java.net.URLClassLoader; import java.util.Iterator; import java.util.ServiceLoader; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * @description */ public class CheckPointStorageFactory { - private static final Log logger = LogFactory.getLog(CheckPointStorageFactory.class); public static final String DEFAULT_CHECKPOINT_TYPE_NAME = "DB"; + private static final Logger logger = LoggerFactory.getLogger(CheckPointStorageFactory.class); private static CheckPointStorageFactory instance; private ServiceLoader loader; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/ICheckPointStorage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/ICheckPointStorage.java index 2be9a5fd..297944ba 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/ICheckPointStorage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/ICheckPointStorage.java @@ -20,20 +20,47 @@ import org.apache.rocketmq.streams.common.channel.source.ISource; /** - * @description 负责checkpoint的保存、恢复 + * @description 负责数据源offset的保存、恢复,一般用在AbstractPullSource上 */ public interface ICheckPointStorage { String TYPE = "checkpoint_storage"; + /** + * 选择 + * + * @return + */ String getStorageName(); - void save(List checkPointState); + /** + * 保存分片和offset到状态存储中 + * + * @param checkPointState + * @param + */ + void save(List checkPointState); - T recover(ISource iSource, String queueID); + /** + * 给数据源恢复状态和存储 + * + * @param iSource + * @param queueId + * @param + * @return + */ + T recover(ISource iSource, String queueId); + /** + * 刷新缓存的状态到存储中 + */ void flush(); + /** + * 把新收集的数据源分片和offset的信息保存到缓存 + * + * @param message + */ void addCheckPointMessage(CheckPointMessage message); void finish(); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/ISplitOffset.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/ISplitOffset.java new file mode 100644 index 00000000..45c7f43d --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/ISplitOffset.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.checkpoint; + +public interface ISplitOffset { + + /** + * need unique in storage, suggest use source namespace+name + * + * @return + */ + String getName(); + + /** + * QUEUE ID + * + * @return + */ + String getQueueId(); + + /** + * OFFSET + * + * @return + */ + String getOffset(); +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/SourceSnapShot.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/SourceSnapShot.java index 0db99f9d..3d7bdfdc 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/SourceSnapShot.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/checkpoint/SourceSnapShot.java @@ -16,6 +16,7 @@ */ package org.apache.rocketmq.streams.common.checkpoint; +import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.JSONObject; import java.io.Serializable; import org.apache.rocketmq.streams.common.metadata.MetaData; @@ -25,10 +26,10 @@ * @create 2021-08-06 16:21:30 * @description */ -public class SourceSnapShot extends Entity implements Serializable { +public class SourceSnapShot extends Entity implements Serializable, ISplitOffset { - private static final long serialVersionUID = 4449170945607357658L; public final static MetaData snapshotTable = new MetaData(); + private static final long serialVersionUID = 4449170945607357658L; static { snapshotTable.setTableName("checkpoint_snapshot"); @@ -72,4 +73,24 @@ public JSONObject toJson() { object.put("value", value); return object; } + + @Override public String getName() { + CheckPoint checkPoint = new CheckPoint(); + checkPoint = checkPoint.fromSnapShot(this); + + return checkPoint.getName(); + } + + @Override public String getQueueId() { + CheckPoint checkPoint = new CheckPoint(); + checkPoint = checkPoint.fromSnapShot(this); + return checkPoint.getQueueId(); + } + + @Override public String getOffset() { + CheckPoint checkPoint = new CheckPoint(); + checkPoint = checkPoint.fromSnapShot(this); + JSONObject jsonObject = JSON.parseObject(checkPoint.getOffset()); + return jsonObject.getString("offset"); + } } 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 index d9d69d4d..4eb55f51 100644 --- 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 @@ -43,6 +43,8 @@ * 编译java源码 */ public class CustomJavaCompiler { + private static Pattern pattern1 = Pattern.compile("package\\s+\\S+\\s*;"); + private static Pattern pattern2 = Pattern.compile("class\\s+\\S+.*\\{"); //类全名 private String fullClassName; private String sourceCode; @@ -59,10 +61,6 @@ public class CustomJavaCompiler { //运行耗时(单位ms) private long runTakeTime; - private static Pattern pattern1 = Pattern.compile("package\\s+\\S+\\s*;"); - - private static Pattern pattern2 = Pattern.compile("class\\s+\\S+.*\\{"); - /** * @param sourceFile java 文件 */ @@ -78,10 +76,34 @@ public CustomJavaCompiler(String sourceCode) { this.fullClassName = getFullClassName(sourceCode); } + /** + * 获取类的全名称 + * + * @param sourceCode 源码 + * @return 类的全名称 + */ + public static String getFullClassName(String sourceCode) { + String className = ""; + Matcher matcher = pattern1.matcher(sourceCode); + if (matcher.find()) { + className = matcher.group().replaceFirst("package", "").replace(";", "").trim() + "."; + } + matcher = pattern2.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; + } + public T compileAndNewInstance() { try { Class aClass = compileClass(); - return (T)aClass.newInstance(); + return (T) aClass.newInstance(); } catch (Exception e) { throw new RuntimeException("compile class error, the class name is " + fullClassName, e); } @@ -149,30 +171,6 @@ public long getRunTakeTime() { return runTakeTime; } - /** - * 获取类的全名称 - * - * @param sourceCode 源码 - * @return 类的全名称 - */ - public static String getFullClassName(String sourceCode) { - String className = ""; - Matcher matcher = pattern1.matcher(sourceCode); - if (matcher.find()) { - className = matcher.group().replaceFirst("package", "").replace(";", "").trim() + "."; - } - matcher = pattern2.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; - } - /** * 自定义一个字符串的源码对象 */ 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 index 8d27f6fa..1d606e2e 100644 --- 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 @@ -20,68 +20,46 @@ import java.util.List; import java.util.Properties; import java.util.concurrent.atomic.AtomicBoolean; -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 org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.utils.ENVUtil; +import org.apache.rocketmq.streams.common.utils.PropertiesUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public abstract class AbstractComponent implements IComponent, 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"; +public abstract class AbstractComponent implements IComponent { + private static final Logger LOG = LoggerFactory.getLogger(AbstractComponent.class); /** * xml的位置,如果没有即默认位置 */ protected PropertyConfigureDescriptorManager configureDescriptorManager = new PropertyConfigureDescriptorManager(); - + protected AtomicBoolean isStart = new AtomicBoolean(false); private Properties properties; - @Override - public boolean init() { + @Override public boolean init() { Properties properties = createDefaultProperty(); return initProperty(properties); } protected Properties createDefaultProperty() { //createENVProperties(); - Properties properties ; - properties = ComponentCreator.getProperties(); - if (properties == null) { - properties = getDefaultProperties(); - } + Properties properties = getDefaultProperties(); if (properties == null) { properties = new Properties(); } - Properties newProperties=new Properties(); + Properties newProperties = new Properties(); newProperties.putAll(properties); addSystemProperties(newProperties); return newProperties; } 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)); + addConfigureDescriptor(new ConfigureDescriptor("jdbc", ConfigurationKey.JDBC_URL, null, true, ConfigurationKey.ENV_JDBC_URL)); + addConfigureDescriptor(new ConfigureDescriptor("jdbc", ConfigurationKey.JDBC_USERNAME, null, true, ConfigurationKey.ENV_JDBC_USERNAME)); + addConfigureDescriptor(new ConfigureDescriptor("jdbc", ConfigurationKey.JDBC_PASSWORD, null, true, ConfigurationKey.ENV_JDBC_PASSWORD)); + addConfigureDescriptor(new ConfigureDescriptor("jdbc", ConfigurationKey.JDBC_DRIVER, ConfigurationKey.DEFAULT_JDBC_DRIVER, false, ConfigurationKey.ENV_JDBC_DRIVER)); + addConfigureDescriptor(new ConfigureDescriptor("http", ConfigurationKey.HTTP_AK, true)); + addConfigureDescriptor(new ConfigureDescriptor("http", ConfigurationKey.HTTP_SK, true)); } protected void addConfigureDescriptor(ConfigureDescriptor configureDescriptor) { @@ -97,7 +75,7 @@ protected void addSystemProperties(Properties properties) { for (List configureDescriptors : configureDescriptorManager.getGroupByConfigures().values()) { for (ConfigureDescriptor configureDescriptor : configureDescriptors) { String key = configureDescriptor.getPropertyKey(); - String value = ENVUtile.getSystemParameter(key); + String value = ENVUtil.getSystemParameter(key); if (value != null) { properties.put(key, value); } @@ -131,24 +109,21 @@ protected Properties createENVProperties() { return null; } - @Override - public boolean initByClassPath(String propertiesPath) { - Properties properties = PropertiesUtils.getResourceProperties(propertiesPath); + @Override public boolean initByClassPath(String propertiesPath) { + Properties properties = PropertiesUtil.getResourceProperties(propertiesPath); return initProperty(properties); } - @Override - public boolean initByFilePath(String filePath) { - Properties properties = PropertiesUtils.loadPropertyByFilePath(filePath); + @Override public boolean initByFilePath(String filePath) { + Properties properties = PropertiesUtil.loadPropertyByFilePath(filePath); return initProperty(properties); } - @Override - public boolean initByPropertiesStr(String... kvs) { + @Override public boolean initByPropertiesStr(String... kvs) { Properties properties = createDefaultProperty(); if (kvs != null && kvs.length > 0) { for (String ky : kvs) { - PropertiesUtils.putProperty(ky, ":", properties); + PropertiesUtil.putProperty(ky, ":", properties); } } return initProperty(properties); @@ -168,7 +143,7 @@ protected boolean initProperty(Properties properties) { protected Properties getDefaultProperties() { try { - return PropertiesUtils.getResourceProperties(getComponentPropertyPath()); + return PropertiesUtil.getResourceProperties(getComponentPropertyPath()); } catch (Exception e) { LOG.error("load jar file error", e); return null; @@ -177,13 +152,10 @@ protected Properties getDefaultProperties() { } private String getComponentPropertyPath() { - return PropertiesUtils.getComponentPropertyPath(this.getClass()); + return PropertiesUtil.getComponentPropertyPath(this.getClass()); } - protected AtomicBoolean isStart = new AtomicBoolean(false); - - @Override - public boolean start(String name) { + @Override public boolean start(String name) { if (isStart.compareAndSet(false, true)) { startComponent(name); } 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 index 3efac351..43f547f7 100644 --- 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 @@ -25,72 +25,49 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Properties; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -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.PropertiesUtil; import org.apache.rocketmq.streams.common.utils.SQLUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; /** * 创建组件,如果参数未发生变化(如果未传入,则是配置文件的参数),返回同一个组件对象,如果发生变化,返回不同的组件对象 */ -@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"; - + 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"; + public static final String UDF_JAR_PATH = "dipper.udf.jar.path"; + public static final String UDF_JAR_OSS_ACCESS_ID = "dipper.udf.jar.oss.access.id"; + public static final String UDF_JAR_OSS_ACCESS_KEY = "dipper.udf.jar.oss.access.key"; private static final Map 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); + Properties tmpProperties = PropertiesUtil.getResourceProperties("dipper.properties"); + if (tmpProperties == null) { + tmpProperties = new Properties(); } + ComponentCreator.properties = tmpProperties; } 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 void updateProperties(Properties properties) { - ComponentCreator.properties.putAll(properties); - } - public static String[] createKV(Properties properties) { List keys = new ArrayList<>(); - Iterator keyIterator = properties.keySet().iterator(); - while (keyIterator.hasNext()) { - keys.add(keyIterator.next().toString()); + for (Object o : properties.keySet()) { + keys.add(o.toString()); } Collections.sort(keys); Iterator it = keys.iterator(); @@ -105,27 +82,22 @@ public static String[] createKV(Properties properties) { return kvs; } - public static void createMemoryProperties(Long pollingTime) { - Properties properties = new Properties(); - properties.put(AbstractComponent.CONNECT_TYPE, IConfigurableService.MEMORY_SERVICE_NAME); - properties.put(AbstractComponent.POLLING_TIME, pollingTime + ""); - ComponentCreator.properties = loadOtherProperty(properties); - } - + @Deprecated(since = "该方法直接改动了系统配置, 后续会下线") public static void createProperties(Properties properties) { ComponentCreator.properties = loadOtherProperty(properties); } + @Deprecated(since = "该方法直接改动了系统配置, 后续会下线") public static void createProperties(String propertiesFilePath) { - Properties properties = PropertiesUtils.getResourceProperties(propertiesFilePath); + Properties properties = PropertiesUtil.getResourceProperties(propertiesFilePath); if (properties == null) { - properties = PropertiesUtils.loadPropertyByFilePath(propertiesFilePath); + properties = PropertiesUtil.loadPropertyByFilePath(propertiesFilePath); } ComponentCreator.properties = loadOtherProperty(properties); } private static Properties loadOtherProperty(Properties tmp, String... kvs) { - Properties properties = new Properties(); + Properties tmpProperties = new Properties(); for (Entry entry : tmp.entrySet()) { String key = (String) entry.getKey(); String value = (String) entry.getValue(); @@ -137,26 +109,65 @@ private static Properties loadOtherProperty(Properties tmp, String... kvs) { } } - properties.put(key, realValue); + tmpProperties.put(key, realValue); } - if (kvs == null || kvs.length == 0) { - return properties; + if (kvs == null) { + return tmpProperties; } for (String kv : kvs) { int startIndex = kv.indexOf(":"); String key = kv.substring(0, startIndex); String value = kv.substring(startIndex + 1); - properties.put(key, value); + tmpProperties.put(key, value); } - return properties; + tmpProperties.putAll(properties); + return tmpProperties; + } + + private static Properties loadOtherProperty(Properties tmp, Properties tmp1) { + Properties tmpProperties = new Properties(); + for (Entry 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("'", ""); + } + } + tmpProperties.put(key, realValue); + } + if (tmp1 == null) { + return tmpProperties; + } + for (Entry entry : tmp1.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("'", ""); + } + } + tmpProperties.put(key, realValue); + } + tmpProperties.putAll(properties); + return tmpProperties; + } + + public static T getComponent(String namespace, Class componentType, Properties properties) { + Properties newProperties = loadOtherProperty(ComponentCreator.properties, properties); + String[] kvArray = createKV(newProperties); + return (T) getComponentInner(namespace, componentType, true, kvArray); } public static T getComponent(String namespace, Class componentType, String... kvs) { return getComponent(namespace, componentType, true, kvs); } - protected static T getComponent(String namespace, Class componentType, boolean isStart, - String... kvs) { + protected static 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); @@ -170,18 +181,21 @@ public static T getComponentNotStart(String namespace, Cl return (T) getComponentInner(namespace, componentType, false, createKV(properties)); } + public static T getComponentNotStart(String namespace, Class componentType, Properties properties) { + Properties newProperties = loadOtherProperty(ComponentCreator.properties, properties); + String[] kvArray = createKV(newProperties); + return (T) getComponentInner(namespace, componentType, false, kvArray); + } + public static T getComponentNotStart(String namespace, Class componentType, String... kvs) { return getComponent(namespace, componentType, false, kvs); } - @Deprecated - public static T getComponentUsingPropertiesFile(String namespace, Class componentType, - String propertiesPath) { + @Deprecated public static T getComponentUsingPropertiesFile(String namespace, Class componentType, String propertiesPath) { return (T) getComponentInner(namespace, componentType, true, propertiesPath); } - private static IComponent getComponentInner(String namespace, Class componentType, boolean needStart, - Object o) { + private static IComponent getComponentInner(String namespace, Class componentType, boolean needStart, Object o) { String key = createKey(componentType, namespace, o); if (key2Component.containsKey(key) && key2Component.get(key) != null) { return key2Component.get(key); @@ -200,9 +214,7 @@ private static IComponent getComponentInner(String namespace, Class 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); + throw new RuntimeException("can not get component. namespace is " + namespace + ", type is " + componentType.getName(), e); } } @@ -235,7 +247,7 @@ private static void initComponent(IComponent component, Object o) { component.init(); } else if (o instanceof String) { String propertiesPath = (String) o; - URL url = PropertiesUtils.class.getClassLoader().getResource(propertiesPath); + URL url = PropertiesUtil.class.getClassLoader().getResource(propertiesPath); if (url != null) { component.initByClassPath(propertiesPath); } else { @@ -247,13 +259,21 @@ private static void initComponent(IComponent component, Object o) { return; } - public static void createProperty(String outputFilePath) { - Properties properties = ComponentCreator.getProperties(); - PropertiesUtils.flush(properties, outputFilePath); - } +// public static void createProperty(String outputFilePath) { +// Properties properties = ComponentCreator.getProperties(); +// PropertiesUtil.flush(properties, outputFilePath); +// } + +// public static Properties getPropertiesDirectly() { +// return properties; +// } +// +// public static Properties getProperties() { +// return SQLParseContext.getProperties(); +// } - public static Properties getProperties() { - return properties; + public static void setProperties(String propertiesPath) { + createProperties(propertiesPath); } public static boolean getPropertyBooleanValue(String key) { @@ -261,9 +281,6 @@ public static boolean getPropertyBooleanValue(String key) { if (value == null) { return false; } - if ("true".equals(value)) { - return true; - } - return false; + return "true".equals(value); } } 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 index ab5a336e..6535438d 100644 --- 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 @@ -31,7 +31,7 @@ public class ConfigureDescriptor { private String envPropertyKey; public ConfigureDescriptor(String groupName, String propertyKey, String defaultValue, boolean requiredSet, - String envPropertyKey) { + String envPropertyKey) { this(groupName, propertyKey, defaultValue, requiredSet); this.envPropertyKey = envPropertyKey; } 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 index 83bd81f2..6290a8ac 100644 --- 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 @@ -22,14 +22,14 @@ import java.util.List; import java.util.Map; import java.util.Properties; -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.ENVUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class PropertyConfigureDescriptorManager { - private static final Log LOG = LogFactory.getLog(PropertyConfigureDescriptorManager.class); + private static final Logger LOG = LoggerFactory.getLogger(PropertyConfigureDescriptorManager.class); protected transient Map> groupByConfigures = new HashMap<>(); @@ -74,7 +74,7 @@ public Properties createENVProperties(List configureDescrip Properties properties = new Properties(); for (ConfigureDescriptor configureDescriptor : configureDescriptorList) { String key = configureDescriptor.getEnvPropertyKey(); - String value = ENVUtile.getENVParameter(key); + String value = ENVUtil.getENVParameter(key); if (configureDescriptor.isRequiredSet() && value == null) { return null; } 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 index b86db07b..7b9dbfaf 100644 --- 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 @@ -23,14 +23,14 @@ import java.util.List; import java.util.Map; import java.util.Properties; -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.ENVUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class PropertyConfigureDiscriptorManager { - private static final Log LOG = LogFactory.getLog(PropertyConfigureDiscriptorManager.class); + private static final Logger LOG = LoggerFactory.getLogger(PropertyConfigureDiscriptorManager.class); protected transient Map> groupbyConfigures = new HashMap<>(); @@ -79,7 +79,7 @@ public Properties createENVProperties(List configureDiscrip Properties properties = new Properties(); for (ConfigureDescriptor configureDiscriptor : configureDiscriptorList) { String key = configureDiscriptor.getEnvPropertyKey(); - String value = ENVUtile.getENVParameter(key); + String value = ENVUtil.getENVParameter(key); // LOG.info("@@@envkey:" + key + ",envValue:" + value); if (configureDiscriptor.isRequiredSet() && value == null) { return null; 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 index bfd76008..51b84eeb 100644 --- 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 @@ -16,188 +16,89 @@ */ package org.apache.rocketmq.streams.common.configurable; -import java.util.HashMap; -import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configurable.annotation.Changeable; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; -import org.apache.rocketmq.streams.common.model.Entity; -import org.apache.rocketmq.streams.common.utils.AESUtil; - -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 privateDatas = new HashMap<>(); - - protected transient IConfigurableService configurableService; - - /** - * 是否完成初始化 - */ - private transient volatile boolean hasInit = false; - - /** - * 是否初始化成功 - */ - protected transient boolean initSuccess = true; - - /** - * 是否已经被销毁 - */ - protected transient boolean isDestroy = false; - - /** - * 数据库的状态字段 - */ - private static final String STATUS = "status"; +import java.util.Properties; +import org.apache.rocketmq.streams.common.utils.IdUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; - @Override - public boolean init() { +public abstract class AbstractConfigurable implements IConfigurable { + + private static final Logger LOGGER = LoggerFactory.getLogger(AbstractConfigurable.class); + + protected String jobName; + protected transient Properties configuration = new Properties(); + private String nameSpace; + private String name; + private String type; + private transient boolean hasInit = false; + + @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); + LOGGER.error("[{}][{}] init configurable error", IdUtil.instanceId(), getName(), e); + throw new RuntimeException("init configurable error ", 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"); + @Override public void destroy() { } 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 (configurable instanceof Entity) { - entity = (Entity) configurable; - } else { - entity = new Entity(); - } - int status = 1; - if (configurable.getPrivateData("status") != null) { - status = Integer.parseInt(configurable.getPrivateData("status")); - } - String theSecretValue; - try { - theSecretValue = AESUtil.aesEncrypt(json, ComponentCreator.getProperties().getProperty(ConfigureFileKey.SECRECY, ConfigureFileKey.SECRECY_DEFAULT)); - } 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 getConfigurableService() { - return (T) configurableService; + public boolean isHasInit() { + return hasInit; } - public void setConfigurableService(IConfigurableService configurableService) { - this.configurableService = configurableService; + public void setHasInit(boolean hasInit) { + this.hasInit = hasInit; } - @Override - public void putPrivateData(String key, T value) { - this.privateDatas.put(key, value); + @Override public String getJobName() { + return jobName; } - @Override - public T getPrivateData(String key) { - return (T) this.privateDatas.get(key); + @Override public void setJobName(String jobName) { + this.jobName = jobName; } - public T removePrivateData(String key) { - return (T) this.privateDatas.remove(key); + @Override public String getNameSpace() { + return nameSpace; } - @Override - public Map getPrivateData() { - return this.privateDatas; + @Override public void setNameSpace(String nameSpace) { + this.nameSpace = nameSpace; } - public void setPrivateDatas(Map privateDatas) { - this.privateDatas = privateDatas; + @Override public String getName() { + return name; } - public boolean isInitSuccess() { - return initSuccess; + @Override public void setName(String name) { + this.name = name; } - public void setInitSuccess(boolean initSuccess) { - this.initSuccess = initSuccess; + @Override public String getType() { + return type; } - public boolean isDestroy() { - return isDestroy; + @Override public void setType(String type) { + this.type = type; } - public boolean isHasInit() { - return hasInit; + @Override public Properties getConfiguration() { + return configuration; } - public void setHasInit(boolean hasInit) { - this.hasInit = hasInit; + @Override + public void setConfiguration(Properties configuration) { + this.configuration = configuration; } - } 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 index 180e9914..9ccc8044 100644 --- 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 @@ -20,75 +20,34 @@ import com.alibaba.fastjson.JSONObject; import java.lang.reflect.Field; import java.lang.reflect.Modifier; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; +import java.util.ArrayList; +import java.util.List; +import org.apache.rocketmq.streams.common.configurable.annotation.ConfigurableReference; import org.apache.rocketmq.streams.common.configurable.annotation.NoSerialized; import org.apache.rocketmq.streams.common.datatype.DataType; +import org.apache.rocketmq.streams.common.model.Entity; import org.apache.rocketmq.streams.common.utils.DataTypeUtil; -import org.apache.rocketmq.streams.common.utils.ENVUtile; -import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.common.utils.ReflectUtil; -import org.apache.rocketmq.streams.common.utils.StringUtil; /** * 这个类自动完成成员变量的序列化,反序列化,以及环境变量的替换 子类只要按pojo实现即可。 有几个要求: 1.需要序列化的类,必须实现getset方法,这块下个版本会优化,去掉这个限制 2.不需要序列化的字段必须加transient 关键字声明 3.成员变量是 DataType支持的类型 */ -public class BasedConfigurable extends AbstractConfigurable { +public class BasedConfigurable extends AbstractConfigurable implements IConfigurableIdentification { /** * 扩展字段 */ public static final String EXTEND_FIELD_NAME = "extendField"; - - private String nameSpace; - - private String configureName; - - private String type; - - protected String version = "1.0"; - - protected long updateFlag = 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; - } + protected transient List envVarRegister = new ArrayList<>(); public JSONObject toJsonObject() { JSONObject jsonObject = new JSONObject(); setJsonObject(jsonObject); - jsonObject.put(IConfigurableService.CLASS_NAME, this.getClass().getName()); + jsonObject.put(CLASS_NAME, this.getClass().getName()); return jsonObject; } - @Override - public String toJson() { + @Override public String toJson() { JSONObject jsonObject = toJsonObject(); return jsonObject.toJSONString(); } @@ -99,11 +58,15 @@ protected void setJsonObject(JSONObject jsonObject) { setJsonObject(thisClass, jsonObject); } - protected void setJsonObject(Class clazz, JSONObject jsonObject) { - if (AbstractConfigurable.class.getName().equals(clazz.getName())) { + protected void setJsonObject(Class clazz, JSONObject jsonObject) { + if (clazz == null) { + return; + } + if (Entity.class.getName().equals(clazz.getName())) { return; } Field[] fields = clazz.getDeclaredFields(); + for (Field field : fields) { if (field.isAnnotationPresent(NoSerialized.class)) { continue; @@ -116,36 +79,46 @@ protected void setJsonObject(Class clazz, JSONObject jsonObject) { continue; } else if (field.getName().endsWith("this$0")) { continue; - } + } else if (field.isAnnotationPresent(ConfigurableReference.class)) { + + IConfigurable configurable = ReflectUtil.getBeanFieldValue(this, field.getName()); + if (configurable != null) { + jsonObject.put(field.getName() + ".type", configurable.getType()); + jsonObject.put(field.getName(), configurable.getName()); + } + 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); } + DataType dataType = null; + try { + dataType = DataTypeUtil.createFieldDataType(this, field.getName()); + Object fieldValue = ReflectUtil.getBeanFieldValue(this, field.getName()); + if (fieldValue != null) { + // 如果是空值则不再处理。入库也没意义 + String fieldValueStr = dataType.toDataJson(fieldValue); + jsonObject.put(field.getName(), fieldValueStr); + } + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException("序列化字段 " + getClass().getName() + "." + field.getName() + "error", e); + } + } - Class parent = clazz.getSuperclass(); + 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; + return 1; } protected void getJsonObject(JSONObject jsonObject) { - Class thisClass = this.getClass(); + Class thisClass = this.getClass(); getJsonObject(thisClass, jsonObject); } - protected void getJsonObject(Class clazz, JSONObject jsonObject) { - if (AbstractConfigurable.class.getName().equals(clazz.getName())) { + protected void getJsonObject(Class clazz, JSONObject jsonObject) { + if (Entity.class.getName().equals(clazz.getName())) { return; } if (Object.class.getName().equals(clazz.getName())) { @@ -153,6 +126,8 @@ protected void getJsonObject(Class clazz, JSONObject jsonObject) { } Field[] fields = clazz.getDeclaredFields(); for (Field field : fields) { + DataType dataType = null; + Object fieldValue = null; if (field.isAnnotationPresent(NoSerialized.class)) { continue; } @@ -162,100 +137,41 @@ protected void getJsonObject(Class clazz, JSONObject jsonObject) { continue; } else if (Modifier.isNative(field.getModifiers())) { continue; + } else if (field.isAnnotationPresent(ConfigurableReference.class)) { + + String configName = jsonObject.getString(field.getName()); + String type = jsonObject.getString(field.getName() + ".type"); + this.envVarRegister.add(new ENVField(type, configName, field.getName())); + continue; + + } else { + dataType = DataTypeUtil.createFieldDataType(this, field.getName()); + String fieldJsonStr = jsonObject.getString(field.getName()); + try { + fieldValue = dataType.getData(fieldJsonStr); + } catch (Exception e) { + e.printStackTrace(); + } + } - DataType dataType = DataTypeUtil.createFieldDataType(this, field.getName()); - String fieldJsonStr = jsonObject.getString(field.getName()); - fieldJsonStr = getENVParameter(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(); + Class parent = clazz.getSuperclass(); getJsonObject(parent, jsonObject); } - /** - * 支持存储env的key值,而具体的值存储在IENVParameter参数中 - * - * @param fieldValue - * @return - */ - protected String getENVParameter(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) { + @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; + public List getEnvVarRegister() { + return envVarRegister; } - public long getUpdateFlag() { - return updateFlag; - } - - public void setUpdateFlag(long updateFlag) { - this.updateFlag = updateFlag; - } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/ENVField.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/ENVField.java new file mode 100644 index 00000000..42df5b00 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/ENVField.java @@ -0,0 +1,53 @@ +/* + * 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 class ENVField { + protected String type; + protected String fieldName; + protected String configurableName; + + public ENVField(String type, String configurableName, String fieldName) { + this.type = type; + this.configurableName = configurableName; + this.fieldName = fieldName; + } + + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + public String getFieldName() { + return fieldName; + } + + public void setFieldName(String fieldName) { + this.fieldName = fieldName; + } + + public String getConfigurableName() { + return configurableName; + } + + public void setConfigurableName(String configurableName) { + this.configurableName = configurableName; + } +} 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 index 59da3c9d..3a204845 100644 --- 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 @@ -17,7 +17,7 @@ package org.apache.rocketmq.streams.common.configurable; import java.io.Serializable; -import java.util.Map; +import java.util.Properties; import org.apache.rocketmq.streams.common.datatype.IJsonable; /** @@ -25,6 +25,9 @@ */ public interface IConfigurable extends IJsonable, IConfigurableIdentification, Serializable { + String CLASS_NAME = "className"; + + String HTTP_SERVICE_ENDPOINT = "dipper.configurable.service.type.http.endpoint"; /** * 把toJson的结果当作一个特殊属性 */ @@ -38,46 +41,52 @@ public interface IConfigurable extends IJsonable, IConfigurableIdentification, S /** * 每个配置有一个独立的名字 * - * @param configureName + * @param name configureName */ - void setConfigureName(String configureName); + void setName(String name); /** * 每个配置有独立的命名空间 * - * @param nameSpace + * @param nameSpace namespace */ void setNameSpace(String nameSpace); /** * 区分配置类型 * - * @param type + * @param type type */ void setType(String type); - boolean init(); - - void destroy(); + /** + * 获取归属的任务名 + * + * @return 任务名称 + */ + String getJobName(); /** - * 设置私有数据 + * 每个配置有独立的命名空间 * - * @param key - * @param value - * @param + * @param jobName namespace */ - void putPrivateData(String key, T value); + void setJobName(String jobName); /** - * 获取私有数据 + * 初始化 * - * @param key - * @param - * @return + * @return 是否成功 */ - T getPrivateData(String key); + boolean init(); + + /** + * 销毁 + */ + void destroy(); + + Properties getConfiguration(); - Map getPrivateData(); + void setConfiguration(Properties configuration); } 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 index e82ac10c..9243511f 100644 --- 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 @@ -22,7 +22,7 @@ public interface IConfigurableIdentification { * * @return 配置项名称 */ - String getConfigureName(); + String getName(); /** * 用于区分不同的业务,用命名空间隔离业务 diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurableQuery.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurableQuery.java new file mode 100644 index 00000000..ac841ac1 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurableQuery.java @@ -0,0 +1,46 @@ +/* + * 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; + +public interface IConfigurableQuery { + /** + * 根据类型查询,同类的configuable对象 + * + * @param type + * @param + * @return + */ + List queryConfigurableByType(String type); + + T queryConfigurable(String configurableType, String name); + + List queryConfigurable(String type); + + Map queryConfigurableMapByType(String type); + + /** + * 获取所有的数据 + * + * @return + */ + Collection findAll(); + +} 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 deleted file mode 100644 index 55a4791f..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurableService.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * 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 FILE_PATH_NAME = "filePathAndName"; - - String HTTP_SERVICE_NAME = "http"; - - String HTTP_SERVICE_ENDPOINT = "dipper.configurable.service.type.http.endpoint"; - - - /** - * 启动定时任务,定期从存储加载对象到内存 - * - * @param namespace - */ - void initConfigurables(String namespace); - - /** - * 从存储加载对象到内存 - * - * @param namespace - * @return - */ - boolean refreshConfigurable(String namespace); - - /** - * 根据type进行配置查询 - * - * @return - */ - @Deprecated - List queryConfigurable(String type); - - /** - * 根据类型查询,同类的configuable对象 - * - * @param type - * @param - * @return - */ - List queryConfigurableByType(String type); - - /** - * 根据type和name进行配置查询 - * - * @return - */ - @Deprecated - IConfigurable queryConfigurableByIdent(String type, String name); - - /** - * 共享实现中:根据namespace,type和name进行配置查询 单业务实现中:根据type和name进行配置查询 - * - * @param identification namespace,type;name - * @return - */ - @Deprecated - IConfigurable queryConfigurableByIdent(String identification); - - /** - * 有具体的存储子类实现,直接写数据到存储,不会更新缓存,所以insert后,直接查询会查询不到,必须再次加载后才能获取 - * - * @param configurable - */ - void insert(IConfigurable configurable); - - /** - * 有具体的存储子类实现,直接写数据到存储,不会更新缓存,所以insert后,直接查询会查询不到,必须再次加载后才能获取 - * - * @param configurable - */ - void update(IConfigurable configurable); - - Map queryConfigurableMapByType(String type); - - T queryConfigurable(String configurableType, String name); - - String getNamespace(); - - /** - * 获取所有的数据 - * - * @return - */ - Collection findAll(); - - /** - * 从存储加载configurable对象,此处加载完,不会改变状态,query接口是无法查询的 - * - * @param type - * @return - */ - List loadConfigurableFromStorage(String type); - -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/annotation/ConfigurableReference.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/annotation/ConfigurableReference.java new file mode 100644 index 00000000..cc2f34e9 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/annotation/ConfigurableReference.java @@ -0,0 +1,27 @@ +/* + * 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.annotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@Target(ElementType.FIELD) +@Retention(RetentionPolicy.RUNTIME) +public @interface ConfigurableReference { +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configuration/ConfigurationKey.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configuration/ConfigurationKey.java new file mode 100644 index 00000000..89ec959c --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configuration/ConfigurationKey.java @@ -0,0 +1,157 @@ +/* + * 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.configuration; + +/** + * 配置文件中key config of dipper server + */ +public class ConfigurationKey { + + /** + * 系统参数: 数据库url, 主要用于租约 + */ + public final static String JDBC_URL = "dipper.rds.jdbc.url"; + public final static String JDBC_USERNAME = "dipper.rds.jdbc.username"; + public final static String JDBC_PASSWORD = "dipper.rds.jdbc.password"; + public final static String JDBC_DRIVER = "dipper.rds.jdbc.driver"; + /** + * 系统参数: 租约的续约周期 + */ + public final static String DIPPER_DISPATCHER_SCHEDULE_TIME = "dipper.dispatcher.schedule.time"; + + /** + * 系统参数: 情报的连接信息,只在专有云使用 + */ + public final static String INTELLIGENCE_JDBC_URL = "intelligence.rds.jdbc.url"; + public final static String INTELLIGENCE_JDBC_USERNAME = "intelligence.rds.jdbc.username"; + public final static String INTELLIGENCE_JDBC_PASSWORD = "intelligence.rds.jdbc.password"; + public final static String INTELLIGENCE_SWITCH = "intelligence.switch.open"; + public final static String INTELLIGENCE_TIP_DB_ENDPOINT = "intelligence.tip.db.endpoint"; + public final static String INTELLIGENCE_AK = "intelligence_ak"; + public final static String INTELLIGENCE_SK = "intelligence_sk"; + public final static String INTELLIGENCE_REGION = "intelligence_region"; + + /* + * 系统参数: 正则引擎的选择,有hyperscan 和 re2j俩种 + */ + public final static String DIPPER_REGEX_ENGINE = "dipper.regex.engine.option"; + + /** + * 系统参数: 超过多长时间,输出慢查询 + */ + public final static String MONITOR_SLOW_TIMEOUT = "dipper.monitor.slow.timeout"; + public final static String MONITOR_PIPELINE_HTML_SWITCH = "dipper.monitor.pipeline.html.switch"; + /** + * 系统参数: join 默认join的时间,单位是秒,默认是5分钟 + */ + public final static String DIPPER_WINDOW_JOIN_DEFAULT_INTERVAL_SIZE = "dipper.window.join.default.interval.size.time"; + /** + * 系统参数:join需要保留几个窗口,3个窗口意味着join的范围是上下15分钟多 + */ + public final static String DIPPER_WINDOW_JOIN_RETAIN_WINDOW_COUNT = "dipper.window.join.default.retain.window.count"; + + /** + * 系统参数:统计默认的窗口大小,单位是秒。默认是滚动窗口,默认是10分钟 + */ + public final static String DIPPER_WINDOW_DEFAULT_INTERVAL_SIZE = "dipper.window.default.interval.size.time"; + + /** + * 系统参数:over partition窗口的默认时间 + */ + + public final static String DIPPER_WINDOW_OVER_DEFAULT_INTERVAL_SIZE = "dipper.window.over.default.interval.size.time"; + public final static String DIPPER_WINDOW_OVER_DEFAULT_EMIT_BEFORE_SECOND = "dipper.window.over.default.emit.before.size.second"; + + /** + * shuffle相关配置,如果后面加上.namespace,则只对某个namespace生效,如window.shuffle.channel.type.namespace=rocketmq,相当于只对这个namespace配置 + */ + public final static String WINDOW_SHUFFLE_CHANNEL_TYPE = "window.shuffle.channel.type";//window 做shuffle中转需要的消息队列类型 + + public final static String WINDOW_SHUFFLE_CHANNEL_PROPERTY_PREFIX = "window.shuffle.channel."; + /** + * 系统参数:是否启动mini batch + */ + public final static String WINDOW_MINI_BATCH_SWITCH = "window.mini.batch.switch"; + + /** + * 系统参数:是否是本地存储 + */ + public final static String WINDOW_STORAGE_IS_LOCAL = "window.storage.is.local"; + + /** + * sql manager使用,选择sql的存储方式 + */ + public final static String SQL_STORAGE_TYPE = "sql.storage.type"; + + public final static String FINGERPRINT_CACHE_SIZE = "fingerprint.cache.size"; + + /** + * 默认的文件存储transport的name + */ + public final static String FILE_TRANSPORT_AK = "dipper.file.transport.ak"; + public final static String FILE_TRANSPORT_SK = "dipper.file.transport.sk"; + public final static String FILE_TRANSPORT_ENDPOINT = "dipper.file.transport.endpoint"; + public final static String FILE_TRANSPORT_DIPPER_DIR = "dipper.file.transport.dipper.dir"; + + public final static String LEASE_STORAGE_NAME = "lease.storage.name";//通过这个配置,可以修改lease 的底层存储 + + //window操作系统执行python脚本时的配置 + public final static String PYTHON_WORK_DIR = "siem.python.workdir"; + + /** + * rocketmq-stream更新模块对应的topic + */ + public final static String RULE_UP_TOPIC = "dipper.console.topic.up"; + + /** + * rocketmq-stream更新模块对应的topic + */ + public final static String RULE_UP_TAG = "dipper.console.topic.up.tags"; + + /** + * rocketmq-stream更新模块对应的topic + */ + public final static String RULE_DOWN_TOPIC = "dipper.console.topic.down"; + /** + * rocketmq-stream更新模块对应的topic + */ + public final static String RULE_DOWN_TAG = "dipper.console.topic.down.tags"; + /** + * rocketmq-stream更新模块对应的tag + */ + public final static String RULE_TOPIC_TAG = "dipper.console.tag"; + /** + * rocketmq-stream更新模块对应的消息渠道类型 默认为metaq + */ + public final static String UPDATE_TYPE = "dipper.console.service.type"; + public final static String HTTP_SERVICE_ENDPOINT = "dipper.configurable.service.type.http.endpoint"; + public final static String SYSLOG_TCP_PORT_PROPERTY_KEY = "dipper.syslog.server.tcp.port";//当需要改变端口值时,通过配置文件增加dipper.syslog.server.tcp.port=新端口的值 + public final static String SYSLOG_UDP_PORT_PROPERTY_KEY = "dipper.syslog.server.udp.port";//当需要改变端口值时,通过配置文件增加dipper.syslog.server.tcp.port=新端口的值 + 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.cj.jdbc.Driver"; + public final static String DEFAULT_JDBC_TABLE_NAME = "rocketmq_streams_checkpoint_table"; + + public final static String BASELINE_ENTITY_CACHE_SIZE = "dipper.baseline.entity.cache.size";//基线中缓存条数 + public final static String RUNTIME_ENV = "dipper.runtime.env"; + +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configuration/EmitInsertConfig.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configuration/EmitInsertConfig.java new file mode 100644 index 00000000..8fbe197f --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configuration/EmitInsertConfig.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.configuration; + +public class EmitInsertConfig { + protected Long emitBeforeValue; + protected Long emitAfterValue; + protected Long emitMaxDelay; + + public Long getEmitBeforeValue() { + return emitBeforeValue; + } + + public void setEmitBeforeValue(long value) { + this.emitBeforeValue = value; + } + + public Long getEmitAfterValue() { + return this.emitAfterValue; + } + + public void setEmitAfterValue(long value) { + this.emitAfterValue = value; + } + + public Long getEmitMaxDelay() { + return emitMaxDelay; + } + + public void setEmitMaxDelay(Long emitMaxDelay) { + this.emitMaxDelay = emitMaxDelay; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configuration/JobConfiguration.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configuration/JobConfiguration.java new file mode 100644 index 00000000..f7b26e02 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configuration/JobConfiguration.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.configuration; + +import java.util.Iterator; +import java.util.Properties; + +public class JobConfiguration { + private final Properties properties; + + public JobConfiguration() { + this.properties = new Properties(); + } + + public JobConfiguration(Properties properties) { + this.properties = properties; + } + + public Properties getProperties() { + return properties; + } + + /** + * 打开pipeline html monitor + */ + public JobConfiguration openHtmlMonitor() { + this.properties.put(ConfigurationKey.MONITOR_PIPELINE_HTML_SWITCH, "true"); + return this; + } + + /** + * 打开pipeline html monitor + */ + public JobConfiguration stageDelayTime(int stageSlowSecond) { + this.properties.put(ConfigurationKey.MONITOR_SLOW_TIMEOUT, stageSlowSecond); + return this; + + } + + /** + * 打开pipeline html monitor + */ + public JobConfiguration windowSize(int windowSizeSecond) { + this.properties.put(ConfigurationKey.DIPPER_WINDOW_DEFAULT_INTERVAL_SIZE, windowSizeSecond); + return this; + + } + + /** + * 打开pipeline html monitor + */ + public JobConfiguration joinWindowSize(int windowSizeSecond, int windowCount) { + this.properties.put(ConfigurationKey.DIPPER_WINDOW_JOIN_DEFAULT_INTERVAL_SIZE, windowSizeSecond); + this.properties.put(ConfigurationKey.DIPPER_WINDOW_JOIN_RETAIN_WINDOW_COUNT, windowCount); + return this; + + } + + /** + * 打开pipeline html monitor + */ + public JobConfiguration overWindowSize(int windowSizeSecond, int emitSecond) { + this.properties.put(ConfigurationKey.DIPPER_WINDOW_OVER_DEFAULT_INTERVAL_SIZE, windowSizeSecond); + this.properties.put(ConfigurationKey.DIPPER_WINDOW_OVER_DEFAULT_EMIT_BEFORE_SECOND, emitSecond); + return this; + + } + + /** + * 如果需要自定义shuffle,可以通过这个配置项实现 + * + * @param channelType 和sql with部分的type相同 + * @param properties 和sql with部分的key/value相同 + * @return + */ + public JobConfiguration shuffle(String channelType, Properties properties) { + this.properties.put(ConfigurationKey.WINDOW_SHUFFLE_CHANNEL_TYPE, channelType); + if (properties != null) { + Iterator it = properties.keySet().iterator(); + while (it.hasNext()){ + String key = it.next().toString(); + String prefixKey = ConfigurationKey.WINDOW_SHUFFLE_CHANNEL_PROPERTY_PREFIX + key; + this.properties.put(prefixKey, properties.get(key)); + } + + } + return this; + } + + public JobConfiguration config(String key, Object value) { + this.properties.put(key, value); + return this; + } + +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configuration/SystemContext.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configuration/SystemContext.java new file mode 100644 index 00000000..532234f3 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configuration/SystemContext.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.configuration; + +import java.util.Arrays; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import org.apache.rocketmq.streams.common.datatype.DateDataType; +import org.apache.rocketmq.streams.common.datatype.MapDataType; +import org.apache.rocketmq.streams.common.datatype.StringDataType; +import org.apache.rocketmq.streams.common.utils.PropertiesUtil; + +public class SystemContext { + static MapDataType mapDataType = new MapDataType(new StringDataType(), new StringDataType()); + static DateDataType dateDataType = new DateDataType(); + private static Properties properties; + + static { + Properties tmpProperties = PropertiesUtil.getResourceProperties("dipper.properties"); + if (tmpProperties == null) { + tmpProperties = new Properties(); + } + SystemContext.properties = tmpProperties; + } + + public static Object getParameter(String key) { + return properties.get(key); + } + + public static void put(String key, Object value) { + properties.put(key, value); + } + + public static Properties getProperties() { + return properties; + } + + public static Integer getIntParameter(String key) { + String value = getStringParameter(key); + if (value != null) { + return Integer.valueOf(value); + } + return null; + } + + public static String getProperty(String key, String defaultValue) { + String value = properties.getProperty(key); + if (value == null) { + return defaultValue; + } + return value; + } + + public static String getProperty(String key) { + return properties.getProperty(key); + } + + public static String getStringParameter(String key) { + return properties.getProperty(key); + } + + public static Boolean getBooleanParameter(String key) { + String value = getStringParameter(key); + if (value != null) { + return Boolean.valueOf(value); + } + return null; + } + + public static Long getLongParameter(String key) { + String value = getStringParameter(key); + if (value != null) { + return Long.valueOf(value); + } + return null; + } + + public static Double getDoubleParameter(String key) { + String value = getStringParameter(key); + if (value != null) { + return Double.valueOf(value); + } + return null; + } + + public static Float getFloatParameter(String key) { + String value = getStringParameter(key); + if (value != null) { + return Float.valueOf(value); + } + return null; + } + + public static List getListParameter(String key) { + String value = getStringParameter(key); + if (value != null) { + String[] values = key.split(","); + Arrays.asList(values); + } + return null; + } + + public static Map getMapParameter(String key) { + String value = getStringParameter(key); + if (value != null) { + + mapDataType.getData(value); + } + return null; + + } + + public static Date getDateParameter(String key) { + String value = getStringParameter(key); + if (value != null) { + dateDataType.getData(value); + } + return null; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configure/ConfigureFileKey.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configure/ConfigureFileKey.java deleted file mode 100644 index 0f7fcc58..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configure/ConfigureFileKey.java +++ /dev/null @@ -1,171 +0,0 @@ -/* - * 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.configure; - -import org.apache.rocketmq.streams.common.component.ComponentCreator; - -/** - * 配置文件中key config of dipper server - */ -public interface ConfigureFileKey { - - String CONNECT_TYPE = "dipper.configurable.service.type"; - /** - * 数据库url - */ - String DB_TYPE = "dipper.rds.jdbc.type"; - String JDBC_URL = "dipper.rds.jdbc.url"; - String JDBC_USERNAME = "dipper.rds.jdbc.username"; - String JDBC_PASSWORD = "dipper.rds.jdbc.password"; - String LEASE_CONSISTENT_HASH_SUFFIX = "dipper.lease.consistent.hash.suffix"; - String JDBC_DRIVER = "dipper.rds.jdbc.driver"; - String JDBC_TABLE_NAME = "dipper.rds.table.name"; - String SECRECY = "dipper.configure.sec.key"; - String SECRECY_DEFAULT = "93ea4211-6218-4353-b9cd-1d21618dbf4e"; - - /** - * 情报的连接信息 - */ - String INTELLIGENCE_JDBC_URL = "intelligence.rds.jdbc.url"; - String INTELLIGENCE_JDBC_USERNAME = "intelligence.rds.jdbc.username"; - String INTELLIGENCE_JDBC_PASSWORD = "intelligence.rds.jdbc.password"; - String INTELLIGENCE_SWTICH = "intelligence.switch.open"; - String INTELLIGENCE_TIP_DB_ENDPOINT = "intelligence.tip.db.endpoint"; - String INTELLIGENCE_AK = "intelligence_ak"; - String INTELLIGENCE_SK = "intelligence_sk"; - String INTELLIGENCE_REGION = "intelligence_region"; - - /* - * regex engine option, "hyperscan or re2j". when this option not set, use hyperscan default - */ - String DIPPER_REGEX_ENGINE = "dipper.regex.engine.option"; - - /** - * whether to start engine when loading configurable operator, true(default) or false - */ - String DIPPER_RUNNING_STATUS = "dipper.running.status"; - - String DIPPER_RUNNING_STATUS_DEFAULT = "true"; - - /** - * 代表常量,不需要在配置文件配置 - */ - String JDBC_COMPATIBILITY_RULEENGINE_TABLE_NAME = "ruleengine_configure"; - /** - * 如果需要兼容老规则引擎规则,且规则存储在ruleengine_configure中时,设置为true。如果老规则迁移到了dipper_configure, 这个值不需要设置或设置成false即可。兼容老的规则引擎,老规则引擎的namespace 是name_space需要通过这个配置告诉resource做适配。 - */ - String JDBC_COMPATIBILITY_OLD_RULEENGINE = "dipper.rds.compatibility.old.ruleengine"; - /** - * - */ - String POLLING_TIME = "dipper.configurable.polling.time"; - /** - * 代理dbchannel的class,需要继承JDBCDataSource抽象类。如果配置这个参数,则会给dbchannel增加一层代理,所有需要db访问的,都是通过open api发送sql给代理 - */ - String DB_PROXY_CLASS_NAME = ComponentCreator.DB_PROXY_CLASS_NAME; - /** - * 创建channel的服务 - */ - String DIPPER_INSTANCE_CHANNEL_CREATOR_SERVICE_NAME = ComponentCreator.DIPPER_INSTANCE_CHANNEL_CREATOR_SERVICE_NAME; - /** - * 默认的文件存储transport的name - */ - String FILE_TRANSPORT_NAME = "dipper.file.transport.name"; - String FILE_TRANSPORT_AK = "dipper.file.transport.ak"; - String FILE_TRANSPORT_SK = "dipper.file.transport.sk"; - String FILE_TRANSPORT_ENDPOINT = "dipper.file.transport.endpoint"; - String FILE_TRANSPORT_DIPPER_DIR = "dipper.file.transport.dipper.dir"; - - /** - * 监控相关的配置,监控输入的级别,有三种:INFO,SLOW,ERROR - */ - String MONITOR_OUTPUT_LEVEL = "dipper.monitor.output.level"; - /** - * 超过多长时间,输出慢查询 - */ - String MONITOR_SLOW_TIMEOUT = "dipper.monitor.slow.timeout"; - /** - * 如果是日志输出,需要指定默认的目录 - */ - String MONTIOR_LOGGER_DIR = "dipper.monitor.loger.dir"; - - /** - * 是否把原始日志备份下来,以备做效果测试 - */ - String INNER_MESSAGE_SWITCH = "dipper.inner.message.save.switch"; - /** - * 是否在做数据回放,如果再做数据回放,自动判读结果是否符合预期 - */ - String DIPPER_ORIG_MESSAGE_PLAYBACK = "dipper.orimessage.playback"; - - //dipper自己实现的延迟队列 - String DIPPER_WINDOW_DELAY_CHANNEL_TOPIC = "dipper.window.delay.channel.topic"; - - //join的默认窗口大小 - String DIPPER_WINDOW_JOIN_DEFAULT_ITERVA_SIZE = "dipper.window.join.default.iterval.size.time"; - // //join需要保留几个窗口,3个窗口意味着join的范围是上下15分钟多 - String DIPPER_WINDOW_JOIN_RETAIN_WINDOW_COUNT = "dipper.window.join.default.retain.window.count"; - //窗口多长延迟多长时间触发,确保多台机器的数据写入存储 - String DIPPER_WINDOW_DEFAULT_FIRE_DELAY_SECOND = "dipper.window.default.fire.delay.second"; - //统计默认的窗口大小,单位是分钟。默认是滚动窗口,大小是1个小时 - String DIPPER_WINDOW_DEFAULT_INERVAL_SIZE = "dipper.window.default.iterval.size.time"; - //统计默认的窗口大小,单位是分钟。默认是滚动窗口,大小是1个小时 - String DIPPER_WINDOW_DEFAULT_TIME_UNIT_ADJUST = "dipper.window.default.time.unit.adjust"; - //over partition窗口的默认时间 - String DIPPER_WINDOW_OVER_DEFAULT_ITERVA_SIZE = "dipper.window.over.default.iterval.size.time"; - - static String getDipperJdbcUrl() { - return ComponentCreator.getProperties().getProperty(JDBC_URL); - } - - static String getDipperJdbcUserName() { - return ComponentCreator.getProperties().getProperty(JDBC_USERNAME); - } - - static String getDipperJdbcPassword() { - return ComponentCreator.getProperties().getProperty(JDBC_PASSWORD); - } - - /** - * shuffle相关配置,如果后面加上.namespace,则只对某个namespace生效,如window.shuffle.channel.type.namespace=rocketmq,相当于只对这个namespace配置 - */ - String WINDOW_SHUFFLE_CHANNEL_TYPE = "window.shuffle.channel.type";//window 做shuffle中转需要的消息队列类型 - //比如rocketmq,需要topic,tags和group,属性值和字段名保持一致即可。配置如下:window.shuffle.channel.topic=abdc window.shuffle.channel.tag=fdd - - String WINDOW_SHUFFLE_CHANNEL_PROPERTY_PREFIX = "window.shuffle.channel."; - String WINDOW_SYSTEM_MESSAGE_CHENNEL_OWNER = "window.system.message.channel.owner";//如果能做消息过滤,只过滤本window的消息,可以配置这个属性,如rocketmq的tags.不支持的会做客户端过滤 - /** - * 通知相关 - */ - String WINDOW_SYSTEM_MESSAGE_CHENNEL_TYPE = "window.system.message.channel.type"; - String WINDOW_SYSTEM_MESSAGE_CHENNEL_PROPERTY_PREFIX = "window.system.message.channel."; - - String LEASE_STORAGE_NAME = "DB";//通过这个配置,可以修改lease 的底层存储 - - String CHECKPOINT_STORAGE_NAME = "checkPointStorageName"; - - String IS_RECOVER_MODE = "isRecover"; - - String IS_ATOMIC_DB_SINK = "isAtomicDbSink"; - - /** - * minibatch - */ - - String WINDOW_MINIBATCH_SWITCH="window.minibatch.switch";//是否启动minibatch - -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configure/StreamsConfigure.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configure/StreamsConfigure.java deleted file mode 100644 index 95986aff..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configure/StreamsConfigure.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * 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.configure; - -import org.apache.rocketmq.streams.common.component.ComponentCreator; - -public class StreamsConfigure { - protected static String EMIT_BEFORE_VALUE_KEY="emit.before.value"; - protected static String EMIT_AFTER_VALUE_KEY="emit.after.value"; - protected static String EMIT_MAX_DELAY_VALUE_KEY="emit.max.value"; - public static Long getEmitBeforeValue(){ - String value= ComponentCreator.getProperties().getProperty(EMIT_BEFORE_VALUE_KEY); - if(value==null){ - return null; - } - return Long.valueOf(value); - } - public static void setEmitBeforeValue(long value){ - ComponentCreator.getProperties().put(EMIT_BEFORE_VALUE_KEY,value+""); - } - public static void setEmitAfterValue(long value){ - ComponentCreator.getProperties().put(EMIT_AFTER_VALUE_KEY,value+""); - } - - public static Long getEmitAfterValue(){ - String value= ComponentCreator.getProperties().getProperty(EMIT_AFTER_VALUE_KEY); - if(value==null){ - return null; - } - return Long.valueOf(value); - } - - public static void setEmitMaxDelayValueKey(long value){ - ComponentCreator.getProperties().put(EMIT_MAX_DELAY_VALUE_KEY,value+""); - } - - public static Long getEmitMaxDelay(){ - String value= ComponentCreator.getProperties().getProperty(EMIT_MAX_DELAY_VALUE_KEY); - if(value==null){ - return null; - } - return Long.valueOf(value); - } - -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/constant/State.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/constant/State.java new file mode 100644 index 00000000..1c72f304 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/constant/State.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.constant; + +public class State { + + public static final String STARTED = "started"; + public static final String STOPPED = "stopped"; + +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/AbstractContext.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/AbstractContext.java index 2ead3cd6..97e5be6a 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/AbstractContext.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/AbstractContext.java @@ -23,9 +23,6 @@ import java.util.Map; import org.apache.rocketmq.streams.common.cache.compress.BitSetCache; import org.apache.rocketmq.streams.common.configurable.IConfigurable; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.interfaces.IBaseStreamOperator; -import org.apache.rocketmq.streams.common.model.ThreadContext; import org.apache.rocketmq.streams.common.monitor.IMonitor; import org.apache.rocketmq.streams.common.monitor.MonitorFactory; import org.apache.rocketmq.streams.common.optimization.FilterResultCache; @@ -38,39 +35,31 @@ public abstract class AbstractContext extends HashMap { public static final String MESSAGE_KEY = "message_context";//如果context需要存在message中,可以用这个key protected static final String CLASS_NAME = "className"; - - protected transient IConfigurableService configurableService; - + /** + * cache filter(regex,like,equals)result + */ + private static String FILTER_CACHE_PREPIX = "__filter_cache_prefix"; @Deprecated protected Map values = new HashMap(); - /** * 是否退出循环,只有在循环模式中应用 在移步stage中,使用,如果消息是因为在stage中被过滤了,则设置这个值为true */ protected boolean isBreak = false; - protected boolean isContinue = true; - protected T message; - /** * 必须强制设置才回生效 */ protected boolean isSplitModel = false; - /** * 如果消息在执行过程中做了拆分,把拆分后的消息放入这个字段 */ protected List splitMessages = new ArrayList(); - protected volatile IMonitor monitor = null; - protected FilterResultCache quickFilterResult; - protected Map homologousResult; - //未触发规则的表达式 - protected List notFireExpressionMonitor=new ArrayList<>(); + protected List notFireExpressionMonitor = new ArrayList<>(); public AbstractContext(T message) { this.message = message; @@ -92,7 +81,6 @@ public void cancelBreak() { public > void syncContext(C subContext) { this.putAll(subContext); this.setValues(subContext.getValues()); - this.setConfigurableService(subContext.getConfigurableService()); this.setSplitModel(subContext.isSplitModel()); this.setMessage(subContext.getMessage()); this.setSplitMessages(subContext.getSplitMessages()); @@ -100,14 +88,13 @@ public > void syncContext(C subContext) { this.isBreak = subContext.isBreak; this.quickFilterResult = subContext.quickFilterResult; this.homologousResult = subContext.homologousResult; - this.isContinue=subContext.isContinue; - this.notFireExpressionMonitor=subContext.notFireExpressionMonitor; + this.isContinue = subContext.isContinue; + this.notFireExpressionMonitor = subContext.notFireExpressionMonitor; } public > C syncSubContext(C subContext) { subContext.putAll(this); subContext.setValues(this.getValues()); - subContext.setConfigurableService(this.getConfigurableService()); subContext.setSplitModel(this.isSplitModel()); subContext.setMessage(this.getMessage()); subContext.setSplitMessages(this.getSplitMessages()); @@ -115,8 +102,8 @@ public > C syncSubContext(C subContext) { subContext.isBreak = isBreak; subContext.quickFilterResult = quickFilterResult; subContext.homologousResult = homologousResult; - subContext.isContinue=isContinue; - subContext.notFireExpressionMonitor=notFireExpressionMonitor; + subContext.isContinue = isContinue; + subContext.notFireExpressionMonitor = notFireExpressionMonitor; return subContext; } @@ -189,15 +176,14 @@ public boolean isSplitModel() { return isSplitModel; } + public void setSplitModel(boolean splitModel) { + isSplitModel = splitModel; + } + public void openSplitModel() { isSplitModel = true; } - /** - * cache filter(regex,like,equals)result - */ - private static String FILTER_CACHE_PREPIX = "__filter_cache_prefix"; - public void setFilterCache(String expressionStr, String varValue, boolean result) { this.put(MapKeyUtil.createKey(FILTER_CACHE_PREPIX, expressionStr, varValue), result); } @@ -230,10 +216,6 @@ public void removeValue(String key) { values.remove(key); } - public void setSplitModel(boolean splitModel) { - isSplitModel = splitModel; - } - public void closeSplitMode(T message) { this.setSplitModel(false); this.splitMessages = new ArrayList<>(); @@ -244,90 +226,10 @@ public Map getValues() { return values; } - public IConfigurableService getConfigurableService() { - return configurableService; - } - - public void setConfigurableService(IConfigurableService configurableService) { - this.configurableService = configurableService; - } - public void setValues(Map values) { this.values = values; } - public static List executeScript(IMessage channelMessage, C context, - List> scriptExpressions) { - List messages = new ArrayList<>(); - if (scriptExpressions == null) { - return messages; - } - boolean isSplitMode = context.isSplitModel(); - context.closeSplitMode(channelMessage); - int nextIndex = 1; - //long start=System.currentTimeMillis(); - try { - executeScript(scriptExpressions.get(0), channelMessage, context, nextIndex, scriptExpressions); - } catch (Exception e) { - e.printStackTrace(); - } - - // System.out.println("================="+(System.currentTimeMillis()-start)); - if (!context.isContinue()) { - context.setSplitModel(isSplitMode || context.isSplitModel()); - return null; - } - - if (context.isSplitModel()) { - messages = context.getSplitMessages(); - } else { - messages.add(context.getMessage()); - } - context.setSplitModel(isSplitMode || context.isSplitModel()); - return messages; - } - - /** - * 执行当前规则,如果规则符合拆分逻辑调拆分逻辑。为了是减少循环次数,一次循环多条规则 - * - * @param currentExpression - * @param channelMessage - * @param context - * @param nextIndex - * @param scriptExpressions - */ - private static void executeScript( - IBaseStreamOperator currentExpression, - IMessage channelMessage, C context, int nextIndex, - List> scriptExpressions) { - //long start=System.currentTimeMillis(); - - /** - * 为了兼容blink udtf,通过localthread把context传给udtf的collector - */ - ThreadContext threadContext = ThreadContext.getInstance(); - threadContext.set(context); - currentExpression.doMessage(channelMessage, context); - - //System.out.println(currentExpression.toString()+" cost time is "+(System.currentTimeMillis()-start)); - if (context.isContinue() == false) { - return; - } - if (nextIndex >= scriptExpressions.size()) { - return; - } - IBaseStreamOperator nextScriptExpression = scriptExpressions.get(nextIndex); - nextIndex++; - if (context.isSplitModel()) { - // start=System.currentTimeMillis(); - executeSplitScript(nextScriptExpression, channelMessage, context, nextIndex, scriptExpressions); - - //System.out.println(currentExpression.toString()+" cost time is "+(System.currentTimeMillis()-start)); - } else { - executeScript(nextScriptExpression, channelMessage, context, nextIndex, scriptExpressions); - } - } - /** * 启动monitor,如果monitor已经启动,则重复调用可重入 * @@ -342,36 +244,6 @@ public IMonitor startMonitor(String name) { return monitor; } - private static void executeSplitScript( - IBaseStreamOperator currentExpression, IMessage channelMessage, C context, int nextIndex, - List> scriptExpressions) { - if (context.getSplitMessages() == null || context.getSplitMessages().size() == 0) { - return; - } - List result = new ArrayList<>(); - List splitMessages = new ArrayList(); - splitMessages.addAll(context.getSplitMessages()); - int splitMessageOffset = 0; - for (IMessage message : splitMessages) { - context.closeSplitMode(message); - message.getHeader().addLayerOffset(splitMessageOffset); - splitMessageOffset++; - executeScript(currentExpression, message, context, nextIndex, scriptExpressions); - if (!context.isContinue()) { - context.cancelBreak(); - continue; - } - if (context.isSplitModel()) { - result.addAll(context.getSplitMessages()); - } else { - result.add(context.getMessage()); - } - - } - context.openSplitModel(); - context.setSplitMessages(result); - } - public IMonitor getCurrentMonitorItem(String... parentNames) { if (monitor != null) { if (monitor.getChildren() == null || monitor.getChildren().size() == 0) { @@ -385,14 +257,13 @@ public IMonitor getCurrentMonitorItem(String... parentNames) { } } - public abstract AbstractContext copy(); + public abstract AbstractContext copy(); protected void copyProperty(AbstractContext context) { Map values = new HashMap<>(); values.putAll(this.getValues()); context.setValues(values); context.putAll(this); - context.setConfigurableService(this.getConfigurableService()); context.setSplitModel(this.isSplitModel()); List messages = new ArrayList<>(); for (T tmp : this.getSplitMessages()) { @@ -416,19 +287,18 @@ public IMonitor createChildrenMonitor(String parentMintorName, IConfigurable con return monitor.createChildren(configurable); } - public NotFireReason getNotFireReason(){ - return (NotFireReason)this.get("NotFireReason"); + public NotFireReason getNotFireReason() { + return (NotFireReason) this.get("NotFireReason"); } - - public void setNotFireReason(NotFireReason notFireReason){ - this.put("NotFireReason",notFireReason); + public void setNotFireReason(NotFireReason notFireReason) { + this.put("NotFireReason", notFireReason); } - - public void removeNotFireReason(){ + public void removeNotFireReason() { this.remove("NotFireReason"); } + public boolean isBreak() { return isBreak; } @@ -437,10 +307,6 @@ public void setBreak(boolean aBreak) { isBreak = aBreak; } - public void setQuickFilterResult(FilterResultCache quickFilterResult) { - this.quickFilterResult = quickFilterResult; - } - public Map getHomologousResult() { return homologousResult; } @@ -462,4 +328,8 @@ public FilterResultCache getQuickFilterResult() { return quickFilterResult; } + public void setQuickFilterResult(FilterResultCache quickFilterResult) { + this.quickFilterResult = quickFilterResult; + } + } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/BatchMessageOffset.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/BatchMessageOffset.java index 321ebc5e..c154f378 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/BatchMessageOffset.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/BatchMessageOffset.java @@ -45,7 +45,7 @@ public BatchMessageOffset() { public static BatchMessageOffset create(IConfigurable configurable, String msg, String offsetName, String currentMessageKey) { BatchMessageOffset offset = new BatchMessageOffset(); offset.setNameSpace(configurable.getNameSpace()); - offset.setConfigureName(offsetName); + offset.setName(offsetName); offset.ownerType = configurable.getType(); if (StringUtil.isEmpty(msg)) { offset.getMessages().put(currentMessageKey, new JSONObject().toJSONString()); @@ -92,6 +92,10 @@ public Map getMessages() { return messages; } + public void setMessages(Map messages) { + this.messages = messages; + } + public String getOwnerType() { return ownerType; } @@ -104,6 +108,10 @@ public long getLastUpdateTime() { return lastUpdateTime; } + public void setLastUpdateTime(long lastUpdateTime) { + this.lastUpdateTime = lastUpdateTime; + } + public String getCurrentMessageKey() { return currentMessageKey; } @@ -111,12 +119,4 @@ public String getCurrentMessageKey() { public void setCurrentMessageKey(String currentMessageKey) { this.currentMessageKey = currentMessageKey; } - - public void setMessages(Map messages) { - this.messages = messages; - } - - public void setLastUpdateTime(long lastUpdateTime) { - this.lastUpdateTime = lastUpdateTime; - } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/IMessage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/IMessage.java index 4b6bed8f..ee89b773 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/IMessage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/IMessage.java @@ -80,5 +80,4 @@ public interface IMessage { */ ISystemMessage getSystemMessage(); - } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/Message.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/Message.java index e11f6d3c..7cb9e5cd 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/Message.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/Message.java @@ -23,13 +23,10 @@ public class Message implements IMessage { - private JSONObject message; - - private boolean isJsonMessage = true; - protected ISystemMessage systemMessage; - protected MessageHeader header = new MessageHeader(); + private JSONObject message; + private boolean isJsonMessage = true; public Message(JSONObject message) { this.message = message; @@ -47,11 +44,27 @@ public Message(JSONObject message) { } } + public static JSONObject parseObject(String msg) { + JSONObject jsonObject = JSONObject.parseObject(msg); + if (jsonObject != null) { + String userObjectString = jsonObject.getString(UserDefinedMessage.class.getName()); + if (userObjectString != null) { + return new UserDefinedMessage(jsonObject, userObjectString); + } + } + return jsonObject; + } + @Override public MessageHeader getHeader() { return this.header; } + @Override + public void setHeader(MessageHeader header) { + this.header = header; + } + @Override public JSONObject getMessageBody() { return this.message; @@ -67,6 +80,10 @@ public boolean isJsonMessage() { return isJsonMessage; } + public void setJsonMessage(boolean jsonMessage) { + isJsonMessage = jsonMessage; + } + @Override public Object getMessageValue() { if (this.message instanceof UserDefinedMessage) { @@ -100,26 +117,6 @@ public Message deepCopy() { return message; } - public static JSONObject parseObject(String msg) { - JSONObject jsonObject = JSONObject.parseObject(msg); - if (jsonObject != null) { - String userObjectString = jsonObject.getString(UserDefinedMessage.class.getName()); - if (userObjectString != null) { - return new UserDefinedMessage(jsonObject, userObjectString); - } - } - return jsonObject; - } - - public void setJsonMessage(boolean jsonMessage) { - isJsonMessage = jsonMessage; - } - - @Override - public void setHeader(MessageHeader header) { - this.header = header; - } - @Override public ISystemMessage getSystemMessage() { return systemMessage; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/MessageHeader.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/MessageHeader.java index 0b105430..dd3bdc73 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/MessageHeader.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/MessageHeader.java @@ -17,12 +17,12 @@ package org.apache.rocketmq.streams.common.context; import com.alibaba.fastjson.JSONObject; +import java.util.Collections; import java.util.HashSet; import java.util.Set; import org.apache.rocketmq.streams.common.channel.source.ISource; import org.apache.rocketmq.streams.common.channel.split.ISplit; -import org.apache.rocketmq.streams.common.monitor.TopologyFilterMonitor; -import org.apache.rocketmq.streams.common.optimization.MessageGlobleTrace; +import org.apache.rocketmq.streams.common.optimization.MessageTrace; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.common.utils.ReflectUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; @@ -40,11 +40,21 @@ public class MessageHeader { */ public static final int SPLIT_OFFST_INIT = 10000000; protected String pipelineName; - + protected Set checkpointQueueIds;//当是系统消息时,保存checkpoint信息 + /** + * 在pipline中消息会被拆分,在有多分支时,会被copy,这个对象会在任何变动时,都保持全局唯一,不允许copy,复制,创建,一个message全局唯一 + */ + protected MessageTrace messageTrace; + /** + * trace id of every message + */ + protected String traceId = IMessage.DEFAULT_MESSAGE_TRACE_ID; + protected String msgRouteFromLable;//消息从哪里来的标签,标记上游节点的标记,主要是通过build table name来标记 + protected String logFingerprintValue;//日志指纹的值 /** * 当前消息的channel信息 */ - private transient ISource source; + private transient ISource source; /** * 路由用到的路由标签,标签的值是stage的label,用于路由stage,可以多个label */ @@ -61,9 +71,7 @@ public class MessageHeader { * 消息的offset,代表整个消息的offset */ private MessageOffset messageOffset = new MessageOffset();//保存消息的offset - - private ISplit messageQueue; - + private ISplit messageQueue; /** * 消息发送时间 */ @@ -80,27 +88,11 @@ public class MessageHeader { * 是否是系统消息 */ private boolean isSystemMessage = false; - - protected Set checkpointQueueIds;//当是系统消息时,保存checkpoint信息 /** * 在循环模式使用,主要表示当前的循环index */ private int loopIndex = -1; - /** - * 在pipline中消息会被拆分,在有多分支时,会被copy,这个对象会在任何变动时,都保持全局唯一,不允许copy,复制,创建,一个message全局唯一 - */ - protected MessageGlobleTrace messageGlobalTrace; - - /** - * trace id of every message - */ - protected String traceId = IMessage.DEFAULT_MESSAGE_TRACE_ID; - - protected String msgRouteFromLable;//消息从哪里来的标签,标记上游节点的标记,主要是通过build table name来标记 - - protected String logFingerprintValue;//日志指纹的值 - public MessageHeader copy() { MessageHeader header = new MessageHeader(); header.setSource(source); @@ -116,7 +108,7 @@ public MessageHeader copy() { header.progress.setCurrentMessage(progress.getCurrentMessage()); header.progress.setOwnerType(progress.getOwnerType()); } - header.messageGlobalTrace = messageGlobalTrace;//这里不必复制,会保持全局唯一 + header.messageTrace = messageTrace;//这里不必复制,会保持全局唯一 header.traceId = traceId; header.msgRouteFromLable = msgRouteFromLable; header.logFingerprintValue = logFingerprintValue; @@ -130,24 +122,24 @@ public JSONObject toJsonObject() { ReflectUtil.setFieldValue2Object(this, jsonObject); return jsonObject; } - public Set createRouteLableSet(String routeLabels){ - if(routeLabels==null){ + + public Set createRouteLabelSet(String routeLabels) { + if (routeLabels == null) { return null; } - String[] lables=MapKeyUtil.splitKey(routeLabels); - Set routeLableSet=new HashSet<>(); - for(String lable:lables){ - routeLableSet.add(lable); - } - return routeLableSet; + String[] labels = MapKeyUtil.splitKey(routeLabels); + Set routeLabelSet = new HashSet<>(); + Collections.addAll(routeLabelSet, labels); + return routeLabelSet; } + /** * 用于路由的标签,标签等于stage的label * * @param labels */ public String addRouteLabel(String... labels) { - this.routeLabels = createLables(routeLabels, labels); + this.routeLabels = createLabels(routeLabels, labels); return this.routeLabels; } @@ -155,19 +147,15 @@ public String addRouteLabel(String... labels) { * 用于路由的标签,标签等于stage的label */ public String addFilterLabel(String... labels) { - this.filterLabels = createLables(filterLabels, labels); + this.filterLabels = createLabels(filterLabels, labels); return this.filterLabels; } - public void setQueueId(String queueId) { - this.queueId = queueId; - } - - public ISource getSource() { + public ISource getSource() { return source; } - public void setSource(ISource source) { + public void setSource(ISource source) { this.source = source; } @@ -175,15 +163,26 @@ public String getRouteLabels() { return routeLabels; } + public void setRouteLabels(String routeLabels) { + this.routeLabels = routeLabels; + } public String getFilterLabels() { return filterLabels; } + public void setFilterLabels(String filterLabels) { + this.filterLabels = filterLabels; + } + public String getQueueId() { return queueId; } + public void setQueueId(String queueId) { + this.queueId = queueId; + } + public boolean isEmptyOffset() { return messageOffset == null; } @@ -195,11 +194,25 @@ public String getOffset() { return messageOffset.getOffsetStr(); } - public ISplit getMessageQueue() { + public void setOffset(String offset) { + messageOffset.parseOffsetStr(offset); + } + + public void setOffset(Integer offset) { + messageOffset.mainOffset = (offset + ""); + messageOffset.isLongOfMainOffset = true; + } + + public void setOffset(Long offset) { + messageOffset.mainOffset = (offset + ""); + messageOffset.isLongOfMainOffset = true; + } + + public ISplit getMessageQueue() { return messageQueue; } - public void setMessageQueue(ISplit messageQueue) { + public void setMessageQueue(ISplit messageQueue) { this.messageQueue = messageQueue; } @@ -210,7 +223,7 @@ public void setMessageQueue(ISplit messageQueue) { * @return */ public boolean greateThan(String dstOffset) { - return messageOffset.greateThan(dstOffset); + return messageOffset.greaterThan(dstOffset); } public String getTraceId() { @@ -224,10 +237,10 @@ public void setTraceId(String traceId) { /** * 创建路由标签 * - * @param routeLabels - * @param labels + * @param routeLabels 标签 + * @param labels 标签名称 */ - protected String createLables(String routeLabels, String... labels) { + protected String createLabels(String routeLabels, String... labels) { if (StringUtil.isEmpty(routeLabels)) { routeLabels = MapKeyUtil.createKey(labels); @@ -238,8 +251,6 @@ protected String createLables(String routeLabels, String... labels) { return routeLabels; } - - public boolean isNeedFlush() { return needFlush; } @@ -248,22 +259,14 @@ public void setNeedFlush(boolean needFlush) { this.needFlush = needFlush; } - public void setRouteLabels(String routeLabels) { - this.routeLabels = routeLabels; - } - - public void setFilterLabels(String filterLabels) { - this.filterLabels = filterLabels; + public long getSendTime() { + return sendTime; } public void setSendTime(long sendTime) { this.sendTime = sendTime; } - public long getSendTime() { - return sendTime; - } - public Boolean getOffsetIsLong() { return messageOffset.isLongOfMainOffset(); } @@ -284,6 +287,10 @@ public boolean isSystemMessage() { return isSystemMessage; } + public void setSystemMessage(boolean systemMessage) { + isSystemMessage = systemMessage; + } + public BatchMessageOffset getProgress() { return progress; } @@ -296,6 +303,10 @@ public int getLoopIndex() { return loopIndex; } + public void setLoopIndex(int loopIndex) { + this.loopIndex = loopIndex; + } + public Set getCheckpointQueueIds() { return checkpointQueueIds; } @@ -304,22 +315,12 @@ public void setCheckpointQueueIds(Set checkpointQueueIds) { this.checkpointQueueIds = checkpointQueueIds; } - public void setLoopIndex(int loopIndex) { - this.loopIndex = loopIndex; + public MessageTrace getMessageTrace() { + return messageTrace; } - public void setSystemMessage(boolean systemMessage) { - isSystemMessage = systemMessage; - } - - - - public MessageGlobleTrace getMessageGlobalTrace() { - return messageGlobalTrace; - } - - public void setMessageGlobalTrace(MessageGlobleTrace messageGlobalTrace) { - this.messageGlobalTrace = messageGlobalTrace; + public void setMessageTrace(MessageTrace messageTrace) { + this.messageTrace = messageTrace; } public String getMsgRouteFromLable() { @@ -338,20 +339,6 @@ public void setLogFingerprintValue(String logFingerprintValue) { this.logFingerprintValue = logFingerprintValue; } - public void setOffset(String offset) { - messageOffset.parseOffsetStr(offset); - } - - public void setOffset(Integer offset) { - messageOffset.mainOffset = (offset + ""); - messageOffset.isLongOfMainOffset = true; - } - - public void setOffset(Long offset) { - messageOffset.mainOffset = (offset + ""); - messageOffset.isLongOfMainOffset = true; - } - public String getPipelineName() { return pipelineName; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/MessageOffset.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/MessageOffset.java index 38950f8f..8f290978 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/MessageOffset.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/MessageOffset.java @@ -18,6 +18,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Objects; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; @@ -28,12 +29,12 @@ public class MessageOffset { /** * 因为是字符串比较,需要有一个固定位数 */ - public static final int LAYER_OFFST_INIT = 10000000; + public static final int LAYER_OFFSET_INIT = 10000000; public static String SPLIT_SIGN = "."; protected String mainOffset;//数据源发送出来的offset。 protected List offsetLayers = new ArrayList<>();//多级offset - protected boolean isLongOfMainOffset = true;//主offset是否是long型 + protected boolean isLongOfMainOffset;//主offset是否是long型 public MessageOffset(String offset, boolean isLongOfMainOffset) { mainOffset = parseOffset(offset, offsetLayers); @@ -41,7 +42,7 @@ public MessageOffset(String offset, boolean isLongOfMainOffset) { } public MessageOffset(Long mainOffset) { - this.mainOffset = mainOffset + ""; + this.mainOffset = String.valueOf(mainOffset); this.isLongOfMainOffset = true; } @@ -51,54 +52,15 @@ public MessageOffset(String mainOffset) { } public MessageOffset(Integer mainOffset) { - this.mainOffset = mainOffset + ""; + this.mainOffset = String.valueOf(mainOffset); this.isLongOfMainOffset = true; } public MessageOffset() { - this.mainOffset = System.nanoTime() + ""; + this.mainOffset = String.valueOf(System.nanoTime()); this.isLongOfMainOffset = true; } - public void parseOffsetStr(String offsetStr) { - mainOffset = parseOffset(offsetStr, offsetLayers); - } - - /** - * 获取offset字符串,通过.把主offset和子offset串接在一起 - * - * @return - */ - public String getOffsetStr() { - String[] offsets = new String[offsetLayers.size() + 1]; - offsets[0] = mainOffset; - int index = 1; - for (Long subOffset : offsetLayers) { - offsets[index] = subOffset + ""; - index++; - } - return MapKeyUtil.createKeyBySign(SPLIT_SIGN, offsets); - } - - /** - * 增加一个子offset - * - * @param index - */ - public void addLayerOffset(long index) { - offsetLayers.add(LAYER_OFFST_INIT + index); - } - - /** - * 比较当前offset是否大于目标offset - * - * @param dstOffset - * @return - */ - public boolean greateThan(String dstOffset) { - return greateThan(getOffsetStr(), dstOffset, isLongOfMainOffset); - } - /** * 比较两个offset,orioffset是否大于dstoffset * @@ -107,8 +69,8 @@ public boolean greateThan(String dstOffset) { * @param isOffsetIsLong * @return */ - public static boolean greateThan(String oriOffset, String dstOffset, boolean isOffsetIsLong) { - if (isOffsetIsLong == false) { + public static boolean greaterThan(String oriOffset, String dstOffset, boolean isOffsetIsLong) { + if (!isOffsetIsLong) { return (oriOffset.compareTo(dstOffset) > 0); } if (StringUtil.isEmpty(dstOffset)) { @@ -116,41 +78,19 @@ public static boolean greateThan(String oriOffset, String dstOffset, boolean isO } List dstOffsetLayers = new ArrayList<>(); - Long dstMainOffset = Long.valueOf(parseOffset(dstOffset, dstOffsetLayers)); + long dstMainOffset = Long.parseLong(Objects.requireNonNull(parseOffset(dstOffset, dstOffsetLayers))); List oriOffsetLayers = new ArrayList<>(); - Long oriMainOffset = Long.valueOf(parseOffset(oriOffset, oriOffsetLayers)); - - if (oriMainOffset > dstMainOffset) { - return true; - } else if (oriMainOffset <= dstMainOffset) { - return false; - } + long oriMainOffset = Long.parseLong(Objects.requireNonNull(parseOffset(oriOffset, oriOffsetLayers))); - for (int i = 0; i < oriOffsetLayers.size(); i++) { - Long origLayerOffset = (i < oriOffsetLayers.size()) ? oriOffsetLayers.get(i) : null; - Long destLayerOffset = (i < dstOffsetLayers.size()) ? dstOffsetLayers.get(i) : null; - if (origLayerOffset != null && destLayerOffset != null) { - if (origLayerOffset > destLayerOffset) { - return true; - } else if (origLayerOffset <= destLayerOffset) { - return false; - } - continue; - } - if (origLayerOffset != null && destLayerOffset == null) { - return true; - } - break; - } - return false; + return oriMainOffset > dstMainOffset; } /** * 解析offset 字符串 * - * @param offset offse字符串,格式:mainoffset.layyer1offset.layer2offset + * @param offset offset字符串,格式:mainoffset.layer1offset.layer2offset * @param offsetLayers 不同层次的offset * @return */ @@ -178,13 +118,56 @@ public static void main(String[] args) { messageOffset.addLayerOffset(2); String offset = (messageOffset.getOffsetStr()); messageOffset = new MessageOffset(offset, false); - System.out.println(messageOffset.greateThan("12345.10000001.10000001")); + System.out.println(messageOffset.greaterThan("12345.10000001.10000001")); + } + + public void parseOffsetStr(String offsetStr) { + mainOffset = parseOffset(offsetStr, offsetLayers); + } + + /** + * 获取offset字符串,通过.把主offset和子offset串接在一起 + * + * @return + */ + public String getOffsetStr() { + String[] offsets = new String[offsetLayers.size() + 1]; + offsets[0] = mainOffset; + int index = 1; + for (Long subOffset : offsetLayers) { + offsets[index] = subOffset + ""; + index++; + } + return MapKeyUtil.createKeyBySign(SPLIT_SIGN, offsets); + } + + /** + * 增加一个子offset + * + * @param index + */ + public void addLayerOffset(long index) { + offsetLayers.add(LAYER_OFFSET_INIT + index); + } + + /** + * 比较当前offset是否大于目标offset + * + * @param dstOffset + * @return + */ + public boolean greaterThan(String dstOffset) { + return greaterThan(getOffsetStr(), dstOffset, isLongOfMainOffset); } public boolean isLongOfMainOffset() { return isLongOfMainOffset; } + public void setLongOfMainOffset(boolean longOfMainOffset) { + isLongOfMainOffset = longOfMainOffset; + } + public List getOffsetLayers() { return offsetLayers; } @@ -193,10 +176,6 @@ public void setOffsetLayers(List offsetLayers) { this.offsetLayers = offsetLayers; } - public void setLongOfMainOffset(boolean longOfMainOffset) { - isLongOfMainOffset = longOfMainOffset; - } - public String getMainOffset() { return mainOffset; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/UserDefinedMessage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/UserDefinedMessage.java index 39a41bc0..db5ee441 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/UserDefinedMessage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/UserDefinedMessage.java @@ -50,7 +50,8 @@ public UserDefinedMessage(Object messageValue) { this(messageValue, null); } - public UserDefinedMessage() {} + public UserDefinedMessage() { + } public UserDefinedMessage(Object messageValue, List columnNames) { this.messageValue = messageValue; @@ -65,7 +66,7 @@ public UserDefinedMessage(Object messageValue, List columnNames) { } } else if (DataTypeUtil.isList(clazz) || DataTypeUtil.isArray(clazz)) { isList = true; - Iterator it = ((Iterable)messageValue).iterator(); + Iterator it = ((Iterable) messageValue).iterator(); int i = 0; while (it.hasNext()) { Object value = it.next(); @@ -78,7 +79,7 @@ public UserDefinedMessage(Object messageValue, List columnNames) { } } else if (DataTypeUtil.isMap(clazz) || messageValue instanceof JSONObject) { isMap = true; - this.fieldMap = (Map)messageValue; + this.fieldMap = (Map) messageValue; } else { isPojo = true; ReflectUtil.scanFields(messageValue, (o, field) -> this.fieldMap.put(field.getName(), field)); @@ -130,7 +131,7 @@ public boolean containsValue(Object value) { @Override public Object get(Object key) { - Object msgValue = getBeanFieldOrJsonValue(messageValue, (String)key); + Object msgValue = getBeanFieldOrJsonValue(messageValue, (String) key); if (msgValue != null) { return msgValue; } @@ -141,7 +142,7 @@ public Object get(Object key) { public JSONObject getJSONObject(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue instanceof JSONObject) { - return (JSONObject)msgValue; + return (JSONObject) msgValue; } return super.getJSONObject(key); } @@ -150,7 +151,7 @@ public JSONObject getJSONObject(String key) { public JSONArray getJSONArray(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue instanceof JSONArray) { - return (JSONArray)msgValue; + return (JSONArray) msgValue; } return super.getJSONArray(key); } @@ -159,7 +160,7 @@ public JSONArray getJSONArray(String key) { public T getObject(String key, Class clazz) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (clazz.isInstance(msgValue)) { - return (T)msgValue; + return (T) msgValue; } return super.getObject(key, clazz); } @@ -168,7 +169,7 @@ public T getObject(String key, Class clazz) { public Boolean getBoolean(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && DataTypeUtil.isBoolean(msgValue.getClass())) { - return (Boolean)msgValue; + return (Boolean) msgValue; } return super.getBoolean(key); } @@ -177,7 +178,7 @@ public Boolean getBoolean(String key) { public byte[] getBytes(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && byte[].class.isInstance(msgValue.getClass())) { - return (byte[])msgValue; + return (byte[]) msgValue; } return super.getBytes(key); } @@ -186,7 +187,7 @@ public byte[] getBytes(String key) { public boolean getBooleanValue(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && DataTypeUtil.isBoolean(msgValue.getClass())) { - return (Boolean)msgValue; + return (Boolean) msgValue; } return super.getBooleanValue(key); } @@ -195,7 +196,7 @@ public boolean getBooleanValue(String key) { public Byte getByte(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && DataTypeUtil.isByte(msgValue.getClass())) { - return (Byte)msgValue; + return (Byte) msgValue; } return super.getByte(key); } @@ -204,7 +205,7 @@ public Byte getByte(String key) { public byte getByteValue(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && DataTypeUtil.isByte(msgValue.getClass())) { - return (Byte)msgValue; + return (Byte) msgValue; } return super.getByteValue(key); } @@ -213,7 +214,7 @@ public byte getByteValue(String key) { public Short getShort(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && DataTypeUtil.isShort(msgValue.getClass())) { - return (Short)msgValue; + return (Short) msgValue; } return super.getShort(key); } @@ -222,7 +223,7 @@ public Short getShort(String key) { public short getShortValue(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && DataTypeUtil.isShort(msgValue.getClass())) { - return (Short)msgValue; + return (Short) msgValue; } return super.getShortValue(key); } @@ -231,7 +232,7 @@ public short getShortValue(String key) { public Integer getInteger(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && DataTypeUtil.isInt(msgValue.getClass())) { - return (Integer)msgValue; + return (Integer) msgValue; } return super.getInteger(key); } @@ -240,7 +241,7 @@ public Integer getInteger(String key) { public int getIntValue(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && DataTypeUtil.isInt(msgValue.getClass())) { - return (int)msgValue; + return (int) msgValue; } return super.getIntValue(key); } @@ -249,7 +250,7 @@ public int getIntValue(String key) { public Long getLong(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && DataTypeUtil.isLong(msgValue.getClass())) { - return (Long)msgValue; + return (Long) msgValue; } return super.getLong(key); } @@ -258,7 +259,7 @@ public Long getLong(String key) { public long getLongValue(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && DataTypeUtil.isLong(msgValue.getClass())) { - return (long)msgValue; + return (long) msgValue; } return super.getLongValue(key); } @@ -267,7 +268,7 @@ public long getLongValue(String key) { public Float getFloat(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && DataTypeUtil.isFloat(msgValue.getClass())) { - return (Float)msgValue; + return (Float) msgValue; } return super.getFloat(key); } @@ -276,7 +277,7 @@ public Float getFloat(String key) { public float getFloatValue(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && DataTypeUtil.isFloat(msgValue.getClass())) { - return (float)msgValue; + return (float) msgValue; } return super.getFloatValue(key); } @@ -285,7 +286,7 @@ public float getFloatValue(String key) { public Double getDouble(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && DataTypeUtil.isDouble(msgValue.getClass())) { - return (Double)msgValue; + return (Double) msgValue; } return super.getDouble(key); } @@ -294,7 +295,7 @@ public Double getDouble(String key) { public double getDoubleValue(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && DataTypeUtil.isDouble(msgValue.getClass())) { - return (double)msgValue; + return (double) msgValue; } return super.getDoubleValue(key); } @@ -303,7 +304,7 @@ public double getDoubleValue(String key) { public BigDecimal getBigDecimal(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && BigDecimal.class.isInstance(msgValue.getClass())) { - return (BigDecimal)msgValue; + return (BigDecimal) msgValue; } return super.getBigDecimal(key); } @@ -312,7 +313,7 @@ public BigDecimal getBigDecimal(String key) { public BigInteger getBigInteger(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && BigInteger.class.isInstance(msgValue.getClass())) { - return (BigInteger)msgValue; + return (BigInteger) msgValue; } return super.getBigInteger(key); } @@ -321,7 +322,7 @@ public BigInteger getBigInteger(String key) { public String getString(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && String.class.isInstance(msgValue.getClass())) { - return (String)msgValue; + return (String) msgValue; } return super.getString(key); } @@ -330,7 +331,7 @@ public String getString(String key) { public Date getDate(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && Date.class.isInstance(msgValue.getClass())) { - return (Date)msgValue; + return (Date) msgValue; } return super.getDate(key); } @@ -339,18 +340,18 @@ public Date getDate(String key) { public java.sql.Date getSqlDate(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && java.sql.Date.class.isInstance(msgValue.getClass())) { - return (java.sql.Date)msgValue; + return (java.sql.Date) msgValue; } - return super.getSqlDate(key); + return (java.sql.Date) super.getSqlDate(key); } @Override public Timestamp getTimestamp(String key) { Object msgValue = getBeanFieldOrJsonValue(messageValue, key); if (msgValue != null && Timestamp.class.isInstance(msgValue.getClass())) { - return (Timestamp)msgValue; + return (Timestamp) msgValue; } - return super.getTimestamp(key); + return (Timestamp) super.getTimestamp(key); } @Override @@ -458,7 +459,7 @@ protected String serializeMessageValue() { data = dataType.toDataJson(messageValue); type = "datatype"; } else if (isMap && JSONObject.class.isInstance(messageValue)) { - JSONObject jsonObject = (JSONObject)messageValue; + JSONObject jsonObject = (JSONObject) messageValue; data = jsonObject.toJSONString(); type = "json"; } else { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ArrayDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ArrayDataType.java index 71b5343d..dc88a416 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ArrayDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ArrayDataType.java @@ -44,6 +44,21 @@ public ArrayDataType() { } + public static String getTypeName() { + return "array"; + } + + /** + * @param args + */ + public static void main(String[] args) throws ClassNotFoundException { + String name = long.class.getName(); + System.out.println(name); + Class clazz = ReflectUtil.forClass(name); +// Float[] strs = new Float[10]; +// ArrayDataType arrayDataType =(ArrayDataType) DataTypeUtil.getDataTypeFromClass(strs.getClass()); + } + @Override public String toDataJson(Object objectArray) { JSONArray jsonArray = new JSONArray(); @@ -106,10 +121,6 @@ public String getName() { return "array"; } - public static String getTypeName() { - return "array"; - } - @Override public String getDataTypeName() { return "array"; @@ -251,17 +262,6 @@ public DataType create() { return new ArrayDataType(); } - /** - * @param args - */ - public static void main(String[] args) throws ClassNotFoundException { - String name = long.class.getName(); - System.out.println(name); - Class clazz = ReflectUtil.forClass(name); -// Float[] strs = new Float[10]; -// ArrayDataType arrayDataType =(ArrayDataType) DataTypeUtil.getDataTypeFromClass(strs.getClass()); - } - @Override public List convert(Object objectArray) { if (objectArray == null) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/BaseDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/BaseDataType.java index 7d170de6..2b7c7d78 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/BaseDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/BaseDataType.java @@ -21,17 +21,15 @@ import java.io.Serializable; import java.io.UnsupportedEncodingException; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.utils.NumberUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public abstract class BaseDataType implements DataType, Serializable { - private static final long serialVersionUID = 987223978957770805L; - protected final static String CODE = "UTF-8"; - - private static final Log LOG = LogFactory.getLog(BaseDataType.class); + private static final long serialVersionUID = 987223978957770805L; + private static final Logger LOG = LoggerFactory.getLogger(BaseDataType.class); /** * 数据类型 @@ -96,7 +94,7 @@ protected Class createClass(String className) { @Override public T convert(Object object) { - return (T)object; + return (T) object; } @Override @@ -184,7 +182,7 @@ protected byte[] numberToBytes(Long number) { protected byte[] createByteArrayFromNumber(long value, int byteCount) { byte[] byteRet = new byte[byteCount]; for (int i = 0; i < byteCount; i++) { - byteRet[i] = (byte)((value >> 8 * i) & 0xff); + byteRet[i] = (byte) ((value >> 8 * i) & 0xff); } return byteRet; } @@ -199,17 +197,17 @@ protected Long createNumberValue(byte[] arr, int offset, int size) { } long value = 0; for (int i = offset; i < offset + size; i++) { - value |= ((long)(arr[i] & 0xff)) << (8 * (i - offset)); + value |= ((long) (arr[i] & 0xff)) << (8 * (i - offset)); } return value; } protected long getMinValue(int byteLength) { - return (long)Math.pow(2, byteLength); + return (long) Math.pow(2, byteLength); } protected long getMaxValue(int byteLength) { - long value = (long)Math.pow(2, byteLength); + long value = (long) Math.pow(2, byteLength); return (value - 1); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/BooleanDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/BooleanDataType.java index a351a691..2610b6b3 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/BooleanDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/BooleanDataType.java @@ -34,6 +34,10 @@ public BooleanDataType() { setDataClazz(Boolean.class); } + public static String getTypeName() { + return boolean.class.getSimpleName(); + } + @Override public String toDataJson(Boolean value) { if (value == null) { @@ -56,10 +60,6 @@ public Boolean getData(String jsonValue) { return null; } - public static String getTypeName() { - return boolean.class.getSimpleName(); - } - public boolean convertValue(Boolean value) { if (value == null) { return false; @@ -83,10 +83,10 @@ public Boolean convert(Object object) { return false; } if (Boolean.class.isInstance(object)) { - return (Boolean)object; + return (Boolean) object; } if (String.class.isInstance(object)) { - return Boolean.valueOf((String)object); + return Boolean.valueOf((String) object); } Integer value = Integer.valueOf(object.toString()); return value == 1 ? true : false; @@ -117,7 +117,7 @@ public byte[] toBytes(Boolean value, boolean isCompress) { if (value == null) { return null; } - byte bytes = value ? (byte)1 : (byte)0; + byte bytes = value ? (byte) 1 : (byte) 0; return createByteArrayFromNumber(bytes, 1); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ByteDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ByteDataType.java index e24c7d32..8a74d48d 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ByteDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ByteDataType.java @@ -31,6 +31,10 @@ public ByteDataType() { setDataClazz(Byte.class); } + public static String getTypeName() { + return "byte"; + } + @Override public String toDataJson(Byte value) { if (value == null) { @@ -47,10 +51,6 @@ public Byte getData(String jsonValue) { return Byte.valueOf(jsonValue); } - public static String getTypeName() { - return "byte"; - } - @Override protected void setFieldValueToJson(JSONObject jsonObject) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ConfigurableDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ConfigurableDataType.java index 346697d4..4cf79518 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ConfigurableDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ConfigurableDataType.java @@ -18,10 +18,10 @@ import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.JSONObject; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.configurable.IConfigurable; import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ConfigurableDataType extends BaseDataType { @@ -32,7 +32,7 @@ public class ConfigurableDataType extends BaseDataType { private static final String VALUE_KEY = "configurable_value"; - private static final Log LOG = LogFactory.getLog(ConfigurableDataType.class); + private static final Logger LOG = LoggerFactory.getLogger(ConfigurableDataType.class); public ConfigurableDataType(Class clazz) { setDataClazz(clazz); @@ -42,6 +42,10 @@ public ConfigurableDataType() { } + public static String getTypeName() { + return "configurable"; + } + @Override public String toDataJson(IConfigurable value) { if (value == null) { @@ -79,7 +83,7 @@ public IConfigurable getData(String jsonValue) { */ private IConfigurable createConfigurable(String className) { try { - return (IConfigurable)Class.forName(className).newInstance(); + return (IConfigurable) Class.forName(className).newInstance(); } catch (Exception e) { e.printStackTrace(); return null; @@ -106,10 +110,6 @@ protected Class[] getSupportClass() { return null; } - public static String getTypeName() { - return "configurable"; - } - @Override public boolean matchClass(Class clazz) { return IConfigurable.class.isAssignableFrom(clazz); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/DateDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/DateDataType.java index 0bb22a6c..1e5de86a 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/DateDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/DateDataType.java @@ -24,14 +24,14 @@ import java.time.ZoneId; import java.util.Date; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.utils.NumberUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class DateDataType extends BaseDataType { private static final long serialVersionUID = 3745784172508184101L; - private static final Log LOG = LogFactory.getLog(DateDataType.class); + private static final Logger LOG = LoggerFactory.getLogger(DateDataType.class); public DateDataType(Class clazz) { setDataClazz(clazz); @@ -41,15 +41,20 @@ public DateDataType() { setDataClazz(Date.class); } - @Override - public String toDataJson(Date value) { + public static void main(String[] args) { + new DateDataType().getData("2017-03-16 16:53:13"); + } + + public static String getTypeName() { + return "date"; + } + + @Override public String toDataJson(Date value) { SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); - String dateString = sdf.format(value); - return dateString; + return sdf.format(value); } - @Override - public Date getData(String jsonValue) { + @Override public Date getData(String jsonValue) { if (jsonValue == null || "N/A".equals(jsonValue)) { return null; } @@ -71,33 +76,22 @@ public Date getData(String jsonValue) { } } - public static void main(String[] args) { - new DateDataType().getData("2017-03-16 16:53:13"); - } - - @Override - public Class getDataClazz() { + @Override public Class getDataClazz() { return Date.class; } - @Override - public Date convert(Object object) { + @Override public Date convert(Object object) { if (object == null) { return null; } - if (Timestamp.class.isInstance(object)) { + if (object instanceof Timestamp) { return new Date(((Timestamp) object).getTime()); } if (object instanceof LocalDateTime) { LocalDateTime tempTime = (LocalDateTime) object; return Date.from(tempTime.atZone(ZoneId.systemDefault()).toInstant()); } - Date convert = (Date) super.convert(object); - return convert; - } - - public static String getTypeName() { - return "date"; + return super.convert(object); } public String toDataJson(Timestamp timestamp) { @@ -105,49 +99,44 @@ public String toDataJson(Timestamp timestamp) { return toDataJson(value); } - @Override - public boolean matchClass(Class clazz) { + @Override public boolean matchClass(Class clazz) { if (Timestamp.class.isAssignableFrom(clazz)) { return true; } + if (LocalDateTime.class.isAssignableFrom(clazz)) { + return true; + } return Date.class.isAssignableFrom(clazz); } - @Override - protected Class[] getSupportClass() { + @Override protected Class[] getSupportClass() { return new Class[] {Timestamp.class, Date.class}; } - @Override - public DataType create() { + @Override public DataType create() { return this; } - @Override - public String getDataTypeName() { + @Override public String getDataTypeName() { return getTypeName(); } - @Override - protected void setFieldValueToJson(JSONObject jsonObject) { + @Override protected void setFieldValueToJson(JSONObject jsonObject) { } - @Override - protected void setFieldValueFromJson(JSONObject jsonObject) { + @Override protected void setFieldValueFromJson(JSONObject jsonObject) { } - @Override - public byte[] toBytes(Date value, boolean isCompress) { + @Override public byte[] toBytes(Date value, boolean isCompress) { if (value == null) { return null; } return createByteArrayFromNumber(value.getTime(), 8); } - @Override - public Date byteToValue(byte[] bytes) { + @Override public Date byteToValue(byte[] bytes) { Long value = createNumberValue(bytes); if (value == null) { return null; @@ -155,8 +144,7 @@ public Date byteToValue(byte[] bytes) { return new Date(value); } - @Override - public Date byteToValue(byte[] bytes, int offset) { + @Override public Date byteToValue(byte[] bytes, int offset) { byte[] bytesArray = NumberUtils.getSubByteFromIndex(bytes, offset, 8); return byteToValue(bytesArray); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/DateTimeDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/DateTimeDataType.java index 0cbc988b..e1410a17 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/DateTimeDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/DateTimeDataType.java @@ -21,7 +21,7 @@ /** * @description */ -public class DateTimeDataType extends DateDataType{ +public class DateTimeDataType extends DateDataType { public DateTimeDataType(Class clazz) { setDataClazz(clazz); @@ -31,13 +31,13 @@ public DateTimeDataType() { setDataClazz(Date.class); } + public static String getTypeName() { + return "datetime"; + } + @Override public String getDataTypeName() { return getTypeName(); } - public static String getTypeName() { - return "datetime"; - } - } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/DoubleDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/DoubleDataType.java index ae3b2bfa..14531bc4 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/DoubleDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/DoubleDataType.java @@ -35,6 +35,14 @@ public DoubleDataType() { setDataClazz(Double.class); } + public static String getTypeName() { + return "double"; + } + + public static void main(String[] args) { + System.out.println(double.class.getSimpleName()); + } + @Override public Double getData(String jsonValue) { if (jsonValue == null || "N/A".equals(jsonValue)) { @@ -48,14 +56,6 @@ public String getName() { return double.class.getSimpleName(); } - public static String getTypeName() { - return "double"; - } - - public static void main(String[] args) { - System.out.println(double.class.getSimpleName()); - } - @Override public String toDataJson(Double value) { if (value == null) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/FloatDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/FloatDataType.java index c68e840f..b96f76ac 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/FloatDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/FloatDataType.java @@ -31,6 +31,30 @@ public FloatDataType() { setDataClazz(Float.class); } + public static String getTypeName() { + return "float"; + } + + public static void main(String[] args) { + FloatDataType fdt = new FloatDataType(); + float f1 = 3.2f; + byte[] b1 = fdt.toBytes(f1, true); + byte[] b2 = fdt.toBytes(f1, false); + float f11 = fdt.byteToValue(b1); + float f12 = fdt.byteToValue(b2); + System.out.println(f11); + System.out.println(f12); + + DoubleDataType dt = new DoubleDataType(); + double d1 = 3.2d; + byte[] db1 = dt.toBytes(d1, true); + byte[] db2 = dt.toBytes(d1, false); + double d11 = dt.byteToValue(db1); + double d12 = dt.byteToValue(db2); + System.out.println(d11); + System.out.println(d12); + } + @Override public Float getData(String jsonValue) { if (jsonValue == null || "N/A".equals(jsonValue)) { @@ -44,10 +68,6 @@ public String getName() { return float.class.getSimpleName(); } - public static String getTypeName() { - return "float"; - } - @Override public String toDataJson(Float value) { return String.valueOf(value); @@ -117,24 +137,4 @@ public Float byteToValue(byte[] bytes, AtomicInteger offset) { offset.set(index + 8); return byteToValue(bytesArray); } - - public static void main(String[] args) { - FloatDataType fdt = new FloatDataType(); - float f1 = 3.2f; - byte[] b1 = fdt.toBytes(f1, true); - byte[] b2 = fdt.toBytes(f1, false); - float f11 = fdt.byteToValue(b1); - float f12 = fdt.byteToValue(b2); - System.out.println(f11); - System.out.println(f12); - - DoubleDataType dt = new DoubleDataType(); - double d1 = 3.2d; - byte[] db1 = dt.toBytes(d1, true); - byte[] db2 = dt.toBytes(d1, false); - double d11 = dt.byteToValue(db1); - double d12 = dt.byteToValue(db2); - System.out.println(d11); - System.out.println(d12); - } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/GenericParamterDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/GenericParamterDataType.java index cf51e983..618ee714 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/GenericParamterDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/GenericParamterDataType.java @@ -20,7 +20,6 @@ import com.alibaba.fastjson.JSONObject; import org.apache.rocketmq.streams.common.utils.StringUtil; - public abstract class GenericParamterDataType extends BaseDataType { protected String genericParamterStr; @@ -33,6 +32,7 @@ protected void setFieldValueToJson(JSONObject jsonObject) { jsonObject.put(GENERIC_PARAMETER, genericParamterStr); } } + public abstract String toDataStr(T t); @Override diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/HllDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/HllDataType.java index f0ab52be..8bd25c44 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/HllDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/HllDataType.java @@ -34,6 +34,15 @@ public HllDataType() { setDataClazz(HLL.class); } + public static void main(String[] args) { + HLL hll = new HLL(30, 8); + hll.addRaw(123456); + HllDataType dataType = new HllDataType(); + String content = dataType.toDataJson(hll); + HLL copy = dataType.getData(content); + System.out.println(copy.cardinality()); + } + @Override protected void setFieldValueToJson(JSONObject jsonObject) { } @@ -67,13 +76,4 @@ public HllDataType() { return null; } - public static void main(String[] args) { - HLL hll = new HLL(30, 8); - hll.addRaw(123456); - HllDataType dataType = new HllDataType(); - String content = dataType.toDataJson(hll); - HLL copy = dataType.getData(content); - System.out.println(copy.cardinality()); - } - } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/IJsonable.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/IJsonable.java index 29919878..d3594cc4 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/IJsonable.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/IJsonable.java @@ -21,14 +21,14 @@ public interface IJsonable { /** * 实现接口的对象序列化为json * - * @return + * @return 返回json字符串 */ String toJson(); /** * 根据json,填充对象的成员变量 * - * @param jsonString + * @param jsonString json字符串 */ void toObject(String jsonString); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/IntDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/IntDataType.java index 87a3f464..99155aad 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/IntDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/IntDataType.java @@ -31,6 +31,10 @@ public IntDataType() { setDataClazz(Integer.class); } + public static String getTypeName() { + return "int"; + } + @Override public Integer getData(String jsonValue) { if (jsonValue == null || "N/A".equals(jsonValue)) { @@ -45,10 +49,6 @@ public String getName() { return int.class.getSimpleName(); } - public static String getTypeName() { - return "int"; - } - @Override public String toDataJson(Integer value) { if (value == null) { @@ -97,7 +97,7 @@ protected void setFieldValueFromJson(JSONObject jsonObject) { @Override public byte[] toBytes(Integer value, boolean isCompress) { if (isCompress) { - return numberToBytes((long)value); + return numberToBytes((long) value); } else { return createByteArrayFromNumber(value, 4); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/JavaBeanDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/JavaBeanDataType.java index ed657fd7..b2d383f9 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/JavaBeanDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/JavaBeanDataType.java @@ -37,9 +37,9 @@ public class JavaBeanDataType extends BaseDataType { - protected static ICache cache = new SoftReferenceCache<>(); - private static final long serialVersionUID = 8749848859175999778L; + protected static ICache cache = new SoftReferenceCache<>(); + protected static ICache metaMap = new SoftReferenceCache<>(); public JavaBeanDataType(Class clazz) { setDataClazz(clazz); @@ -49,114 +49,10 @@ public JavaBeanDataType() { } - @Override - public String toDataJson(Object value) { - if (value == null) { - return null; - } - byte[] bytes = serializeJavabean(value); - return Base64Utils.encode(bytes); - } - - @Override - public Object getData(String jsonValue) { - if (StringUtil.isEmpty(jsonValue)) { - return null; - } - byte[] bytes = Base64Utils.decode(jsonValue); - return deserializeJavaBean(bytes); - } - - @Override - public String getName() { - return "javaBean"; - } - public static String getTypeName() { return "javaBean"; } - @Override - public boolean matchClass(Class clazz) { - - AtomicBoolean atomicBoolean = cache.get(clazz.getName()); - if (atomicBoolean != null) { - return atomicBoolean.get(); - } - atomicBoolean = new AtomicBoolean(true); - - AtomicBoolean success = atomicBoolean; - - ReflectUtil.scanFields(clazz, new IFieldProcessor() { - @Override - public void doProcess(Object o, Field field) { - field.setAccessible(true); - DataType dataType = DataTypeUtil.createDataTypeByField(o, field); - - if (dataType == null) { - success.set(false); - } - if (!success.get()) { - return; - } - } - }); - cache.put(clazz.getName(), success); - return success.get(); - } - - @Override - public DataType create() { - return new JavaBeanDataType(); - } - - @Override - public String getDataTypeName() { - return getTypeName(); - } - - @Override public byte[] toBytes(Object value, boolean isCompress) { - if (value == null) { - return null; - } - - byte[] result = serializeJavabean(value); - - byte[] bytes = new byte[result.length + 2]; - byte[] lenBytes = createByteArrayFromNumber(result.length, 2); - bytes[0] = lenBytes[0]; - bytes[1] = lenBytes[1]; - for (int i = 2; i < bytes.length; i++) { - bytes[i] = result[i - 2]; - } - return bytes; - } - - @Override public Object byteToValue(byte[] bytes) { - byte[] valueBytes = new byte[bytes.length - 2]; - for (int i = 0; i < valueBytes.length; i++) { - valueBytes[i] = bytes[i + 2]; - } - return deserializeJavaBean(valueBytes); - } - - @Override - protected void setFieldValueToJson(JSONObject jsonObject) { - - } - - @Override - protected void setFieldValueFromJson(JSONObject jsonObject) { - - } - - @Override - protected Class[] getSupportClass() { - return new Class[] {dataClazz}; - } - - protected static ICache metaMap = new SoftReferenceCache<>(); - /** * 把一个对象序列化成字节,对象中的字段是datatype支持的 * @@ -342,6 +238,84 @@ public void doProcess(Object o, Field field) { return (T) object; } + @Override + public String toDataJson(Object value) { + if (value == null) { + return null; + } + byte[] bytes = serializeJavabean(value); + return Base64Utils.encode(bytes); + } + + @Override + public Object getData(String jsonValue) { + if (StringUtil.isEmpty(jsonValue)) { + return null; + } + byte[] bytes = Base64Utils.decode(jsonValue); + return deserializeJavaBean(bytes); + } + + @Override + public String getName() { + return "javaBean"; + } + + @Override + public boolean matchClass(Class clazz) { + return true; + } + + @Override + public DataType create() { + return new JavaBeanDataType(); + } + + @Override + public String getDataTypeName() { + return getTypeName(); + } + + @Override public byte[] toBytes(Object value, boolean isCompress) { + if (value == null) { + return null; + } + + byte[] result = serializeJavabean(value); + + byte[] bytes = new byte[result.length + 2]; + byte[] lenBytes = createByteArrayFromNumber(result.length, 2); + bytes[0] = lenBytes[0]; + bytes[1] = lenBytes[1]; + for (int i = 2; i < bytes.length; i++) { + bytes[i] = result[i - 2]; + } + return bytes; + } + + @Override public Object byteToValue(byte[] bytes) { + byte[] valueBytes = new byte[bytes.length - 2]; + for (int i = 0; i < valueBytes.length; i++) { + valueBytes[i] = bytes[i + 2]; + } + return deserializeJavaBean(valueBytes); + } + + @Override + protected void setFieldValueToJson(JSONObject jsonObject) { + + } + + @Override + protected void setFieldValueFromJson(JSONObject jsonObject) { + + } + + @Override + protected Class[] getSupportClass() { + return new Class[] {dataClazz}; + } + protected static class ClassMeta { Map dataTypeMap = new HashMap<>(); Map useGetMethod = new HashMap<>(); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/JsonableDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/JsonableDataType.java index f0a0af92..7db2161e 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/JsonableDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/JsonableDataType.java @@ -18,7 +18,8 @@ import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.JSONObject; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; +import org.apache.rocketmq.streams.common.configurable.IConfigurable; import org.apache.rocketmq.streams.common.utils.ReflectUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; @@ -34,12 +35,16 @@ public JsonableDataType() { } + public static String getTypeName() { + return "jsonable"; + } + @Override public String toDataJson(Object value) { - IJsonable jsonable = (IJsonable)value; + IJsonable jsonable = (IJsonable) value; String msg = jsonable.toJson(); JSONObject jsonObject = JSON.parseObject(msg); - jsonObject.put(IConfigurableService.CLASS_NAME, value.getClass().getName()); + jsonObject.put(BasedConfigurable.CLASS_NAME, value.getClass().getName()); return jsonObject.toJSONString(); } @@ -51,8 +56,8 @@ public Object getData(String jsonValue) { IJsonable jsonable = null; try { JSONObject jsonObject = JSON.parseObject(jsonValue); - String className = jsonObject.getString(IConfigurableService.CLASS_NAME); - jsonable = (IJsonable)ReflectUtil.forInstance(className); + String className = jsonObject.getString(IConfigurable.CLASS_NAME); + jsonable = (IJsonable) ReflectUtil.forInstance(className); jsonable.toObject(jsonValue); return jsonable; } catch (Exception e) { @@ -83,10 +88,6 @@ public String getName() { return "jsonable"; } - public static String getTypeName() { - return "jsonable"; - } - @Override protected void setFieldValueToJson(JSONObject jsonObject) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ListDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ListDataType.java index fd5d2c85..bec182f2 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ListDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ListDataType.java @@ -51,6 +51,17 @@ public ListDataType() { setDataClazz(List.class); } + public static String getTypeName() { + return "list"; + } + + public static void main(String[] args) { + ListDataType listDataType = new ListDataType(new StringDataType()); + List list = listDataType.getData("[\"fdsdfds\",\"dfs\"]"); + list = listDataType.getData(listDataType.toDataStr(list)); + System.out.println(listDataType.toDataStr(list)); + } + @Override public String toDataStr(List value) { StringBuilder stringBuilder = new StringBuilder(); @@ -160,10 +171,6 @@ protected boolean isQuickModel(String jsonValue) { return !jsonValue.trim().startsWith("{") && !jsonValue.trim().startsWith("["); } - public static String getTypeName() { - return "set"; - } - @Override public boolean matchClass(Class clazz) { return List.class.isAssignableFrom(clazz); @@ -274,11 +281,8 @@ protected String createGenericParameterStr() { return List.class.getName() + "<" + subStr + ">"; } - public static void main(String[] args) { - ListDataType listDataType = new ListDataType(new StringDataType()); - List list = listDataType.getData("[\"fdsdfds\",\"dfs\"]"); - list = listDataType.getData(listDataType.toDataStr(list)); - System.out.println(listDataType.toDataStr(list)); + public DataType getParadigmType() { + return paradigmType; } public void setParadigmType(DataType paradigmType) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/LongDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/LongDataType.java index 45c6cce1..1605e460 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/LongDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/LongDataType.java @@ -37,6 +37,10 @@ public LongDataType() { setDataClazz(Long.class); } + public static String getTypeName() { + return "long"; + } + @Override public Long getData(String jsonValue) { if (jsonValue == null || "N/A".equals(jsonValue)) { @@ -104,23 +108,11 @@ public Long convert(Object object) { return super.convert(object); } - @Override - public void setDataClazz(Class dataClazz) { - if (BigInteger.class.isAssignableFrom(dataClazz)) { - dataClazz = Long.class; - } - super.setDataClazz(dataClazz); - } - @Override public String getName() { return Long.class.getSimpleName(); } - public static String getTypeName() { - return "long"; - } - public String toDataJson(BigInteger bigInteger) { Long value = convert(bigInteger); return toDataJson(value); @@ -131,6 +123,14 @@ public Class getDataClazz() { return Long.class; } + @Override + public void setDataClazz(Class dataClazz) { + if (BigInteger.class.isAssignableFrom(dataClazz)) { + dataClazz = Long.class; + } + super.setDataClazz(dataClazz); + } + @Override public String toDataJson(Long value) { if (value == null) { @@ -148,11 +148,11 @@ public long convertValue(Long value) { @Override public byte[] toBytes(Long value, boolean isCompress) { - if(value==null){ + if (value == null) { return null; } if (isCompress) { - return numberToBytes((long)value); + return numberToBytes((long) value); } else { return createByteArrayFromNumber(value, 8); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/MapDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/MapDataType.java index fd74dec4..de5c7a69 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/MapDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/MapDataType.java @@ -54,14 +54,25 @@ public MapDataType(DataType keyParadigmType, DataType valueParadigmType) { this.setGenericParameterStr(createGenericParameterStr()); } + public MapDataType() { + } + + public static String getTypeName() { + return "map"; + } + + public static void main(String[] args) { + MapDataType mapDataType = new MapDataType(new StringDataType(), new StringDataType()); + Map values = mapDataType.getData("'d,f':d,'fd:f':'d',e:f"); + PrintUtil.print(values); + + } + @Override public void setDataClazz(Class dataClazz) { this.dataClazz = Map.class; } - public MapDataType() { - } - @Override public String toDataJson(Map value) { JSONArray mapJson = new JSONArray(); @@ -222,10 +233,6 @@ public String getName() { return Map.class.getSimpleName(); } - public static String getTypeName() { - return "kv"; - } - @Override public void parseGenericParameter(String genericParameterString) { if (StringUtil.isEmpty(genericParameterString)) { @@ -345,17 +352,18 @@ public String getDataTypeName() { return getTypeName(); } - public static void main(String[] args) { - MapDataType mapDataType = new MapDataType(new StringDataType(), new StringDataType()); - Map values = mapDataType.getData("'d,f':d,'fd:f':'d',e:f"); - PrintUtil.print(values); - + public DataType getKeyParadigmType() { + return keyParadigmType; } public void setKeyParadigmType(DataType keyParadigmType) { this.keyParadigmType = keyParadigmType; } + public DataType getValueParadigmType() { + return valueParadigmType; + } + public void setValueParadigmType(DataType valueParadigmType) { this.valueParadigmType = valueParadigmType; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/SetDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/SetDataType.java index 83dda43e..aae21f2e 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/SetDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/SetDataType.java @@ -54,6 +54,10 @@ public SetDataType() { } + public static String getTypeName() { + return "set"; + } + @Override public String toDataStr(Set value) { StringBuilder stringBuilder = new StringBuilder(); @@ -215,10 +219,6 @@ protected boolean isQuickModel(String jsonValue) { return false; } - public static String getTypeName() { - return "list"; - } - @Override public boolean matchClass(Class clazz) { return Set.class.isAssignableFrom(clazz); @@ -272,6 +272,10 @@ protected String createGenericParameterStr() { return List.class.getName() + "<" + subStr + ">"; } + public DataType getParadigmType() { + return paradigmType; + } + public void setParadigmType(DataType paradigmType) { this.paradigmType = paradigmType; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ShortDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ShortDataType.java index 63a3391c..cf7c7715 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ShortDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ShortDataType.java @@ -34,6 +34,10 @@ public ShortDataType() { setDataClazz(Short.class); } + public static String getTypeName() { + return "short"; + } + @Override public String toDataJson(Short value) { if (value == null) { @@ -85,14 +89,10 @@ public String getName() { return Short.class.getSimpleName(); } - public static String getTypeName() { - return "short"; - } - @Override public byte[] toBytes(Short value, boolean isCompress) { if (isCompress) { - return numberToBytes((long)value); + return numberToBytes((long) value); } else { return createByteArrayFromNumber(value, 2); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/StringDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/StringDataType.java index f504498d..25d18e9a 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/StringDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/StringDataType.java @@ -32,6 +32,10 @@ public StringDataType(Class clazz) { setDataClazz(clazz); } + public static String getTypeName() { + return "string"; + } + @Override public String getData(String jsonValue) { return jsonValue; @@ -42,10 +46,6 @@ public String getName() { return String.class.getSimpleName(); } - public static String getTypeName() { - return "string"; - } - @Override public String toDataJson(String value) { return value; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/dboperator/IDBDriver.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/dboperator/IDBDriver.java index 927a2c23..1d3bd19f 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/dboperator/IDBDriver.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/dboperator/IDBDriver.java @@ -40,6 +40,22 @@ public interface IDBDriver { */ void execute(String sql); + /** + * 以预编译的方式执行sql + * + * @param sql 含参数的sql + * @param params 参数 + */ + int execute(String sql, Object[] params); + + /** + * 以预编译的方式执行查询 + * + * @param sql 含参数的sql + * @param params 参数 + */ + List> executeQuery(String sql, Object[] params); + /** * 查询多行数据 * @@ -64,9 +80,9 @@ public interface IDBDriver { */ long executeInsert(final String sql); - void executSqls(String... sqls); + void executeSqls(String... sqls); - void executSqls(Collection sqls); + void executeSqls(Collection sqls); /** * 分批查询数据,每次批次是batchSize,避免一次加载数据太多 diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/disruptor/DisruptorProducer.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/disruptor/DisruptorProducer.java index 830bf549..092156ef 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/disruptor/DisruptorProducer.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/disruptor/DisruptorProducer.java @@ -19,11 +19,11 @@ import com.lmax.disruptor.InsufficientCapacityException; import com.lmax.disruptor.RingBuffer; import com.lmax.disruptor.dsl.Disruptor; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class DisruptorProducer { - private static final Log LOG = LogFactory.getLog(DisruptorProducer.class); + private static final Logger LOG = LoggerFactory.getLogger(DisruptorProducer.class); private Disruptor disruptor; public DisruptorProducer(Disruptor disruptor) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/enums/StageType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/enums/StageType.java new file mode 100644 index 00000000..bd4a48a2 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/enums/StageType.java @@ -0,0 +1,52 @@ +package org.apache.rocketmq.streams.common.enums; + +import org.apache.rocketmq.streams.common.topology.model.AbstractStage; +import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; +import org.apache.rocketmq.streams.common.topology.stages.JoinEndChainStage; +import org.apache.rocketmq.streams.common.topology.stages.JoinStartChainStage; +import org.apache.rocketmq.streams.common.topology.stages.OutputChainStage; +import org.apache.rocketmq.streams.common.topology.stages.ScriptChainStage; +import org.apache.rocketmq.streams.common.topology.stages.SynchronousWindowChainStage; +import org.apache.rocketmq.streams.common.topology.stages.UnionEndChainStage; +import org.apache.rocketmq.streams.common.topology.stages.UnionStartChainStage; + +public enum StageType { + + SOURCE("source"), + OUTPUT("output"), + FILTER("filter"), + SCRIPT("script"), + JOIN("join"), + WINDOW("window"), + UNION("union"), + UNKNOWN("unknown"), + ; + + private final String type; + + StageType(String type) { + this.type = type; + } + + public static String getTypeForStage(AbstractStage stage) { + if (stage instanceof FilterChainStage) { + return FILTER.getType(); + } else if (stage instanceof ScriptChainStage) { + return SCRIPT.getType(); + } else if (stage instanceof JoinStartChainStage || stage instanceof JoinEndChainStage) { + return JOIN.getType(); + } else if (stage instanceof SynchronousWindowChainStage) { + return WINDOW.getType(); + } else if (stage instanceof OutputChainStage) { + return OUTPUT.getType(); + } else if (stage instanceof UnionStartChainStage || stage instanceof UnionEndChainStage) { + return UNION.getType(); + } else { + return UNKNOWN.getType(); + } + } + + public String getType() { + return type; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/exception/CanNotSupportThisMethod.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/exception/CanNotSupportThisMethod.java new file mode 100644 index 00000000..be95a3f5 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/exception/CanNotSupportThisMethod.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.exception; + +public class CanNotSupportThisMethod extends Exception { + public CanNotSupportThisMethod() { + this("can not support this method"); + + } + + public CanNotSupportThisMethod(String message) { + super(message); + } + + public CanNotSupportThisMethod(String message, Throwable e) { + super(message, e); + } + +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/functions/FlatMapFunction.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/functions/FlatMapFunction.java index 87802b9e..2c2307e9 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/functions/FlatMapFunction.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/functions/FlatMapFunction.java @@ -19,7 +19,7 @@ import java.io.Serializable; import java.util.List; -public interface FlatMapFunction extends Function, Serializable { +public interface FlatMapFunction extends Function, Serializable { List flatMap(IN message) throws Exception; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/functions/Function.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/functions/Function.java index f6e283c3..a4e014c4 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/functions/Function.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/functions/Function.java @@ -16,4 +16,5 @@ */ package org.apache.rocketmq.streams.common.functions; -public interface Function extends java.io.Serializable {} +public interface Function extends java.io.Serializable { +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/functions/ReduceFunction.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/functions/ReduceFunction.java index d6c1c686..a79fa996 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/functions/ReduceFunction.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/functions/ReduceFunction.java @@ -16,7 +16,7 @@ */ package org.apache.rocketmq.streams.common.functions; -public interface ReduceFunction extends Function{ +public interface ReduceFunction extends Function { R reduce(R acccumulator, O msg); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IBaseStreamOperator.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IBaseStreamOperator.java index e8809ea6..3a0d65ba 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IBaseStreamOperator.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IBaseStreamOperator.java @@ -24,8 +24,8 @@ public interface IBaseStreamOperator> matchExpression(String expressionStr, JSONObject msg, boolean needAll, + String script); + + Map matchExpression(String expressionStr, JSONObject msg); + + List> matchExpression(String msgFieldName, String dimFieldName, JSONObject msg); + + List getIndexs(); + + String addIndex(String... indexs); + + void startLoadDimData(); + +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IFilterService.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IFilterService.java index 9758852c..9b4591a6 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IFilterService.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IFilterService.java @@ -42,7 +42,14 @@ public interface IFilterService { */ List executeRule(IMessage message, AbstractContext context, T... rules); - + /** + * 执行规则,返回触发的规则 + * + * @param message 消息 + * @param rules 规则 + * @return 触发的规则 + */ + List executeRule(IMessage message, AbstractContext context, T rules); /** * 执行规则,返回触发的规则 @@ -52,6 +59,7 @@ public interface IFilterService { * @return 触发的规则 */ List executeRule(IMessage message, AbstractContext context, List rules); + /** * create rule 根据表达式,创建一个规则 * diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/INeedAalignWaiting.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/INeedAalignWaiting.java new file mode 100644 index 00000000..be6df852 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/INeedAalignWaiting.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.interfaces; + +import org.apache.rocketmq.streams.common.context.IMessage; + +/** + * 需要对齐等待,主要是join场景 + */ +public interface INeedAalignWaiting { + boolean alignWaiting(IMessage msg); +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IPropertyEnable.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IPropertyEnable.java index a2adc165..1eb4bb42 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IPropertyEnable.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IPropertyEnable.java @@ -24,15 +24,15 @@ */ public interface IPropertyEnable { - void initProperty(Properties properties); - static T create(String className, Properties properties) { IPropertyEnable propertyEnable = ReflectUtil.forInstance(className); propertyEnable.initProperty(properties); - return (T)propertyEnable; + return (T) propertyEnable; } static T create(Class claszz, Properties properties) { return create(claszz.getName(), properties); } + + void initProperty(Properties properties); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/ISerialize.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/ISerialize.java index a675f382..0a9d2b9f 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/ISerialize.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/ISerialize.java @@ -18,5 +18,4 @@ public interface ISerialize { - } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IStage.java new file mode 100644 index 00000000..0e4c58cb --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IStage.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.interfaces; + +import org.apache.rocketmq.streams.common.context.AbstractContext; +import org.apache.rocketmq.streams.common.context.IMessage; + +public interface IStage extends ISystemMessageProcessor, IStreamOperator { + + boolean executeStage(T t, AbstractContext context); +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IStreamOperator.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IStreamOperator.java index 7d5cd564..d7ebecaa 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IStreamOperator.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IStreamOperator.java @@ -22,7 +22,7 @@ /** * 处理一条消息 */ -public interface IStreamOperator extends IBaseStreamOperator { +public interface IStreamOperator extends IBaseStreamOperator> { String TYPE = "receiver"; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/ISystemMessage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/ISystemMessage.java index 2ce4361b..22756c8c 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/ISystemMessage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/ISystemMessage.java @@ -17,4 +17,12 @@ package org.apache.rocketmq.streams.common.interfaces; public interface ISystemMessage { + int CHECK_POINT = 1; + int SPLIT_ADD = 2; + int SPLIT_REMOVE = 3; + int WATER_MARK = 4; + int BATCH_FINISH = 5; + int CHANGE_TABLE_NAME = 6; + + int getSystemMessageType(); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/ISystemMessageProcessor.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/ISystemMessageProcessor.java index 9a7ac9e8..27abf7ad 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/ISystemMessageProcessor.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/ISystemMessageProcessor.java @@ -16,8 +16,60 @@ */ package org.apache.rocketmq.streams.common.interfaces; -import org.apache.rocketmq.streams.common.channel.source.ISplitChangedListener; -import org.apache.rocketmq.streams.common.checkpoint.ICheckPoint; +import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; +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 ISystemMessageProcessor extends ICheckPoint, ISplitChangedListener,IBatchMessageFinishNotify { +public interface ISystemMessageProcessor { + + /** + * 当有新分片启动时,发系统消息,告诉所有算子,需要准备的,提前准备资源,比如窗口计算,在精确计算的场景,会把远程存储恢复到本地 + * 消息不会传递到shuffle侧,shuffle后端组件通过shuffle source通知 + * + * @param message + * @param context + */ + void addNewSplit(IMessage message, AbstractContext context, NewSplitMessage newSplitMessage); + + /** + * 当有分片移除时,发系统消息,告诉所有算子,刷新本地缓存。 + * 消息不会传递到shuffle侧,shuffle后端组件通过shuffle source通知 + * + * @param message + * @param context + */ + void removeSplit(IMessage message, AbstractContext context, RemoveSplitMessage removeSplitMessage); + + /** + * 当channel保存commit offset时,发送系统消息,收到系统消息的stage,需要刷新缓存。 + * 消息不会传递到shuffle侧,shuffle后端组件通过shuffle source通知 + * + * @param message + * @param context + */ + void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage); + + /** + * 当是批量消息时,消息完成后,可以通过发送BatchFinishMessage消息,通知所有算子完成计算,此时窗口也会提前触发 + * 消息会传递到shuffle侧,通过shuffle所有组件都会收到 + * + * @param message + * @param context + */ + void batchMessageFinish(IMessage message, AbstractContext context, BatchFinishMessage checkPointMessage); + + /** + * 系统正式启动前,发通知给stage,需要提前初始化资源的stage,可以完成资源初始化,如dim,可以开始加载维表数据 + * 消息不会传递到shuffle侧,shuffle后端组件通过shuffle source通知 + */ + void startJob(); + + /** + * 系统正式关闭前,发通知给stage,完成资源回收 + * 消息不会传递到shuffle侧,shuffle后端组件通过shuffle source通知 + */ + void stopJob(); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/schedule/IScheduleCondition.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/ITaskStart.java similarity index 87% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/schedule/IScheduleCondition.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/ITaskStart.java index 8c857820..b2a09b3c 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/schedule/IScheduleCondition.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/ITaskStart.java @@ -14,9 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.common.schedule; +package org.apache.rocketmq.streams.common.interfaces; -public interface IScheduleCondition { +public interface ITaskStart { - boolean canExecute(); + void startTask(); } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/IScriptUDFInit.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IUDF.java similarity index 90% rename from rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/IScriptUDFInit.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IUDF.java index 653f8bf9..4db6007a 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/IScriptUDFInit.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/IUDF.java @@ -14,12 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.script.service; +package org.apache.rocketmq.streams.common.interfaces; /** * 主要用于blink udf转化,blink udf最终会转化成一个script的子类,如果实现了这个接口。在init时,会调用映射的初始化方法名称 blink udf 映射的是open方法 */ -public interface IScriptUDFInit { +public interface IUDF { /** * 初始化的方法名 @@ -34,4 +34,8 @@ public interface IScriptUDFInit { * @return */ Object getInstance(); + + void loadUDF(); + + void destroy(); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/logger/LoggerOutputChannel.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/logger/LoggerOutputChannel.java index 34203e05..e35ff11e 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/logger/LoggerOutputChannel.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/logger/LoggerOutputChannel.java @@ -62,7 +62,7 @@ protected boolean batchInsert(List messages) { for (IMessage msg : messages) { String result = msg.getMessageValue().toString(); if (JSONObject.class.isInstance(msg.getMessageValue())) { - result = JsonableUtil.formatJson((JSONObject)msg.getMessageValue()); + result = JsonableUtil.formatJson((JSONObject) msg.getMessageValue()); } if (isFirst) { isFirst = false; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/logger/MyDailyRollingFileAppender.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/logger/MyDailyRollingFileAppender.java index 2933815b..8b93b2e5 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/logger/MyDailyRollingFileAppender.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/logger/MyDailyRollingFileAppender.java @@ -40,6 +40,10 @@ */ public class MyDailyRollingFileAppender extends FileAppender { + /** + * The gmtTimeZone is used only in computeCheckPeriod() method. + */ + public static final TimeZone gmtTimeZone = TimeZone.getTimeZone("GMT"); // The code assumes that the following constants are in a increasing // sequence. static final int TOP_OF_TROUBLE = -1; @@ -49,45 +53,31 @@ public class MyDailyRollingFileAppender extends FileAppender { static final int TOP_OF_DAY = 3; static final int TOP_OF_WEEK = 4; static final int TOP_OF_MONTH = 5; - - /** - * The gmtTimeZone is used only in computeCheckPeriod() method. - */ - public static final TimeZone gmtTimeZone = TimeZone.getTimeZone("GMT"); - + public int checkPeriod = TOP_OF_TROUBLE; /** * The default maximum file size is 20MB. */ protected long maxFileSize = 20 * 1024 * 1024; - /** * The date pattern. By default, the pattern is set to "'.'yyyy-MM-dd" meaning daily rollover. */ private String datePattern = "'.'yyyy-MM-dd"; - /** * There is one backup file by default. */ private int maxBackupIndex = 1; - /** * The log file will be renamed to the value of the scheduledFilename variable when the next interval is entered. For example, if the rollover period is one hour, the log file will be renamed to the value of "scheduledFilename" at the beginning of the next hour. The precise time when a rollover occurs depends on logging activity. */ private String scheduledFilename; - /** * The next time we estimate a rollover should occur. */ private long nextCheck = System.currentTimeMillis() - 1; - private Date now = new Date(); - private SimpleDateFormat sdf; - private MyRollingCalendar rollingCalendar = new MyRollingCalendar(); - public int checkPeriod = TOP_OF_TROUBLE; - /** * The default constructor does nothing. */ @@ -106,10 +96,36 @@ public MyDailyRollingFileAppender(Layout layout, String filename, String datePat } /** - * The DatePattern takes a string in the same format as expected by {@link SimpleDateFormat}. This options determines the rollover schedule. + * 测试方法 */ - public void setDatePattern(String pattern) { - datePattern = pattern; + public static void main(String args[]) { + String className = MyDailyRollingFileAppender.class.getName(); + + MyDailyRollingFileAppender myAppender = new MyDailyRollingFileAppender(); + + myAppender.setDatePattern("'.'yyyy-MM-dd-HH"); + myAppender.setFile("E:/test/mylogApender/test.log"); + myAppender.setMaxBackupIndex(3); + + PatternLayout patternLayout = new PatternLayout("%d [%-5p]-[%t]-[%c{1}] (%F:%L) %m%n"); + myAppender.setLayout(patternLayout); + + System.out.println("getMaxBackupIndex():" + myAppender.getMaxBackupIndex()); + + myAppender.activateOptions(); + + for (int i = 0; i < 50000; i++) { + myAppender.subAppend(new LoggingEvent(className, Logger.getLogger(className), Level.INFO, + "test,hello " + i, null)); + + try { + Thread.sleep(100); + } catch (InterruptedException ex) { + ex.printStackTrace(); + } + + System.out.println("Fine attesa"); + } } /** @@ -120,12 +136,10 @@ public String getDatePattern() { } /** - * 设置日志文件最大备份数 - *

- * The MaxBackupIndex option determines how many backup files are kept before the oldest is erased. This option takes a positive integer value. If set to zero, then there will be no backup files and the log file will be renamed to the value of the scheduledFilename variable when the next interval is entered. + * The DatePattern takes a string in the same format as expected by {@link SimpleDateFormat}. This options determines the rollover schedule. */ - public void setMaxBackupIndex(int maxBackups) { - this.maxBackupIndex = maxBackups; + public void setDatePattern(String pattern) { + datePattern = pattern; } /** @@ -135,6 +149,15 @@ public int getMaxBackupIndex() { return maxBackupIndex; } + /** + * 设置日志文件最大备份数 + *

+ * The MaxBackupIndex option determines how many backup files are kept before the oldest is erased. This option takes a positive integer value. If set to zero, then there will be no backup files and the log file will be renamed to the value of the scheduledFilename variable when the next interval is entered. + */ + public void setMaxBackupIndex(int maxBackups) { + this.maxBackupIndex = maxBackups; + } + @Override public void activateOptions() { super.activateOptions(); @@ -328,7 +351,7 @@ private void rollOverBySize() { now.setTime(currentTimeMillis); if (fileName != null && qw != null) { - long size = ((CountingQuietWriter)qw).getCount(); + long size = ((CountingQuietWriter) qw).getCount(); if (size >= maxFileSize) { // close current file, and rename it this.closeFile(); @@ -374,7 +397,7 @@ public synchronized void setFile(String fileName, boolean append, boolean buffer super.setFile(fileName, append, this.bufferedIO, this.bufferSize); if (append) { File f = new File(fileName); - ((CountingQuietWriter)qw).setCount(f.length()); + ((CountingQuietWriter) qw).setCount(f.length()); } } @@ -382,39 +405,6 @@ public synchronized void setFile(String fileName, boolean append, boolean buffer protected void setQWForFiles(Writer writer) { this.qw = new CountingQuietWriter(writer, errorHandler); } - - /** - * 测试方法 - */ - public static void main(String args[]) { - String className = MyDailyRollingFileAppender.class.getName(); - - MyDailyRollingFileAppender myAppender = new MyDailyRollingFileAppender(); - - myAppender.setDatePattern("'.'yyyy-MM-dd-HH"); - myAppender.setFile("E:/test/mylogApender/test.log"); - myAppender.setMaxBackupIndex(3); - - PatternLayout patternLayout = new PatternLayout("%d [%-5p]-[%t]-[%c{1}] (%F:%L) %m%n"); - myAppender.setLayout(patternLayout); - - System.out.println("getMaxBackupIndex():" + myAppender.getMaxBackupIndex()); - - myAppender.activateOptions(); - - for (int i = 0; i < 50000; i++) { - myAppender.subAppend(new LoggingEvent(className, Logger.getLogger(className), Level.INFO, - "test,hello " + i, null)); - - try { - Thread.sleep(100); - } catch (InterruptedException ex) { - ex.printStackTrace(); - } - - System.out.println("Fine attesa"); - } - } } /** @@ -422,8 +412,6 @@ public static void main(String args[]) { */ class MyRollingCalendar extends GregorianCalendar { - private static final long serialVersionUID = 1L; - public static final int TOP_OF_TROUBLE = -1; public static final int TOP_OF_MINUTE = 0; public static final int TOP_OF_HOUR = 1; @@ -431,7 +419,7 @@ class MyRollingCalendar extends GregorianCalendar { public static final int TOP_OF_DAY = 3; public static final int TOP_OF_WEEK = 4; public static final int TOP_OF_MONTH = 5; - + private static final long serialVersionUID = 1L; int type = TOP_OF_TROUBLE; MyRollingCalendar() { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/AbstractMetaData.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/AbstractMetaData.java index 05d23ff1..38357649 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/AbstractMetaData.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/AbstractMetaData.java @@ -25,7 +25,7 @@ import java.util.List; import java.util.Map; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.configurable.IConfigurable; import org.apache.rocketmq.streams.common.datatype.DataJsonable; import org.apache.rocketmq.streams.common.datatype.DataType; import org.apache.rocketmq.streams.common.datatype.ListDataType; @@ -34,11 +34,9 @@ public abstract class AbstractMetaData extends BasedConfigurable implements DataJsonable> { public static final String TYPE = "metaData"; - private String nameSpace; - private String type = TYPE; - private String configureName; protected List> metaDataFields = new ArrayList>(); protected String dataSourceName; + protected transient Map> metaDataFieldMap = new HashMap>(); private String tableName; private String tableNameAlias; @@ -46,8 +44,6 @@ public AbstractMetaData() { setType(TYPE); } - protected transient Map> metaDataFieldMap = new HashMap>(); - public List> getMetaDataFields() { return metaDataFields; } @@ -125,36 +121,6 @@ public Map getData(String jsonValue) { return dataset; } - @Override - public void setNameSpace(String nameSpace) { - this.nameSpace = nameSpace; - } - - @Override - public void setType(String type) { - this.type = type; - } - - @Override - public void setConfigureName(String configureName) { - this.configureName = configureName; - } - - @Override - public String getNameSpace() { - return nameSpace; - } - - @Override - public String getType() { - return type; - } - - @Override - public String getConfigureName() { - return configureName; - } - public String getDataSourceName() { return dataSourceName; } @@ -187,10 +153,9 @@ public void setTableNameAlias(String tableNameAlias) { this.tableNameAlias = tableNameAlias; } - @Override - public String toJson() { + @Override public String toJson() { JSONObject jsonObject = new JSONObject(); - jsonObject.put(IConfigurableService.CLASS_NAME, this.getClass().getName()); + jsonObject.put(IConfigurable.CLASS_NAME, this.getClass().getName()); JSONArray jsonArray = new JSONArray(); Iterator i$ = this.metaDataFields.iterator(); @@ -212,39 +177,4 @@ public String toJson() { protected abstract void getJsonValue(JSONObject var1); - @Override - public void toObject(String jsonString) { - JSONObject jsonObject = JSONObject.parseObject(jsonString); - String jsonArrayString = jsonObject.getString("metaDataFields"); - JSONArray jsonArray = JSON.parseArray(jsonArrayString); - List> resourceFields = new ArrayList(); - Map> resourceFieldMap = new HashMap(); - for (int i = 0; i < jsonArray.size(); ++i) { - String fieldJson = jsonArray.getString(i); - MetaDataField metaDataField = new MetaDataField(); - metaDataField.toObject(fieldJson); - String dataTypestr = ""; - - try { - if (metaDataField.getDataType() != null) { - dataTypestr = MetaDataField.getDataTypeStrByType(metaDataField.getDataType()); - } - } catch (Exception var12) { - dataTypestr = "String"; - } - - metaDataField.setDataTypeStr(dataTypestr); - resourceFields.add(metaDataField); - resourceFieldMap.put(metaDataField.getFieldName(), metaDataField); - } - - this.dataSourceName = jsonObject.getString("dataSourceName"); - this.tableName = jsonObject.getString("tableName"); - this.tableNameAlias = jsonObject.getString("tableNameAlias"); - this.metaDataFields = resourceFields; - this.metaDataFieldMap = resourceFieldMap; -// this.getJsonValue(jsonObject); - this.setJsonValue(jsonObject); - - } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/LogicMetaData.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/LogicMetaData.java index b9e9cd94..7886345d 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/LogicMetaData.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/LogicMetaData.java @@ -22,14 +22,13 @@ /** * @description logic table meta */ -public class LogicMetaData extends MetaData{ +public class LogicMetaData extends MetaData { String tableNamePattern; // table Set tableNames; // table_20210731000000 - - public LogicMetaData(){ + public LogicMetaData() { tableNames = new HashSet<>(); } @@ -41,17 +40,16 @@ public void setTableNamePattern(String tableNamePattern) { this.tableNamePattern = tableNamePattern; } - - public LogicMetaData addTableName(String tableName){ + public LogicMetaData addTableName(String tableName) { tableNames.add(tableName); return this; } - public boolean isContainsTable(String tableName){ + public boolean isContainsTable(String tableName) { return tableNames.contains(tableName); } - public boolean removeTable(String tableName){ + public boolean removeTable(String tableName) { return tableNames.remove(tableName); } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaData.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaData.java index b4f04d3c..6ab8adac 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaData.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaData.java @@ -25,6 +25,7 @@ import java.sql.SQLException; import java.util.HashSet; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Set; import org.apache.rocketmq.streams.common.datatype.DataType; @@ -36,39 +37,16 @@ public class MetaData extends AbstractMetaData { private static final String PRI_KEY = "id"; - + /** + * primaryFieldNames, + * List uniqueIndexFieldNames, List> indexFieldNamesList + */ + protected String primaryFieldNames;//主键字段,多个逗号分隔 + protected String uniqueIndexFieldNames;//唯一索引字段,多个逗号分隔 + protected List indexFieldNamesList;//多个索引字段,每个索引以逗号分隔 private Long id; - private String idFieldName; - @Override - public Long getId() { - return id; - } - - @Override - public void setId(Long id) { - this.id = id; - } - - public String getIdFieldName() { - return idFieldName; - } - - public void setIdFieldName(String idFieldName) { - this.idFieldName = idFieldName; - } - - @Override - protected void setJsonValue(JSONObject jsonObject) { - this.idFieldName = jsonObject.getString("idFieldName"); - } - - @Override - protected void getJsonValue(JSONObject jsonObject) { - jsonObject.put("idFieldName", this.idFieldName); - } - public static MetaData createMetaData(ResultSet metaResult) throws SQLException { MetaData metaData = new MetaData(); if (metaResult == null) { @@ -154,7 +132,9 @@ public static MetaData createMetaDate(Object object, Map paras) metaDataField.setDataType(datatype); //TODO if (datatype instanceof MapDataType) { - paras.put(columnName, value.toString()); + if (value != null) { + paras.put(columnName, value.toString()); + } } else { paras.put(columnName, value); } @@ -208,6 +188,56 @@ public static String getColumnNameFromFieldName(String para) { return sb.toString(); } + public Long getId() { + return id; + } + + public void setId(Long id) { + this.id = id; + } + + public String getIdFieldName() { + return idFieldName; + } + + public void setIdFieldName(String idFieldName) { + this.idFieldName = idFieldName; + } + + @Override + protected void setJsonValue(JSONObject jsonObject) { + this.idFieldName = jsonObject.getString("idFieldName"); + } + + @Override + protected void getJsonValue(JSONObject jsonObject) { + jsonObject.put("idFieldName", this.idFieldName); + } + + public String getPrimaryFieldNames() { + return primaryFieldNames; + } + + public void setPrimaryFieldNames(String primaryFieldNames) { + this.primaryFieldNames = primaryFieldNames; + } + + public String getUniqueIndexFieldNames() { + return uniqueIndexFieldNames; + } + + public void setUniqueIndexFieldNames(String uniqueIndexFieldNames) { + this.uniqueIndexFieldNames = uniqueIndexFieldNames; + } + + public List getIndexFieldNamesList() { + return indexFieldNamesList; + } + + public void setIndexFieldNamesList(List indexFieldNamesList) { + this.indexFieldNamesList = indexFieldNamesList; + } + /** * 如果某个类,不想自己的类名转化成表名,可以指定类名 */ @@ -222,5 +252,4 @@ public static String getColumnNameFromFieldName(String para) { String value() default ""; } - } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaDataField.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaDataField.java index 5a035ae9..583607be 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaDataField.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaDataField.java @@ -18,8 +18,7 @@ import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.JSONObject; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import org.apache.rocketmq.streams.common.configurable.IConfigurable; import org.apache.rocketmq.streams.common.datatype.BooleanDataType; import org.apache.rocketmq.streams.common.datatype.DataType; import org.apache.rocketmq.streams.common.datatype.FloatDataType; @@ -36,7 +35,6 @@ public class MetaDataField extends Entity implements IJsonable { public static final String RULE = "rule"; private static final long serialVersionUID = 3590425799189771820L; - private static final Log RULEENGINE_MESSAGE_LOG = LogFactory.getLog("ruleengine_message"); private String fieldName; private DataType dataType; private Boolean isRequired; @@ -45,6 +43,40 @@ public class MetaDataField extends Entity implements IJsonable { @Deprecated private String dataTypeStr; + public static DataType getDataTypeByStr(String dataType) { + DataType dt = null; + if ("String".equals(dataType)) { + dt = new StringDataType(); + } else if ("long".equals(dataType)) { + dt = new LongDataType(); + } else if ("int".equals(dataType)) { + dt = new IntDataType(); + } else if ("float".equals(dataType)) { + dt = new FloatDataType(); + } else if ("boolean".equals(dataType)) { + dt = new BooleanDataType(); + } else { + dt = new StringDataType(); + } + return dt; + } + + public static String getDataTypeStrByType(DataType dataType) { + String dataTypeStr = ""; + if (dataType instanceof StringDataType) { + dataTypeStr = "String"; + } else if (dataType instanceof LongDataType) { + dataTypeStr = "long"; + } else if (dataType instanceof IntDataType) { + dataTypeStr = "int"; + } else if (dataType instanceof FloatDataType) { + dataTypeStr = "float"; + } else { + dataTypeStr = "String"; + } + return dataTypeStr; + } + public String getFieldName() { return fieldName; } @@ -121,6 +153,7 @@ public String toJson() { jsonObject.put("dataType", dataType.toJson()); jsonObject.put("isRequired", isRequired); jsonObject.put("isPrimary", isPrimary); + jsonObject.put(IConfigurable.CLASS_NAME,this.getClass().getName()); return jsonObject.toJSONString(); } @@ -134,38 +167,4 @@ public void toObject(String jsonString) { this.isPrimary = jsonObject.getBoolean("isPrimary"); } - public static DataType getDataTypeByStr(String dataType) { - DataType dt = null; - if ("String".equals(dataType)) { - dt = new StringDataType(); - } else if ("long".equals(dataType)) { - dt = new LongDataType(); - } else if ("int".equals(dataType)) { - dt = new IntDataType(); - } else if ("float".equals(dataType)) { - dt = new FloatDataType(); - } else if ("boolean".equals(dataType)) { - dt = new BooleanDataType(); - } else { - dt = new StringDataType(); - } - return dt; - } - - public static String getDataTypeStrByType(DataType dataType) { - String dataTypeStr = ""; - if (dataType instanceof StringDataType) { - dataTypeStr = "String"; - } else if (dataType instanceof LongDataType) { - dataTypeStr = "long"; - } else if (dataType instanceof IntDataType) { - dataTypeStr = "int"; - } else if (dataType instanceof FloatDataType) { - dataTypeStr = "float"; - } else { - dataTypeStr = "String"; - } - return dataTypeStr; - } - } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaDataUtils.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaDataUtils.java index 402dc912..69cef564 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaDataUtils.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaDataUtils.java @@ -28,7 +28,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import org.apache.rocketmq.streams.common.component.AbstractComponent; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; import org.apache.rocketmq.streams.common.utils.DataTypeUtil; /** @@ -36,7 +36,7 @@ */ public class MetaDataUtils { - private static final String DEFAULT_DRIVER = AbstractComponent.DEFAULT_JDBC_DRIVER; + private static final String DEFAULT_DRIVER = ConfigurationKey.DEFAULT_JDBC_DRIVER; /** * 去除id字段,主要用于拼接sql diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/JobConfigure.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/JobConfigure.java new file mode 100644 index 00000000..dc9912d5 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/JobConfigure.java @@ -0,0 +1,62 @@ +/* + * 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.model; + +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; + +public class JobConfigure extends BasedConfigurable { + + /** + * 配置名称 + */ + private String configureName; + + /** + * 配置类型 + */ + private String configureType; + + /** + * 配置空间 + */ + private String configureNamespace; + + @Override public String getName() { + return configureName; + } + + @Override public void setName(String name) { + this.configureName = name; + } + + public String getConfigureType() { + return configureType; + } + + public void setConfigureType(String configureType) { + this.configureType = configureType; + } + + public String getConfigureNamespace() { + return configureNamespace; + } + + public void setConfigureNamespace(String configureNamespace) { + this.configureNamespace = configureNamespace; + } +} + diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/JobStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/JobStage.java new file mode 100644 index 00000000..c1bf4a5f --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/JobStage.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.model; + +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; + +public class JobStage extends BasedConfigurable { + + /** + * 机器名称 + */ + private String machineName; + + /** + * stage名称 + */ + private String stageName; + + /** + * stage类型 + */ + private String stageType; + + /** + * stage下游集合 + */ + private String nextStageLables; + + /** + * stage上游集合 + */ + private String prevStageLables; + + /** + * stage内容 + */ + private String stageContent; + private String sqlContent; + + public String getMachineName() { + return machineName; + } + + public void setMachineName(String machineName) { + this.machineName = machineName; + } + + public String getStageName() { + return stageName; + } + + public void setStageName(String stageName) { + this.stageName = stageName; + } + + public String getStageType() { + return stageType; + } + + public void setStageType(String stageType) { + this.stageType = stageType; + } + + public String getNextStageLables() { + return nextStageLables; + } + + public void setNextStageLables(String nextStageLables) { + this.nextStageLables = nextStageLables; + } + + public String getPrevStageLables() { + return prevStageLables; + } + + public void setPrevStageLables(String prevStageLables) { + this.prevStageLables = prevStageLables; + } + + public String getStageContent() { + return stageContent; + } + + public void setStageContent(String stageContent) { + this.stageContent = stageContent; + } + + public String getSqlContent() { + return sqlContent; + } + + public void setSqlContent(String sqlContent) { + this.sqlContent = sqlContent; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/NameCreator.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/NameCreator.java index 39a4838c..16cc8cd9 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/NameCreator.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/NameCreator.java @@ -22,15 +22,25 @@ import org.apache.rocketmq.streams.common.utils.MapKeyUtil; public class NameCreator { - private transient AtomicInteger nameCreator = new AtomicInteger(100000000); + private final transient AtomicInteger nameCreator = new AtomicInteger(100000000); - private Map creatorMap = new HashMap<>(); + private final Map creatorMap = new HashMap<>(); + + public static String getFirstPrefix(String name, String... types) { + for (String type : types) { + int index = name.indexOf("_" + type); + if (index != -1) { + return name.substring(0, index); + } + } + return name; + } /** * 每个规则一个名字生成器,expression name * - * @param names - * @return + * @param names 名称 + * @return NameCreator实例 */ public NameCreator createOrGet(String... names) { String ruleName = MapKeyUtil.createKeyBySign("_", names); @@ -49,12 +59,7 @@ public NameCreator createOrGet(String... names) { return nameCreator; } - public String createNewName(String... names) { - NameCreator nameCreator = createOrGet(names); - return nameCreator.createName(names); - } - public String createName(String... namePrefix) { - return MapKeyUtil.createKeyBySign("_", MapKeyUtil.createKeyBySign("_", namePrefix), nameCreator.incrementAndGet() + ""); + return MapKeyUtil.createKeyBySign("_", MapKeyUtil.createKeyBySign("_", namePrefix), String.valueOf(nameCreator.incrementAndGet())); } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/NameCreatorContext.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/NameCreatorContext.java index ba21be0e..6788e9f6 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/NameCreatorContext.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/NameCreatorContext.java @@ -16,20 +16,18 @@ */ package org.apache.rocketmq.streams.common.model; +import java.util.concurrent.Executors; + public class NameCreatorContext { - protected static ThreadLocal threadLocal = new ThreadLocal<>(); - private NameCreatorContext(NameCreator nameCreator) { - threadLocal.set(nameCreator); - } + private static ThreadLocal threadLocal = new ThreadLocal<>(); public static NameCreator get() { - NameCreator nameCreator = threadLocal.get(); if (nameCreator == null) { nameCreator = new NameCreator(); - threadLocal.set(nameCreator); } + threadLocal.set(nameCreator); return nameCreator; } @@ -37,8 +35,38 @@ public static void remove() { threadLocal.remove(); } - public static NameCreatorContext init(NameCreator nameCreator) { - return new NameCreatorContext(nameCreator); + public static void main(String[] args) { + + System.out.println("主线程: " + NameCreatorContext.get().createName("1111", "22222")); + System.out.println("主线程: " + NameCreatorContext.get().createName("1111", "22222")); + System.out.println("主线程: " + NameCreatorContext.get().createName("1111", "22222")); + System.out.println("主线程: " + NameCreatorContext.get().createName("1111", "22222")); + System.out.println("主线程: " + NameCreatorContext.get().createName("3333", "4444")); + System.out.println("主线程: " + NameCreatorContext.get().createName("3333", "4444")); + System.out.println("主线程: " + NameCreatorContext.get().createName("3333", "4444")); + + Executors.newSingleThreadExecutor().execute(new Runnable() { + @Override + public void run() { + System.out.println("线程1: " + NameCreatorContext.get().createName("1111", "22222")); + System.out.println("线程1: " + NameCreatorContext.get().createName("1111", "22222")); + System.out.println("线程1: " + NameCreatorContext.get().createName("1111", "22222")); + System.out.println("线程1: " + NameCreatorContext.get().createName("1111", "22222")); + } + }); + + Executors.newSingleThreadExecutor().execute(new Runnable() { + @Override + public void run() { + System.out.println("线程2: " + NameCreatorContext.get().createName("3333", "4444")); + System.out.println("线程2: " + NameCreatorContext.get().createName("3333", "4444")); + System.out.println("线程2: " + NameCreatorContext.get().createName("3333", "4444")); + System.out.println("线程2: " + NameCreatorContext.get().createName("3333", "4444")); + System.out.println("线程2: " + NameCreatorContext.get().createName("3333", "4444")); + System.out.println("线程2: " + NameCreatorContext.get().createName("3333", "4444")); + } + }); + } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/Pair.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/Pair.java new file mode 100644 index 00000000..e27184ea --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/Pair.java @@ -0,0 +1,122 @@ +/* + * 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.model; + +import java.io.Serializable; + +public class Pair implements Serializable { + + /** + * Key of this Pair. + */ + private K key; + /** + * Value of this this Pair. + */ + private V value; + + /** + * Creates a new pair + * + * @param key The key for this pair + * @param value The value to use for this pair + */ + public Pair(K key, V value) { + this.key = key; + this.value = value; + } + + /** + * Gets the key for this pair. + * + * @return key for this pair + */ + public K getKey() { + return key; + } + + /** + * Gets the value for this pair. + * + * @return value for this pair + */ + public V getValue() { + return value; + } + + /** + *

String representation of this + * Pair.

+ * + *

The default name/value delimiter '=' is always used.

+ * + * @return String representation of this Pair + */ + @Override + public String toString() { + return key + "=" + value; + } + + /** + *

Generate a hash code for this Pair.

+ * + *

The hash code is calculated using both the name and + * the value of the Pair.

+ * + * @return hash code for this Pair + */ + @Override + public int hashCode() { + // name's hashCode is multiplied by an arbitrary prime number (13) + // in order to make sure there is a difference in the hashCode between + // these two parameters: + // name: a value: aa + // name: aa value: a + return key.hashCode() * 13 + (value == null ? 0 : value.hashCode()); + } + + /** + *

Test this Pair for equality with another + * Object.

+ * + *

If the Object to be tested is not a + * Pair or is null, then this method + * returns false.

+ * + *

Two Pairs are considered equal if and only if + * both the names and values are equal.

+ * + * @param o the Object to test for + * equality with this Pair + * @return true if the given Object is + * equal to this Pair else false + */ + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o instanceof Pair) { + Pair pair = (Pair) o; + if (key != null ? !key.equals(pair.key) : pair.key != null) + return false; + if (value != null ? !value.equals(pair.value) : pair.value != null) + return false; + return true; + } + return false; + } +} diff --git a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/ConfigurableServcieType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/SQLCompileContextForSource.java similarity index 58% rename from rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/ConfigurableServcieType.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/SQLCompileContextForSource.java index a4998ff7..cb6b9063 100644 --- a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/ConfigurableServcieType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/SQLCompileContextForSource.java @@ -14,16 +14,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.configurable.service; +package org.apache.rocketmq.streams.common.model; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.channel.source.ISource; -@Deprecated -public class ConfigurableServcieType { +/** + * 在sql编译时,需要把source放到上下文中,因为window shuffle需要根据source的分片来创建shuffle 队列 + */ +public class SQLCompileContextForSource extends ThreadLocal { + + private static final SQLCompileContextForSource instance = new SQLCompileContextForSource(); - public static final String DEFAULT_SERVICE_NAME = IConfigurableService.DEFAULT_SERVICE_NAME; - public static final String MEMORY_SERVICE_NAME = IConfigurableService.MEMORY_SERVICE_NAME; - public static final String FILE_SERVICE_NAME = IConfigurableService.FILE_SERVICE_NAME; - public static final String HTTP_SERVICE_NAME = IConfigurableService.HTTP_SERVICE_NAME; + private SQLCompileContextForSource() { + } -} + public static SQLCompileContextForSource getInstance() { + return instance; + } +} \ No newline at end of file diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/ServiceName.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/ServiceName.java index 6d3d1ce7..b30c8dd8 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/ServiceName.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/ServiceName.java @@ -30,7 +30,5 @@ String aliasName() default ""; - - String name() default ""; } \ No newline at end of file diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/StageRelation.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/StageRelation.java new file mode 100644 index 00000000..3298e84c --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/StageRelation.java @@ -0,0 +1,112 @@ +/* + * 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.model; + +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; + +public class StageRelation extends BasedConfigurable { + + private String viewName; + + private String groupName; + + private String sqlContent; + + private String startLabel; + + private String endLabel; + + private String stageLabels; + + private String childrenNames; + + private String parentName; + + private String userId; + + public String getViewName() { + return viewName; + } + + public void setViewName(String viewName) { + this.viewName = viewName; + } + + public String getGroupName() { + return groupName; + } + + public void setGroupName(String groupName) { + this.groupName = groupName; + } + + public String getSqlContent() { + return sqlContent; + } + + public void setSqlContent(String sqlContent) { + this.sqlContent = sqlContent; + } + + public String getStartLabel() { + return startLabel; + } + + public void setStartLabel(String startLabel) { + this.startLabel = startLabel; + } + + public String getEndLabel() { + return endLabel; + } + + public void setEndLabel(String endLabel) { + this.endLabel = endLabel; + } + + public String getStageLabels() { + return stageLabels; + } + + public void setStageLabels(String stageLabels) { + this.stageLabels = stageLabels; + } + + public String getChildrenNames() { + return childrenNames; + } + + public void setChildrenNames(String childrenNames) { + this.childrenNames = childrenNames; + } + + public String getParentName() { + return parentName; + } + + public void setParentName(String parentName) { + this.parentName = parentName; + } + + public String getUserId() { + return userId; + } + + public void setUserId(String userId) { + this.userId = userId; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/ConsoleMonitorManager.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/ConsoleMonitorManager.java index a3837ae0..2092252f 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/ConsoleMonitorManager.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/ConsoleMonitorManager.java @@ -16,50 +16,40 @@ */ package org.apache.rocketmq.streams.common.monitor; -import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.JSONObject; -import java.util.ArrayList; import java.util.Date; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.channel.IChannel; import org.apache.rocketmq.streams.common.channel.source.ISource; -import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.configuration.SystemContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.monitor.model.JobStage; import org.apache.rocketmq.streams.common.monitor.model.TraceIdsDO; import org.apache.rocketmq.streams.common.monitor.model.TraceMonitorDO; import org.apache.rocketmq.streams.common.monitor.service.MonitorDataSyncService; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; +import org.apache.rocketmq.streams.common.threadpool.ScheduleFactory; import org.apache.rocketmq.streams.common.topology.model.AbstractStage; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ConsoleMonitorManager { - private static final Log LOG = LogFactory.getLog(ConsoleMonitorManager.class); - public static final int MSG_FILTERED = -1; public static final int MSG_NOT_FLOWED = 0; public static final int MSG_FLOWED = 1; - + private static final Logger LOG = LoggerFactory.getLogger(ConsoleMonitorManager.class); private static ConsoleMonitorManager monitorManager = new ConsoleMonitorManager(); private Map cache = new ConcurrentHashMap(); private Map traceCache = new ConcurrentHashMap(); private Set validTraceIds = new HashSet(); private MonitorDataSyncService monitorDataSyncService = MonitorDataSyncServiceFactory.create(); - public static ConsoleMonitorManager getInstance() { - return monitorManager; - } - /** * 上面使用 static 定义并初始化了一个 ConsoleMonitorManager 实例,所以当这类被加载时,就会执行构造方法 * 构造方法会开启一个定时任务,任务会定时执行一个线程,该线程动作如下: @@ -71,15 +61,14 @@ public ConsoleMonitorManager() { if (!isConsoleOpen()) { return; } - - ScheduledExecutorService executorService = Executors.newScheduledThreadPool(1); - executorService.scheduleAtFixedRate(new Runnable() { - @Override - public void run() { + ScheduleFactory.getInstance().execute(ConsoleMonitorManager.class.getName() + "-console_monitor", new Runnable() { + @Override public void run() { try { queryValidTraceIds(); Map jobStageMap = cache; - cache = new ConcurrentHashMap(); + synchronized (this) { + cache = new ConcurrentHashMap(); + } long current = System.currentTimeMillis(); for (JobStage jobStage : jobStageMap.values()) { jobStage.setMachineName(""); @@ -126,6 +115,18 @@ public void run() { }, 20, 30, TimeUnit.SECONDS); } + public static ConsoleMonitorManager getInstance() { + return monitorManager; + } + + public static void main(String[] args) { + long a = 6l; + System.out.println((float) (a / ((10000 - 79) / 1000))); + System.out.println((float) (a / 16)); + System.out.println((10000 - 79) * 1.0 / 1000); +// Math. + } + public Set getValidTraceIds() { return validTraceIds; } @@ -141,7 +142,7 @@ public void reportChannel(ChainPipeline pipeline, ISource source, IMessage messa long clientTime = message.getHeader().getSendTime(); JSONObject msg = message.getMessageBody(); //目前pipeline只支持一个channel 暂时写死channel name - JobStage jobStage = getJobStage(source.getConfigureName() + "_source_0"); + JobStage jobStage = getJobStage(source.getName() + "_source_0"); // // 必须是开启了 traceId,才能对 dipper_trace_monitor 表中的数据进行修改 // // 只是消息中带有 traceId 不行 @@ -168,7 +169,7 @@ public void reportChannel(ChainPipeline pipeline, ISource source, IMessage messa // } jobStage.getSafeInput().incrementAndGet(); - jobStage.setLastInputMsgObj(msg); +// jobStage.setLastInputMsgObj(msg); if (clientTime != 0) { jobStage.setLastInputMsgTime(new Date(clientTime)); } else { @@ -182,7 +183,7 @@ public void reportChannel(ChainPipeline pipeline, ISource source, IMessage messa if (!message.getHeader().isSystemMessage()) { // 记录 traceId // getTraceMonitor() 会从 traceCache 中取 traceMonitor,没有会新建一个并添加到 traceCache 中 - TraceMonitorDO traceMonitor = getTraceMonitor(source.getConfigureName() + "_source_0", traceId); + TraceMonitorDO traceMonitor = getTraceMonitor(source.getName() + "_source_0", traceId); // 表示消息正常流转过了此 stage traceMonitor.setStatus(1); traceMonitor.getSafeInput().incrementAndGet(); @@ -191,7 +192,7 @@ public void reportChannel(ChainPipeline pipeline, ISource source, IMessage messa traceMonitor.setLastOutputMsgTime(new Date()); traceMonitor.setInputLastMsg(msg.toJSONString()); traceMonitor.setOutputLastMsg(msg.toJSONString()); - traceMonitor.setJobName(pipeline.getConfigureName()); + traceMonitor.setJobName(pipeline.getName()); } } @@ -208,8 +209,8 @@ public void reportInput(AbstractStage stage, IMessage message) { JSONObject msg = message.getMessageBody(); JobStage jobStage = getJobStage(stage.getLabel()); jobStage.getSafeInput().incrementAndGet(); - jobStage.setLastInputMsgObj(msg); - jobStage.setLastInputMsg(msg.toJSONString()); +// jobStage.setLastInputMsgObj(msg); +// jobStage.setLastInputMsg(msg.toJSONString()); jobStage.setLastInputMsgTime(new Date()); String traceId = message.getHeader().getTraceId(); @@ -220,7 +221,7 @@ public void reportInput(AbstractStage stage, IMessage message) { traceMonitor.getSafeInput().incrementAndGet(); traceMonitor.setInputLastMsg(msg.toJSONString()); traceMonitor.setLastInputMsgTime(new Date()); - traceMonitor.setJobName(stage.getPipeline().getConfigureName()); + traceMonitor.setJobName(stage.getPipeline().getName()); } } @@ -259,11 +260,11 @@ public void reportOutput(AbstractStage stage, IMessage message, int status, Stri traceMonitor.setOutputLastMsg(msg.toJSONString()); traceMonitor.setLastOutputMsgTime(new Date()); } - traceMonitor.setJobName(stage.getPipeline().getConfigureName()); + traceMonitor.setJobName(stage.getPipeline().getName()); } } - public JobStage getJobStage(String uniqKey) { + public synchronized JobStage getJobStage(String uniqKey) { // String key = createKey(uniqKey); JobStage jobStage = cache.get(uniqKey); if (jobStage == null) { @@ -314,16 +315,6 @@ public void queryValidTraceIds() { } - private boolean isConsoleOpen() { - String configurableServiceType = ComponentCreator.getProperties().getProperty(DataSyncConstants.UPDATE_TYPE); - if (DataSyncConstants.UPDATE_TYPE_ROCKETMQ.equalsIgnoreCase(configurableServiceType) || - DataSyncConstants.UPDATE_TYPE_HTTP.equalsIgnoreCase(configurableServiceType) || - DataSyncConstants.UPDATE_TYPE_DB.equalsIgnoreCase(configurableServiceType)) { - return true; - } - return false; - } - // public void saveJobName(List traceMonitorDOS){ // if (traceMonitorDOS==null || traceMonitorDOS.size()==0){ // return; @@ -383,6 +374,14 @@ private boolean isConsoleOpen() { // head.next = tail; // } + private boolean isConsoleOpen() { + String configurableServiceType = SystemContext.getProperty(ConfigurationKey.UPDATE_TYPE); + if (DataSyncConstants.UPDATE_TYPE_ROCKETMQ.equalsIgnoreCase(configurableServiceType) || DataSyncConstants.UPDATE_TYPE_HTTP.equalsIgnoreCase(configurableServiceType) || DataSyncConstants.UPDATE_TYPE_DB.equalsIgnoreCase(configurableServiceType)) { + return true; + } + return false; + } + class ListNode { public String key; public String value; @@ -399,14 +398,6 @@ public ListNode() { } } - public static void main(String[] args) { - long a = 6l; - System.out.println((float) (a / ((10000 - 79) / 1000))); - System.out.println((float) (a / 16)); - System.out.println((10000 - 79) * 1.0 / 1000); -// Math. - } - } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/DataSyncConstants.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/DataSyncConstants.java index 612e15a4..2d718c18 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/DataSyncConstants.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/DataSyncConstants.java @@ -18,36 +18,6 @@ public class DataSyncConstants { - /** - * rocketmq-stream更新模块对应的topic - */ - public static final String RULE_UP_TOPIC = "dipper.console.topic.up"; - - /** - * rocketmq-stream更新模块对应的topic - */ - public static final String RULE_UP_TAG = "dipper.console.topic.up.tags"; - - /** - * rocketmq-stream更新模块对应的topic - */ - public static final String RULE_DOWN_TOPIC = "dipper.console.topic.down"; - - /** - * rocketmq-stream更新模块对应的topic - */ - public static final String RULE_DOWN_TAG = "dipper.console.topic.down.tags"; - - /** - * rocketmq-stream更新模块对应的tag - */ - public static final String RULE_TOPIC_TAG = "dipper.console.tag"; - - /** - * rocketmq-stream更新模块对应的消息渠道类型 默认为metaq - */ - public static final String UPDATE_TYPE = "dipper.console.service.type"; - public static final String UPDATE_TYPE_HTTP = "http"; public static final String UPDATE_TYPE_DB = "db"; public static final String UPDATE_TYPE_ROCKETMQ = "rocketmq"; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/HttpUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/HttpUtil.java index b5c789ae..bb527af2 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/HttpUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/HttpUtil.java @@ -51,13 +51,11 @@ public class HttpUtil { public static final String CHARSET = "UTF-8"; public static final int TIMOUT = 10000; public static final int CONNECT_TIMOUT = 10000; - + private static CloseableHttpClient httpclient; protected String accessId; protected String accessIdSecret; protected String endPoint; - private static CloseableHttpClient httpclient; - public HttpUtil(String accessId, String accessIdSecret, String endPoint) { this.accessId = accessId; this.accessIdSecret = accessIdSecret; @@ -65,43 +63,6 @@ public HttpUtil(String accessId, String accessIdSecret, String endPoint) { init(); } - private void init() { - RequestConfig.Builder configBuilder = RequestConfig.custom(); - configBuilder.setConnectionRequestTimeout(CONNECT_TIMOUT); - configBuilder.setConnectTimeout(CONNECT_TIMOUT); - configBuilder.setSocketTimeout(TIMOUT); - SSLConnectionSocketFactory sslsf = null; - try { - SSLContext sslcontext = SSLContexts.custom().loadTrustMaterial(new TrustStrategy() { - @Override - public boolean isTrusted(X509Certificate[] chain, String authType) throws CertificateException { - return true; - } - }).build(); - sslsf = new SSLConnectionSocketFactory(sslcontext, new HostnameVerifier() { - @Override - public boolean verify(String s, SSLSession sslSession) { - return true; - } - }); - } catch (Exception e) { - throw new RuntimeException(e); - } - ConnectionConfig connectionConfig = ConnectionConfig.custom().setCharset(Consts.UTF_8).build(); - Registry socketFactoryRegistry = RegistryBuilder.create().register("https", sslsf).register("http", new PlainConnectionSocketFactory()).build(); - PoolingHttpClientConnectionManager connManager = new PoolingHttpClientConnectionManager(socketFactoryRegistry); - connManager.setDefaultConnectionConfig(connectionConfig); - connManager.setMaxTotal(500); - connManager.setDefaultMaxPerRoute(50); - HttpClientBuilder clientBuilder = HttpClients.custom(); - - clientBuilder.setDefaultRequestConfig(configBuilder.build()); - clientBuilder.setSSLSocketFactory(sslsf); - clientBuilder.setConnectionManager(connManager); - - httpclient = clientBuilder.build(); - } - public static String getContent(String url) { return getContent(url, null); } @@ -244,5 +205,42 @@ public static String replaceSpace(String str) { } return str; } + + private void init() { + RequestConfig.Builder configBuilder = RequestConfig.custom(); + configBuilder.setConnectionRequestTimeout(CONNECT_TIMOUT); + configBuilder.setConnectTimeout(CONNECT_TIMOUT); + configBuilder.setSocketTimeout(TIMOUT); + SSLConnectionSocketFactory sslsf = null; + try { + SSLContext sslcontext = SSLContexts.custom().loadTrustMaterial(new TrustStrategy() { + @Override + public boolean isTrusted(X509Certificate[] chain, String authType) throws CertificateException { + return true; + } + }).build(); + sslsf = new SSLConnectionSocketFactory(sslcontext, new HostnameVerifier() { + @Override + public boolean verify(String s, SSLSession sslSession) { + return true; + } + }); + } catch (Exception e) { + throw new RuntimeException(e); + } + ConnectionConfig connectionConfig = ConnectionConfig.custom().setCharset(Consts.UTF_8).build(); + Registry socketFactoryRegistry = RegistryBuilder.create().register("https", sslsf).register("http", new PlainConnectionSocketFactory()).build(); + PoolingHttpClientConnectionManager connManager = new PoolingHttpClientConnectionManager(socketFactoryRegistry); + connManager.setDefaultConnectionConfig(connectionConfig); + connManager.setMaxTotal(500); + connManager.setDefaultMaxPerRoute(50); + HttpClientBuilder clientBuilder = HttpClients.custom(); + + clientBuilder.setDefaultRequestConfig(configBuilder.build()); + clientBuilder.setSSLSocketFactory(sslsf); + clientBuilder.setConnectionManager(connManager); + + httpclient = clientBuilder.build(); + } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/IMonitor.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/IMonitor.java index 51baeed8..b1305b06 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/IMonitor.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/IMonitor.java @@ -50,7 +50,7 @@ public interface IMonitor { * @return */ static IMonitor createMonitor(IConfigurable configurable) { - String name = MapKeyUtil.createKeyBySign(".", configurable.getType(), configurable.getNameSpace(), configurable.getConfigureName()); + String name = MapKeyUtil.createKeyBySign(".", configurable.getType(), configurable.getNameSpace(), configurable.getName()); IMonitor monitor = new DipperMonitor(); monitor.startMonitor(name); return monitor; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/MonitorDataSyncServiceFactory.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/MonitorDataSyncServiceFactory.java index cf8125b6..65da568d 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/MonitorDataSyncServiceFactory.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/MonitorDataSyncServiceFactory.java @@ -16,9 +16,8 @@ */ package org.apache.rocketmq.streams.common.monitor; -import org.apache.rocketmq.streams.common.component.AbstractComponent; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.configuration.SystemContext; import org.apache.rocketmq.streams.common.monitor.service.MonitorDataSyncService; import org.apache.rocketmq.streams.common.monitor.service.impl.DBMonitorDataSyncImpl; import org.apache.rocketmq.streams.common.monitor.service.impl.HttpMonitorDataSyncImpl; @@ -27,13 +26,13 @@ public class MonitorDataSyncServiceFactory { public static MonitorDataSyncService create() { - String configureType = ComponentCreator.getProperties().getProperty(DataSyncConstants.UPDATE_TYPE); + String configureType = SystemContext.getStringParameter(ConfigurationKey.UPDATE_TYPE); if (DataSyncConstants.UPDATE_TYPE_DB.equalsIgnoreCase(configureType)) { return new DBMonitorDataSyncImpl(); } else if (DataSyncConstants.UPDATE_TYPE_HTTP.equalsIgnoreCase(configureType)) { - String accessId = ComponentCreator.getProperties().getProperty(AbstractComponent.HTTP_AK); - String accessIdSecret = ComponentCreator.getProperties().getProperty(AbstractComponent.HTTP_SK); - String endPoint = ComponentCreator.getProperties().getProperty(IConfigurableService.HTTP_SERVICE_ENDPOINT); + String accessId = SystemContext.getStringParameter(ConfigurationKey.HTTP_AK); + String accessIdSecret = SystemContext.getStringParameter(ConfigurationKey.HTTP_SK); + String endPoint = SystemContext.getStringParameter(ConfigurationKey.HTTP_SERVICE_ENDPOINT); return new HttpMonitorDataSyncImpl(accessId, accessIdSecret, endPoint); } else if (DataSyncConstants.UPDATE_TYPE_ROCKETMQ.equalsIgnoreCase(configureType)) { return new RocketMQMonitorDataSyncImpl(); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/MonitorFactory.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/MonitorFactory.java index 89f4b0ea..c1e63110 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/MonitorFactory.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/MonitorFactory.java @@ -17,12 +17,14 @@ package org.apache.rocketmq.streams.common.monitor; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import org.apache.rocketmq.streams.common.cache.softreference.ICache; import org.apache.rocketmq.streams.common.cache.softreference.impl.SoftReferenceCache; import org.apache.rocketmq.streams.common.channel.sink.ISink; +import org.apache.rocketmq.streams.common.interfaces.ITaskStart; import org.apache.rocketmq.streams.common.logger.LoggerOutputChannel; import org.apache.rocketmq.streams.common.monitor.impl.DipperMonitor; @@ -31,13 +33,12 @@ public class MonitorFactory { public static final String NAMESPACE_PIPLINES = "namespace.";//所有的pipline public static final String PIPLINE_START_UP = "startup"; public static final String PIPLINE_START_UP_ERROR = "errorChannels"; - private static ICache monitorICache = new SoftReferenceCache(); + private static final ICache monitorICache = new SoftReferenceCache(); + private static final List> defaultOutput = new ArrayList<>(); + private static final Map>> outputs = new HashMap<>(); + private static final Map> loggerOutputChannelMap = new HashMap<>(); public static String LOG_ROOT_DIR = "/tmp/dipper/logs"; - private static ISink loggerOutputDataSource;//默认的输出channel,输出到日志 - - private static List defalutOutput = new ArrayList<>(); - private static Map> outputs = new HashMap<>(); - private static Map loggerOutputChannelMap = new HashMap<>(); + private static ISink loggerOutputDataSource;//默认的输出channel,输出到日志 public static IMonitor createMonitor(String name) { IMonitor monitor = new DipperMonitor(); @@ -61,18 +62,12 @@ public static IMonitor getOrCreateMonitor(String name) { * @param name * @param channels */ - public static void addChannel(String name, ISink... channels) { - List outputDataSources = outputs.get(name); - if (outputDataSources == null) { - outputDataSources = new ArrayList<>(); - outputs.put(name, outputDataSources); - } + public static void addChannel(String name, ISink... channels) { + List> outputDataSources = outputs.computeIfAbsent(name, k -> new ArrayList<>()); if (channels == null || channels.length == 0) { return; } - for (ISink outputDataSource : channels) { - outputDataSources.add(outputDataSource); - } + Collections.addAll(outputDataSources, channels); } /** @@ -80,24 +75,24 @@ public static void addChannel(String name, ISink... channels) { * * @param channels */ - public static void addChannel(ISink... channels) { - if (channels == null || channels.length == 0) { + public static void addChannel(ISink... channels) { + if (channels == null) { return; } - for (ISink outputDataSource : channels) { + for (ISink outputDataSource : channels) { if (outputDataSource != null) { - defalutOutput.add(outputDataSource); + defaultOutput.add(outputDataSource); } } } - public static List getOutputDataSource(String name, String level) { - List outputDataSources = outputs.get(name); + public static List> getOutputDataSource(String name, String level) { + List> outputDataSources = outputs.get(name); if (outputDataSources != null) { return outputDataSources; } - if (defalutOutput != null && defalutOutput.size() > 0) { - return defalutOutput; + if (defaultOutput != null && defaultOutput.size() > 0) { + return defaultOutput; } return null; } @@ -110,8 +105,8 @@ public static void initLogDir(String rootLogDir) { LOG_ROOT_DIR = rootLogDir; } - public static ISink createOrGetLogOutputDatasource(String outputName) { - ISink loggerOutputDataSource = loggerOutputChannelMap.get(outputName); + public static ISink createOrGetLogOutputDatasource(String outputName) { + ISink loggerOutputDataSource = loggerOutputChannelMap.get(outputName); if (loggerOutputDataSource != null) { return loggerOutputDataSource; } @@ -123,9 +118,10 @@ public static ISink createOrGetLogOutputDatasource(String outputName) { loggerOutputDataSource = new LoggerOutputChannel(LOG_ROOT_DIR, outputName); loggerOutputChannelMap.put(outputName, loggerOutputDataSource); - loggerOutputDataSource.init(); + ((ITaskStart) loggerOutputDataSource).startTask(); loggerOutputDataSource.openAutoFlush(); return loggerOutputDataSource; + } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/group/MonitorCommander.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/group/MonitorCommander.java index 63d7ea70..45722858 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/group/MonitorCommander.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/group/MonitorCommander.java @@ -21,12 +21,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import org.apache.commons.lang3.concurrent.BasicThreadFactory; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.channel.sink.ISink; import org.apache.rocketmq.streams.common.context.Message; import org.apache.rocketmq.streams.common.datatype.BooleanDataType; @@ -38,10 +33,13 @@ import org.apache.rocketmq.streams.common.metadata.MetaDataField; import org.apache.rocketmq.streams.common.monitor.IMonitor; import org.apache.rocketmq.streams.common.monitor.MonitorFactory; +import org.apache.rocketmq.streams.common.threadpool.ScheduleFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class MonitorCommander { - private static final Log logger = LogFactory.getLog(MonitorCommander.class); + private static final Logger logger = LoggerFactory.getLogger(MonitorCommander.class); private final static MonitorCommander monitorManager = new MonitorCommander(); @@ -97,8 +95,7 @@ public void init(ISink... outputDataSource) { outputDataSourceList.add(source); } } - ScheduledExecutorService monitorService = new ScheduledThreadPoolExecutor(1, new BasicThreadFactory.Builder().namingPattern("monitor-schedule-pool-%d").build()); - monitorService.scheduleWithFixedDelay(() -> { + ScheduleFactory.getInstance().execute(this.getClass().getName() + "-monitor_schedule", () -> { Map groupedMap; synchronized (object) { groupedMap = groupedMonitorInfoMap; @@ -117,11 +114,11 @@ public void init(ISink... outputDataSource) { result.put("errorCount", gmi.getErrorCount()); result.put("slowCount", gmi.getSlowCount()); //本地打印 - loggerOutputDataSource.batchAdd(new Message(result),null); + loggerOutputDataSource.batchAdd(new Message(result), null); loggerOutputDataSource.flush(); //远程输出 for (ISink source : outputDataSourceList) { - source.batchAdd(new Message(result),null); + source.batchAdd(new Message(result), null); } } //flush出去 diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/impl/DipperMonitor.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/impl/DipperMonitor.java index f63286c6..11730eeb 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/impl/DipperMonitor.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/impl/DipperMonitor.java @@ -20,12 +20,10 @@ import com.alibaba.fastjson.JSONObject; import java.util.ArrayList; import java.util.List; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.channel.sink.ISink; -import org.apache.rocketmq.streams.common.component.ComponentCreator; import org.apache.rocketmq.streams.common.configurable.IConfigurable; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.configuration.SystemContext; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.Message; @@ -41,63 +39,70 @@ import org.apache.rocketmq.streams.common.utils.DataTypeUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class DipperMonitor implements IMonitor { - private static final Log LOG = LogFactory.getLog(DipperMonitor.class); - + public static final String SLOW_NAME = "slow";//对于执行慢的stage进行记录 + public static final String ERROR_NAME = "error";//对于跑错误的stage进行记录 + private static final Logger LOG = LoggerFactory.getLogger(DipperMonitor.class); private static final String MONITO_SLOW = "SLOW"; private static final String MONITO_INFO = "INFO"; private static final String MONITO_ERROR = "ERROR"; private static final int MONITOR_SLOW_TIMEOUT = 30;//默认慢查询的输出是30s - public static final String SLOW_NAME = "slow";//对于执行慢的stage进行记录 - public static final String ERROR_NAME = "error";//对于跑错误的stage进行记录 + protected static String LEVEL = null; + protected static Integer SLOW_TIMEOUT; + protected static String monitorOutputLevel = null; + /** + * 输出时会用到,比如输出到数据库,则每个字段名,需要和metadata一致 + */ + private static MetaData metaData = new MetaData(); + + static { + //根据配置文件,把默认的监控配置读出来,主要是level和slow_timeout + + String timeoutStr = SystemContext.getStringParameter(ConfigurationKey.MONITOR_SLOW_TIMEOUT); + if (StringUtil.isNotEmpty(timeoutStr)) { + SLOW_TIMEOUT = Integer.valueOf(timeoutStr); + } + } + + static { + metaData.setTableName("monitor_data"); + metaData.setIdFieldName("id"); + metaData.getMetaDataFields().add(createMetaDataField("id", new LongDataType())); + metaData.getMetaDataFields().add(createMetaDataField(MONITOR_CHILDREN)); + metaData.getMetaDataFields().add(createMetaDataField(MONITOR_CONTEXT_MSG)); + metaData.getMetaDataFields().add(createMetaDataField(MONITOR_COST, new IntDataType())); + metaData.getMetaDataFields().add(createMetaDataField(MONITOR_ERROR_MSG)); + metaData.getMetaDataFields().add(createMetaDataField(MONITOR_SAMPLE_DATA)); + metaData.getMetaDataFields().add(createMetaDataField(MONITOR_SUCCESS, new BooleanDataType())); + metaData.getMetaDataFields().add(createMetaDataField(MONTIOR_SLOW, new BooleanDataType())); + metaData.getMetaDataFields().add(createMetaDataField(MONTIOR_NAME)); + + } + protected String info = MONITO_INFO; protected String debug = MONITO_ERROR; protected String warn = MONITO_SLOW + MONITO_ERROR; protected String level = debug; - protected Integer timeoutSecond = MONITOR_SLOW_TIMEOUT; - protected long startTime = System.currentTimeMillis(); protected long endTime; protected volatile boolean success = true;//本次监控是否有发生错误的逻辑 protected Exception e;//如果发生了错误,具体的错误是什么 protected String[] errorMsgs;//如果发生了错误,具体的错误信息是什么 protected String name;//监控名,业务方确定,不重复即可 - protected Object value;//执行结果 - protected volatile long cost;//花费时长 - protected volatile JSONObject sampleData;//一些上下文的采样数据,便于出错,或慢的时候做分析 - protected List contextMsgs = new ArrayList<>();//上下文信息,必要的时候,可以设置上下文信息 - protected List children = new ArrayList<>();//监控小项,一个监控,可以有多个项目 - - protected static String LEVEL = null; - protected static Integer SLOW_TIMEOUT; - protected static String monitorOutputLevel = null; - private String type; - static { - /** - * 根据配置文件,把默认的监控配置读出来,主要是level和slow_timeout - */ - monitorOutputLevel = ConfigureFileKey.MONITOR_OUTPUT_LEVEL; - LEVEL = ComponentCreator.getProperties().getProperty(monitorOutputLevel); - String timeoutStr = ComponentCreator.getProperties().getProperty(ConfigureFileKey.MONITOR_SLOW_TIMEOUT); - if (StringUtil.isNotEmpty(timeoutStr)) { - SLOW_TIMEOUT = Integer.valueOf(timeoutStr); - } - } - public DipperMonitor() { - /** - * 可以在配置文件通过默认配置加.name的方式覆盖默认配置。这里是读取自定义配置,如果没有则用通用配置,通用配置也没有,用默认配置 - */ + //可以在配置文件通过默认配置加.name的方式覆盖默认配置。这里是读取自定义配置,如果没有则用通用配置,通用配置也没有,用默认配置 if (StringUtil.isNotEmpty(LEVEL)) { this.level = LEVEL; } @@ -112,14 +117,36 @@ private DipperMonitor(DipperMonitor parent) { this.timeoutSecond = parent.timeoutSecond; } + private static MetaDataField createMetaDataField(String name) { + return createMetaDataField(name, new StringDataType()); + } + + /** + * 因为所有字段都是字符类型, + * + * @param name + * @return + */ + private static MetaDataField createMetaDataField(String name, DataType dataType) { + MetaDataField metaDataField = new MetaDataField(); + metaDataField.setFieldName(name); + metaDataField.setIsRequired(false); + metaDataField.setDataType(dataType); + return metaDataField; + } + + public static String getMonitorOutputLevel() { + return monitorOutputLevel; + } + protected void initProperty() { String selfMonitorOutputLevel = monitorOutputLevel + "." + name; - String level = ComponentCreator.getProperties().getProperty(selfMonitorOutputLevel); + String level = SystemContext.getStringParameter(selfMonitorOutputLevel); if (StringUtil.isNotEmpty(level)) { this.level = level; } - String timeoutStr = ComponentCreator.getProperties().getProperty(ConfigureFileKey.MONITOR_SLOW_TIMEOUT + "." + name); + String timeoutStr = SystemContext.getStringParameter(ConfigurationKey.MONITOR_SLOW_TIMEOUT + "." + name); if (StringUtil.isNotEmpty(timeoutStr)) { this.timeoutSecond = Integer.valueOf(timeoutStr); } @@ -137,7 +164,7 @@ public IMonitor createChildren(String... childrenName) { @Override public IMonitor createChildren(IConfigurable configurable) { - String name = MapKeyUtil.createKeyBySign(".", configurable.getType(), configurable.getNameSpace(), configurable.getConfigureName()); + String name = MapKeyUtil.createKeyBySign(".", configurable.getType(), configurable.getNameSpace(), configurable.getName()); return createChildren(name); } @@ -180,6 +207,20 @@ public String getType() { return type; } + // protected JSONObject createChildren(){ + // JSONObject result=null; + // if(this.children!=null&&this.children.size()>0){ + // //如果有子记录,则把子记录记录下来 + // result=new JSONObject(); + // JSONArray jsonArray=new JSONArray(); + // for(int i=0;i0){ - // //如果有子记录,则把子记录记录下来 - // result=new JSONObject(); - // JSONArray jsonArray=new JSONArray(); - // for(int i=0;i 0) { //如果有上下文信息,则把上下文放进去 result = new JSONObject(); JSONArray jsonArray = new JSONArray(); - for (int i = 0; i < contextMsgs.size(); i++) { - jsonArray.add(contextMsgs.get(i)); - } + jsonArray.addAll(contextMsgs); result.put(MONITOR_CONTEXT_MSG, jsonArray); } return result; @@ -281,11 +306,11 @@ public JSONObject report(String level) { result.put(MONITOR_COST, cost); //只输出到慢日志文件 - if (level.indexOf(MONITO_SLOW) != -1) { + if (level.contains(MONITO_SLOW)) { result.put(MONTIOR_SLOW, isSlow()); Object object = this.getValue(); if (object != null) { - if (JSONObject.class.isInstance(object)) { + if (object instanceof JSONObject) { result.put(MONITOR_RESULT, object); } else { DataType dataType = DataTypeUtil.getDataTypeFromClass(object.getClass()); @@ -294,14 +319,14 @@ public JSONObject report(String level) { } } //只输出到错误日志文件 - if (level.indexOf(MONITO_ERROR) != -1) { + if (level.contains(MONITO_ERROR)) { JSONObject error = createErrorJson(); if (error != null) { result.put(MONITOR_ERROR_MSG, error); } } //只输出到info日志目录 - if (level.indexOf(MONITO_INFO) != -1) { + if (level.contains(MONITO_INFO)) { JSONObject context = createContext(); if (context != null) { result.put(MONITOR_CONTEXT_MSG, context); @@ -312,7 +337,7 @@ public JSONObject report(String level) { result.put(MONTIOR_SLOW, isSlow()); Object object = this.getValue(); if (object != null) { - if (JSONObject.class.isInstance(object)) { + if (object instanceof JSONObject) { result.put(MONITOR_RESULT, object); } else { DataType dataType = DataTypeUtil.getDataTypeFromClass(object.getClass()); @@ -328,57 +353,19 @@ public JSONObject report(String level) { return result; } - /** - * 输出时会用到,比如输出到数据库,则每个字段名,需要和metadata一致 - */ - private static MetaData metaData = new MetaData(); - - static { - metaData.setTableName("monitor_data"); - metaData.setIdFieldName("id"); - metaData.getMetaDataFields().add(createMetaDataField("id", new LongDataType())); - metaData.getMetaDataFields().add(createMetaDataField(MONITOR_CHILDREN)); - metaData.getMetaDataFields().add(createMetaDataField(MONITOR_CONTEXT_MSG)); - metaData.getMetaDataFields().add(createMetaDataField(MONITOR_COST, new IntDataType())); - metaData.getMetaDataFields().add(createMetaDataField(MONITOR_ERROR_MSG)); - metaData.getMetaDataFields().add(createMetaDataField(MONITOR_SAMPLE_DATA)); - metaData.getMetaDataFields().add(createMetaDataField(MONITOR_SUCCESS, new BooleanDataType())); - metaData.getMetaDataFields().add(createMetaDataField(MONTIOR_SLOW, new BooleanDataType())); - metaData.getMetaDataFields().add(createMetaDataField(MONTIOR_NAME)); - - } - - private static MetaDataField createMetaDataField(String name) { - return createMetaDataField(name, new StringDataType()); - } - - /** - * 因为所有字段都是字符类型, - * - * @param name - * @return - */ - private static MetaDataField createMetaDataField(String name, DataType dataType) { - MetaDataField metaDataField = new MetaDataField(); - metaDataField.setFieldName(name); - metaDataField.setIsRequired(false); - metaDataField.setDataType(dataType); - return metaDataField; - } - @Override public void output() { String level = this.level.toUpperCase(); - if (level.indexOf(MONITO_INFO) != -1) { + if (level.contains(MONITO_INFO)) { output2Channel(MONITO_INFO); } //慢日志文件 - if (level.indexOf(MONITO_SLOW) != -1 && timeoutSecond != null && isSlow()) { + if (level.contains(MONITO_SLOW) && timeoutSecond != null && isSlow()) { output2Channel(MONITO_SLOW); } //错误日志文件 - if (level.indexOf(MONITO_ERROR) != -1 && success == false) { + if (level.contains(MONITO_ERROR) && !success) { output2Channel(MONITO_ERROR); } @@ -391,19 +378,19 @@ public void output() { */ protected void output2Channel(String level) { JSONObject result = report(level); - List outputDataSourceList = MonitorFactory.getOutputDataSource(name, level); + List> outputDataSourceList = MonitorFactory.getOutputDataSource(name, level); if (outputDataSourceList == null) { outputDataSourceList = new ArrayList<>(); } - ISink loggerOutputDataSource = MonitorFactory.createOrGetLogOutputDatasource( + ISink loggerOutputDataSource = MonitorFactory.createOrGetLogOutputDatasource( this.name + "_" + level.toLowerCase()); if (loggerOutputDataSource != null) { outputDataSourceList.add(loggerOutputDataSource); } else { LOG.error("loggerOutputDataSource is null name=" + name + " level=" + level.toLowerCase()); } - for (ISink channel : outputDataSourceList) { + for (ISink channel : outputDataSourceList) { if (channel == null) { LOG.error("channel is null name=" + name + " level=" + level.toLowerCase() + " size" + outputDataSourceList.size()); continue; @@ -414,15 +401,15 @@ protected void output2Channel(String level) { } } catch (Exception e) { - LOG.error("openAutoFlush error" + e.getMessage() + channel.getConfigureName() + "" + channel.getClass(), e); + LOG.error("openAutoFlush error" + e.getMessage() + channel.getName() + "" + channel.getClass(), e); } try { if (channel != null) { - channel.batchAdd(new Message(result),null); + channel.batchAdd(new Message(result), null); } } catch (Exception e) { - LOG.error("batchAdd error" + e.getMessage() + channel.getConfigureName() + "" + channel.getClass(), e); + LOG.error("batchAdd error" + e.getMessage() + channel.getName() + "" + channel.getClass(), e); } } @@ -456,9 +443,7 @@ public JSONObject getSampleData() { @Override public JSONObject setSampleData(AbstractContext context) { - JSONObject sampleData = getSampleData(context); - - this.sampleData = sampleData; + this.sampleData = getSampleData(context); return this.sampleData; } @@ -505,8 +490,4 @@ public Object getValue() { public List getContextMsgs() { return contextMsgs; } - - public static String getMonitorOutputLevel() { - return monitorOutputLevel; - } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/impl/MonitorItem.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/impl/MonitorItem.java index 9817f948..d6f61774 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/impl/MonitorItem.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/impl/MonitorItem.java @@ -30,6 +30,8 @@ public class MonitorItem extends JSONObject { protected Exception e; protected String name; protected int index; + private JSONObject groupInfos = new JSONObject(); + private Set noRepeate = new HashSet<>(); public MonitorItem(String name) { this.name = name; @@ -62,7 +64,7 @@ public MonitorItem endMonitor() { } public long getCost() { - return (Long)this.get("cost"); + return (Long) this.get("cost"); } public MonitorItem occureError(Exception e, String... messages) { @@ -93,9 +95,6 @@ public void addMessage(String key, JSONObject value) { this.put(key, value); } - private JSONObject groupInfos = new JSONObject(); - private Set noRepeate = new HashSet<>(); - /** * 基于组增加日志,每组日志存储在一起 * diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/impl/NothingMonitorItem.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/impl/NothingMonitorItem.java index 314393d8..02914b4f 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/impl/NothingMonitorItem.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/impl/NothingMonitorItem.java @@ -21,6 +21,8 @@ //请用imonitor和dippermonitor public class NothingMonitorItem extends MonitorItem { + public static JSONObject emptyJson = new JSONObject(); + public NothingMonitorItem(String name) { super(name); } @@ -44,8 +46,6 @@ public void addMessage(String key, String value) { } - public static JSONObject emptyJson = new JSONObject(); - public JSONObject compare(JSONObject oriMessage, JSONObject destMessage) { return emptyJson; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/JobStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/JobStage.java index ebb61924..bc1c7e17 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/JobStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/JobStage.java @@ -110,13 +110,6 @@ public class JobStage { private long createTime = System.currentTimeMillis(); - /** - * setter for column 主键 - */ - public void setId(int id) { - this.id = id; - } - /** * getter for column 主键 */ @@ -125,10 +118,10 @@ public int getId() { } /** - * setter for column 任务名称 + * setter for column 主键 */ - public void setJobName(String jobName) { - this.jobName = jobName; + public void setId(int id) { + this.id = id; } /** @@ -139,10 +132,10 @@ public String getJobName() { } /** - * setter for column 机器名称 + * setter for column 任务名称 */ - public void setMachineName(String machineName) { - this.machineName = machineName; + public void setJobName(String jobName) { + this.jobName = jobName; } /** @@ -153,10 +146,10 @@ public String getMachineName() { } /** - * setter for column stage名称 + * setter for column 机器名称 */ - public void setStageName(String stageName) { - this.stageName = stageName; + public void setMachineName(String machineName) { + this.machineName = machineName; } /** @@ -167,10 +160,10 @@ public String getStageName() { } /** - * setter for column stage类型 + * setter for column stage名称 */ - public void setStageType(String stageType) { - this.stageType = stageType; + public void setStageName(String stageName) { + this.stageName = stageName; } /** @@ -181,10 +174,10 @@ public String getStageType() { } /** - * setter for column 输入数据量 + * setter for column stage类型 */ - public void setInput(long input) { - this.input = input; + public void setStageType(String stageType) { + this.stageType = stageType; } /** @@ -195,10 +188,10 @@ public long getInput() { } /** - * setter for column 输出数据量 + * setter for column 输入数据量 */ - public void setOutput(long output) { - this.output = output; + public void setInput(long input) { + this.input = input; } /** @@ -208,6 +201,13 @@ public long getOutput() { return this.output; } + /** + * setter for column 输出数据量 + */ + public void setOutput(long output) { + this.output = output; + } + public double getTps() { return tps; } @@ -216,13 +216,6 @@ public void setTps(double tps) { this.tps = tps; } - /** - * setter for column 最后输入数据 - */ - public void setLastInputMsg(String lastInputMsg) { - this.lastInputMsg = lastInputMsg; - } - /** * getter for column 最后输入数据 */ @@ -231,10 +224,10 @@ public String getLastInputMsg() { } /** - * setter for column 最后输入数据时间 + * setter for column 最后输入数据 */ - public void setLastInputMsgTime(Date lastInputMsgTime) { - this.lastInputMsgTime = lastInputMsgTime; + public void setLastInputMsg(String lastInputMsg) { + this.lastInputMsg = lastInputMsg; } /** @@ -245,10 +238,10 @@ public Date getLastInputMsgTime() { } /** - * setter for column 最后输出数据时间 + * setter for column 最后输入数据时间 */ - public void setLastOutputMsgTime(Date lastOutputMsgTime) { - this.lastOutputMsgTime = lastOutputMsgTime; + public void setLastInputMsgTime(Date lastInputMsgTime) { + this.lastInputMsgTime = lastInputMsgTime; } /** @@ -259,10 +252,10 @@ public Date getLastOutputMsgTime() { } /** - * setter for column stage下游集合 + * setter for column 最后输出数据时间 */ - public void setNextStageLables(String nextStageLables) { - this.nextStageLables = nextStageLables; + public void setLastOutputMsgTime(Date lastOutputMsgTime) { + this.lastOutputMsgTime = lastOutputMsgTime; } /** @@ -273,10 +266,10 @@ public String getNextStageLables() { } /** - * setter for column stage上游集合 + * setter for column stage下游集合 */ - public void setPrevStageLables(String prevStageLables) { - this.prevStageLables = prevStageLables; + public void setNextStageLables(String nextStageLables) { + this.nextStageLables = nextStageLables; } /** @@ -287,10 +280,10 @@ public String getPrevStageLables() { } /** - * setter for column stage内容 + * setter for column stage上游集合 */ - public void setStageContent(String stageContent) { - this.stageContent = stageContent; + public void setPrevStageLables(String prevStageLables) { + this.prevStageLables = prevStageLables; } /** @@ -300,6 +293,13 @@ public String getStageContent() { return this.stageContent; } + /** + * setter for column stage内容 + */ + public void setStageContent(String stageContent) { + this.stageContent = stageContent; + } + public Date getPingTime() { return pingTime; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/TraceIdsDO.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/TraceIdsDO.java index a205cde9..3e033ee0 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/TraceIdsDO.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/TraceIdsDO.java @@ -52,13 +52,6 @@ public class TraceIdsDO { private String useStatus; private String jobName; - /** - * setter for column 主键 - */ - public void setId(int id) { - this.id = id; - } - /** * getter for column 主键 */ @@ -67,10 +60,10 @@ public int getId() { } /** - * setter for column tarceid + * setter for column 主键 */ - public void setTraceId(String traceId) { - this.traceId = traceId; + public void setId(int id) { + this.id = id; } /** @@ -81,10 +74,10 @@ public String getTraceId() { } /** - * setter for column 过期时间 + * setter for column tarceid */ - public void setGmtExpire(Date gmtExpire) { - this.gmtExpire = gmtExpire; + public void setTraceId(String traceId) { + this.traceId = traceId; } /** @@ -95,10 +88,10 @@ public Date getGmtExpire() { } /** - * setter for column 创建时间 + * setter for column 过期时间 */ - public void setGmtCreate(Date gmtCreate) { - this.gmtCreate = gmtCreate; + public void setGmtExpire(Date gmtExpire) { + this.gmtExpire = gmtExpire; } /** @@ -108,6 +101,13 @@ public Date getGmtCreate() { return this.gmtCreate; } + /** + * setter for column 创建时间 + */ + public void setGmtCreate(Date gmtCreate) { + this.gmtCreate = gmtCreate; + } + public String getUseStatus() { return useStatus; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/TraceMonitorDO.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/TraceMonitorDO.java index 5f278883..167a699f 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/TraceMonitorDO.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/TraceMonitorDO.java @@ -94,13 +94,6 @@ public class TraceMonitorDO { */ private String jobName; - /** - * setter for column 主键 - */ - public void setId(int id) { - this.id = id; - } - /** * getter for column 主键 */ @@ -109,10 +102,10 @@ public int getId() { } /** - * setter for column traceid + * setter for column 主键 */ - public void setTraceId(String traceId) { - this.traceId = traceId; + public void setId(int id) { + this.id = id; } /** @@ -123,10 +116,10 @@ public String getTraceId() { } /** - * setter for column stagename + * setter for column traceid */ - public void setStageName(String stageName) { - this.stageName = stageName; + public void setTraceId(String traceId) { + this.traceId = traceId; } /** @@ -137,10 +130,10 @@ public String getStageName() { } /** - * setter for column 输入数量 + * setter for column stagename */ - public void setInputNumber(int inputNumber) { - this.inputNumber = inputNumber; + public void setStageName(String stageName) { + this.stageName = stageName; } /** @@ -151,10 +144,10 @@ public int getInputNumber() { } /** - * setter for column 输入最后一条消息 + * setter for column 输入数量 */ - public void setInputLastMsg(String inputLastMsg) { - this.inputLastMsg = inputLastMsg; + public void setInputNumber(int inputNumber) { + this.inputNumber = inputNumber; } /** @@ -165,10 +158,10 @@ public String getInputLastMsg() { } /** - * setter for column 输出数量 + * setter for column 输入最后一条消息 */ - public void setOutputNumber(int outputNumber) { - this.outputNumber = outputNumber; + public void setInputLastMsg(String inputLastMsg) { + this.inputLastMsg = inputLastMsg; } /** @@ -179,10 +172,10 @@ public int getOutputNumber() { } /** - * setter for column 输出最后一条消息 + * setter for column 输出数量 */ - public void setOutputLastMsg(String outputLastMsg) { - this.outputLastMsg = outputLastMsg; + public void setOutputNumber(int outputNumber) { + this.outputNumber = outputNumber; } /** @@ -192,6 +185,13 @@ public String getOutputLastMsg() { return this.outputLastMsg; } + /** + * setter for column 输出最后一条消息 + */ + public void setOutputLastMsg(String outputLastMsg) { + this.outputLastMsg = outputLastMsg; + } + public Date getLastInputMsgTime() { return lastInputMsgTime; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/impl/HttpMonitorDataSyncImpl.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/impl/HttpMonitorDataSyncImpl.java index b0bf8d6a..64c5820e 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/impl/HttpMonitorDataSyncImpl.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/impl/HttpMonitorDataSyncImpl.java @@ -27,9 +27,7 @@ import java.nio.charset.UnsupportedCharsetException; import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; import java.util.List; -import java.util.Map; import org.apache.http.HttpEntity; import org.apache.http.ParseException; import org.apache.http.client.methods.CloseableHttpResponse; @@ -45,14 +43,10 @@ public class HttpMonitorDataSyncImpl implements MonitorDataSyncService { - protected String endPoint; - public static final String GET_TRACE_IDS = "/queryValidTraceId"; - public static final String UPDATE_JOBSTAGE = "/updateJobStage"; - public static final String ADD_TRACEMONITOR = "/insertTraceMonitor"; - + protected String endPoint; protected HttpUtil client; public HttpMonitorDataSyncImpl(String accessId, String accessIdSecret, String endPoint) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/impl/RocketMQMonitorDataSyncImpl.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/impl/RocketMQMonitorDataSyncImpl.java index 95707ab5..8691850a 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/impl/RocketMQMonitorDataSyncImpl.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/impl/RocketMQMonitorDataSyncImpl.java @@ -19,34 +19,32 @@ import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.JSONArray; import com.alibaba.fastjson.JSONObject; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collection; import java.util.List; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyStatus; import org.apache.rocketmq.client.consumer.listener.MessageListenerOrderly; import org.apache.rocketmq.client.producer.DefaultMQProducer; import org.apache.rocketmq.common.message.Message; import org.apache.rocketmq.common.message.MessageExt; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.monitor.DataSyncConstants; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.configuration.SystemContext; import org.apache.rocketmq.streams.common.monitor.model.JobStage; import org.apache.rocketmq.streams.common.monitor.model.TraceIdsDO; import org.apache.rocketmq.streams.common.monitor.model.TraceMonitorDO; import org.apache.rocketmq.streams.common.monitor.service.MonitorDataSyncService; -import org.apache.rocketmq.streams.common.utils.IPUtil; import org.apache.rocketmq.streams.common.utils.RuntimeUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class RocketMQMonitorDataSyncImpl implements MonitorDataSyncService { - protected final Log LOG = LogFactory.getLog(RocketMQMonitorDataSyncImpl.class); - public static final String QUERYVALIDTRACEID = "queryValidTraceId"; public static final String UPDATEJOBSTAGE = "updateJobStage"; public static final String INSERTTRACEMONITOR = "insertTraceMonitor"; - + protected final Logger LOG = LoggerFactory.getLogger(RocketMQMonitorDataSyncImpl.class); private Updater updater; private List traceIdsDOS; @@ -108,14 +106,12 @@ private String dealMessage(String message) { class Updater { public static final String RESULT_SUCCESS = "success"; public static final String RESULT_FAILED = "failed"; - + public String CHARSET = "UTF-8"; protected Long pullIntervalMs; - protected String ruleUpTopic = ComponentCreator.getProperties().getProperty(DataSyncConstants.RULE_UP_TOPIC); - protected String ruleUpTag = ComponentCreator.getProperties().getProperty(DataSyncConstants.RULE_UP_TAG, "T_MSG_DIPPER_RULE"); - protected String ruleDownTopic = ComponentCreator.getProperties().getProperty(DataSyncConstants.RULE_DOWN_TOPIC); - protected String ruleDownTag = ComponentCreator.getProperties().getProperty(DataSyncConstants.RULE_DOWN_TAG, "T_MSG_DIPPER_RULE_PUSH"); - public String CHARSET = "UTF-8"; - + protected String ruleUpTopic = SystemContext.getProperty(ConfigurationKey.RULE_UP_TOPIC); + protected String ruleUpTag = SystemContext.getProperty(ConfigurationKey.RULE_UP_TAG, "T_MSG_DIPPER_RULE"); + protected String ruleDownTopic = SystemContext.getProperty(ConfigurationKey.RULE_DOWN_TOPIC); + protected String ruleDownTag = SystemContext.getProperty(ConfigurationKey.RULE_DOWN_TAG, "T_MSG_DIPPER_RULE_PUSH"); protected transient DefaultMQProducer producer; public void init() { @@ -170,10 +166,26 @@ protected void sendMsg(JSONObject msg) { sendMsg(msg, ruleUpTopic); } +// protected void sendMsg(byte[] msg) { +// sendMsg(msg, ruleUpTopic); +// } + protected void sendMsg(JSONObject msg, String topic) { try { - LOG.info("sendMsg is: " + msg.toJSONString() + " topic is: " + topic + " tag is: " + ruleUpTag); - Message message = new Message(topic, ruleUpTag, null, msg.toJSONString().getBytes("UTF-8")); +// byte[] bytes = CompressUtil.gZip(msg.toJSONString()); + byte[] bytes = msg.toJSONString().getBytes(StandardCharsets.UTF_8); + LOG.info("sendMsg is: " + msg.toJSONString() + " topic is: " + topic + " tag is: " + ruleUpTag + " byte length is " + bytes.length); + Message message = new Message(topic, ruleUpTag, null, bytes); + producer.send(message); + } catch (Exception e) { + LOG.error("updater sendMsg error: ", e); + e.printStackTrace(); + } + } + + protected void sendMsg(byte[] msg, String topic) { + try { + Message message = new Message(topic, ruleUpTag, null, msg); producer.send(message); } catch (Exception e) { LOG.error("updater sendMsg error: ", e); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/FilterResultCache.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/FilterResultCache.java index d5320f49..dadd30ba 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/FilterResultCache.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/FilterResultCache.java @@ -20,33 +20,36 @@ import org.apache.rocketmq.streams.common.cache.compress.BitSetCache; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; + /** * cache the expressions' result */ public abstract class FilterResultCache { protected BitSetCache.BitSet bitSet; - protected Map expression2IndexMap; + protected Map expression2IndexMap; - public FilterResultCache(BitSetCache.BitSet bitSet, Map expression2IndexMap){ - this.bitSet=bitSet; - this.expression2IndexMap=expression2IndexMap; + public FilterResultCache(BitSetCache.BitSet bitSet, Map expression2IndexMap) { + this.bitSet = bitSet; + this.expression2IndexMap = expression2IndexMap; } + /** * if the expression in cache ,return the cache result else return null + * * @param expression regex/like/other * @return */ - public Boolean isMatch(String varName,String functionName, String expression){ - Integer index=this.expression2IndexMap.get(MapKeyUtil.createKey(varName,functionName,expression)); - if(index==null||bitSet==null){ + public Boolean isMatch(String varName, String functionName, String expression) { + Integer index = this.expression2IndexMap.get(MapKeyUtil.createKey(varName, functionName, expression)); + if (index == null || bitSet == null) { return null; } - return bitSet.get(index); + return bitSet.get(index); } public Map getExpression2IndexMap() { return expression2IndexMap; } - public abstract Boolean isMatch(IMessage msg,Object expression); + public abstract Boolean isMatch(IMessage msg, Object expression); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/HomologousVar.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/HomologousVar.java index 43457f0b..f917f4f7 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/HomologousVar.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/HomologousVar.java @@ -21,8 +21,7 @@ public class HomologousVar { protected String varName; protected Integer index; - - public String getHomologousVarKey(){ + public String getHomologousVarKey() { return sourceVarName; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/IHomologousCalculate.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/IHomologousCalculate.java new file mode 100644 index 00000000..ca9d9a1f --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/IHomologousCalculate.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.optimization; + +import org.apache.rocketmq.streams.common.context.AbstractContext; +import org.apache.rocketmq.streams.common.context.IMessage; + +public interface IHomologousCalculate { + void calculate(IMessage message, AbstractContext context); +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/IHomologousOptimization.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/IHomologousOptimization.java index ffbaf502..5545a0ab 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/IHomologousOptimization.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/IHomologousOptimization.java @@ -17,14 +17,18 @@ package org.apache.rocketmq.streams.common.optimization; import java.util.List; -import org.apache.rocketmq.streams.common.context.AbstractContext; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; +import org.apache.rocketmq.streams.common.topology.metric.NotFireReason; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; +import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; public interface IHomologousOptimization { - void optimizate(List> pipelineList, int cacheSize, int preFingerprintCacheSize); + void optimizate(List> pipelineList); + IHomologousCalculate createHomologousCalculate(); + + void setPeFingerprintForPipeline(ChainPipeline pipeline); + + NotFireReason analysisNotFireReason(FilterChainStage stage, String fieldValue, List notFireExpressionMonitor); - void calculate(IMessage message, AbstractContext context); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/LikeRegex.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/LikeRegex.java index 10d7cd92..21c0e525 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/LikeRegex.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/LikeRegex.java @@ -18,35 +18,43 @@ import java.util.ArrayList; import java.util.List; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * 可以用sql中的like表示正则,系统负责完成转化 */ public class LikeRegex { - private static final Log LOG = LogFactory.getLog(LikeRegex.class); public static final String SPECAIL_WORD = "%"; + private static final Logger LOG = LoggerFactory.getLogger(LikeRegex.class); public static String[] regexSpecialWords = {"\\(", "\\)", "\\*", "\\+", "\\.", "\\[", "\\]", "\\?", "\\^", "\\{", "\\}", "\\|"}; //public static String[] regexSpecialWords = {"\\\\","\\$", "\\(", "\\)", "\\*", "\\+", "\\.", "\\[", "\\]", "\\?", "\\^", "\\{", "\\}", "\\|"}; protected String likeStr; protected boolean isStartFlag = true; protected boolean isEndFlag = true; - protected boolean hasUnderline=false; + protected boolean hasUnderline = false; protected List quickMatchWord = new ArrayList<>(); - protected List specailWordIndex=new ArrayList<>(); + protected List specailWordIndex = new ArrayList<>(); public LikeRegex(String likeStr) { this.likeStr = likeStr; parse(); } + public static void main(String[] args) { + String content = "xCurrentVersion\\Windows\\load"; + String likeStr = "$Current$ersionWindows?\\load$"; + LikeRegex likeRegex = new LikeRegex(likeStr); + + System.out.println(likeRegex.createRegex()); + } + public void parse() { String tmp = likeStr; - if(tmp.indexOf("_")!=-1){ - hasUnderline=true; + if (tmp.indexOf("_") != -1) { + hasUnderline = true; } if (tmp == null) { return; @@ -74,9 +82,9 @@ public boolean match(String content) { if (content == null) { return false; } - if(hasUnderline){ - String regex=createRegex(); - return StringUtil.matchRegex(content,regex); + if (hasUnderline) { + String regex = createRegex(); + return StringUtil.matchRegex(content, regex); } if (quickMatchWord == null || quickMatchWord.size() == 0) { LOG.warn("like may be parse error, words is empty " + likeStr); @@ -102,50 +110,50 @@ public boolean match(String content) { return true; } - public String createRegex(){ - StringBuilder regex=new StringBuilder(); + public String createRegex() { + StringBuilder regex = new StringBuilder(); - boolean isFirst=true; - for(String word:this.quickMatchWord){ - if(isFirst){ - isFirst=false; - }else { + boolean isFirst = true; + for (String word : this.quickMatchWord) { + if (isFirst) { + isFirst = false; + } else { regex.append(".*"); } - String subRegex=word; + String subRegex = word; - for(String regexSpecialWord:regexSpecialWords){ + for (String regexSpecialWord : regexSpecialWords) { try { - subRegex = subRegex.replaceAll(regexSpecialWord,"\\"+regexSpecialWord); + subRegex = subRegex.replaceAll(regexSpecialWord, "\\" + regexSpecialWord); - }catch (Exception e){ - LOG.error(regexSpecialWord+" "+subRegex+"\r\n"+ e.getMessage(),e); + } catch (Exception e) { + LOG.error(regexSpecialWord + " " + subRegex + "\r\n" + e.getMessage(), e); throw new RuntimeException(e); } } - if (subRegex.indexOf("$")!=-1){ - String newSubRegex=""; - for(int i=0;i= sourceCount) { return (targetCount == 0 ? sourceCount : -1); } @@ -205,7 +213,9 @@ public int indexOf(char[] source, int sourceOffset, int sourceCount, for (int i = sourceOffset + fromIndex; i <= max; i++) { /* Look for first character. */ if (source[i] != first && first != '_') { - while (++i <= max && source[i] != first && first != '_') { ; } + while (++i <= max && source[i] != first && first != '_') { + ; + } } /* Found first character, now look at the rest of v2 */ @@ -213,7 +223,9 @@ public int indexOf(char[] source, int sourceOffset, int sourceCount, int j = i + 1; int end = j + targetCount - 1; for (int k = targetOffset + 1; j < end && (source[j] - == target[k] || target[k] == '_'); j++, k++) { ; } + == target[k] || target[k] == '_'); j++, k++) { + ; + } if (j == end) { /* Found whole string. */ @@ -223,12 +235,4 @@ public int indexOf(char[] source, int sourceOffset, int sourceCount, } return -1; } - - public static void main(String[] args) { - String content = "xCurrentVersion\\Windows\\load"; - String likeStr = "$Current$ersionWindows?\\load$"; - LikeRegex likeRegex = new LikeRegex(likeStr); - - System.out.println(likeRegex.createRegex()); - } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/MessageGlobleTrace.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/MessageTrace.java similarity index 67% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/MessageGlobleTrace.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/MessageTrace.java index f33a7a2c..6f72b31d 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/MessageGlobleTrace.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/MessageTrace.java @@ -22,12 +22,13 @@ /** * 判断pipeline是否完成一个分支的执行,如果未完成,可以配合LogFingerprintFilter做指纹过滤。某个指纹如果确定不触发某个规则,直接丢弃、 在pipeline中消息会被拆分,在有多分支时,会被copy,这个对象会在任何变动时,都保持全局唯一,不允许copy,复制,创建,一个message全局唯一 */ -public class MessageGlobleTrace { +public class MessageTrace { protected boolean existFinishBranch = false;//存在有完成的分支 - private MessageGlobleTrace() { + private MessageTrace() { } + /** * 是否有完成的分支 * @@ -35,7 +36,7 @@ private MessageGlobleTrace() { * @return */ public static Boolean existFinishBranch(IMessage message) { - MessageGlobleTrace trace = message.getHeader().getMessageGlobalTrace(); + MessageTrace trace = message.getHeader().getMessageTrace(); if (trace == null) { return null; } @@ -48,7 +49,7 @@ public static Boolean existFinishBranch(IMessage message) { * @param message */ public static void finishPipeline(IMessage message) { - MessageGlobleTrace trace = message.getHeader().getMessageGlobalTrace(); + MessageTrace trace = message.getHeader().getMessageTrace(); if (trace == null) { return; } @@ -61,19 +62,28 @@ public static void finishPipeline(IMessage message) { * @param message * @return */ - public static MessageGlobleTrace joinMessage(IMessage message) { + public static MessageTrace joinMessage(IMessage message) { MessageHeader messageHeader = message.getHeader(); - if (messageHeader.getMessageGlobalTrace() == null) { - messageHeader.setMessageGlobalTrace(new MessageGlobleTrace()); + if (messageHeader.getMessageTrace() == null) { + messageHeader.setMessageTrace(new MessageTrace()); } - return messageHeader.getMessageGlobalTrace(); + return messageHeader.getMessageTrace(); } - public static void clear(IMessage message) { - MessageHeader messageHeader = message.getHeader(); - if (messageHeader.getMessageGlobalTrace() == null) { - messageHeader.setMessageGlobalTrace(new MessageGlobleTrace()); + public static void setResult(IMessage message, Boolean result) { + MessageTrace trace = message.getHeader().getMessageTrace(); + if (trace == null || result == null) { + return; } - messageHeader.getMessageGlobalTrace().existFinishBranch = false; + trace.existFinishBranch = result; } + + // public static void clear(IMessage message) { + // MessageHeader messageHeader = message.getHeader(); + // if (messageHeader.getMessageGlobalTrace() == null) { + // messageHeader.setMessageGlobalTrace(new MessageGlobleTrace()); + // } + // + // messageHeader.getMessageGlobalTrace().existFinishBranch = false; + // } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/OptimizationRegex.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/OptimizationRegex.java index db890d62..7c37f4cf 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/OptimizationRegex.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/OptimizationRegex.java @@ -26,17 +26,13 @@ import org.apache.rocketmq.streams.common.utils.StringUtil; public class OptimizationRegex { + //正则表达式,对应的关键词优化器 + public static Map optimizationRegexMap = new HashMap<>(); private static int orWordCount = 21;//对于or,超过多少就不处理了 - private static String[] regexSpecialWords = {"\\", "$", "(", ")", "*", "+", ".", "[", "]", "?", "^", "{", "}", "|"}; //正则表达式特殊字符 private static String[] replaceSpecialWords = {"&", "@", "~"};//用于替换转义字符的特殊字符 - private static String[] regexSpecialWordsForSplit = {"$", "*", "+", ".", "?", "^"};//分割单词的分割符 - - //正则表达式,对应的关键词优化器 - public static Map optimizationRegexMap = new HashMap<>(); - protected String regex;//原来的正则表达式 protected boolean supportOptimizate = false;//是否支持优化 protected List andWords = new ArrayList<>();//and 关系的关键词 @@ -288,7 +284,6 @@ public String toString() { return MapKeyUtil.createKey("&", allWordList); } - public boolean isSupportOptimizate() { return supportOptimizate; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/RegexEngine.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/RegexEngine.java index 7f439da2..9afaad4a 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/RegexEngine.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/RegexEngine.java @@ -18,29 +18,32 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.configuration.SystemContext; /** * regex engine */ public class RegexEngine { - protected AtomicBoolean hasCompile = new AtomicBoolean(false); - protected static final String RE2J_ENGINE = "re2j"; - protected static final String HYPER_SCAN_ENGINE = "hyperscan"; - + protected AtomicBoolean hasCompile = new AtomicBoolean(false); protected IStreamRegex engine = new HyperscanEngine<>(); public RegexEngine() { - String option = ComponentCreator.getProperties().getProperty(ConfigureFileKey.DIPPER_REGEX_ENGINE); + String option = SystemContext.getStringParameter(ConfigurationKey.DIPPER_REGEX_ENGINE); if (RE2J_ENGINE.equalsIgnoreCase(option)) { engine = new Re2Engine<>(); } } + public RegexEngine(boolean isRE2J) { + if (isRE2J) { + engine = new Re2Engine<>(); + } + } + public void addRegex(String regex, T context) { engine.addRegex(regex, context); hasCompile.set(false); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/TaskOptimization.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/TaskOptimization.java deleted file mode 100644 index 2f0bd955..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/TaskOptimization.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * 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.optimization; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.ServiceLoader; -import org.apache.rocketmq.streams.common.context.AbstractContext; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; - -public class TaskOptimization { - protected int homologousExpressionCaseSize = 1000000; - protected int preFingerprintCaseSize = 1000000; - - /** - * Automatically parses pipelines, generates pre-filter fingerprints and expression estimates - */ - protected transient volatile IHomologousOptimization homologousOptimization; - - public TaskOptimization(int homologousExpressionCaseSize, int preFingerprintCaseSize) { - this.homologousExpressionCaseSize = homologousExpressionCaseSize; - this.preFingerprintCaseSize = preFingerprintCaseSize; - } - - public void openOptimization(ChainPipeline... pipelines) { - if (pipelines == null) { - return; - } - List> pipelineList = new ArrayList<>(); - Collections.addAll(pipelineList, pipelines); - openOptimization(pipelineList); - } - - public void openOptimization(List> pipelines) { - if (this.homologousOptimization == null) { - synchronized (this) { - if (this.homologousOptimization == null) { - Iterable iterable = ServiceLoader.load(IHomologousOptimization.class); - Iterator it = iterable.iterator(); - if (it.hasNext()) { - this.homologousOptimization = it.next(); - this.homologousOptimization.optimizate(pipelines, this.homologousExpressionCaseSize, this.preFingerprintCaseSize); - } - } - } - } - } - - public void calculateOptimizationExpression(IMessage message, AbstractContext context) { - if (homologousOptimization != null) { - homologousOptimization.calculate(message, context); - } - } -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/FingerprintCache.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/FingerprintCache.java index aae3c33c..a3ac6a77 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/FingerprintCache.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/FingerprintCache.java @@ -21,7 +21,8 @@ import java.util.List; import java.util.Map; import org.apache.rocketmq.streams.common.cache.compress.BitSetCache; -import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.configuration.SystemContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.common.utils.PrintUtil; @@ -33,12 +34,20 @@ * can set cache size by sdk or property file */ public class FingerprintCache { + public static String FIELD_VALUE_SPLIT_SIGN = ";;;;;"; protected static FingerprintCache fingerprintCache; - protected static int CACHE_SIZE = 5000000;//default cache size,support 3000000 log size - public static String FIELD_VALUE_SPLIT_SIGN=";;;;;"; + protected static int CACHE_SIZE = 3000000;//default cache size,support 3000000 log size + protected static Map> logFingerprintFieldNameListMap = new HashMap<>(); + + static { + Integer sizeValue = SystemContext.getIntParameter(ConfigurationKey.FINGERPRINT_CACHE_SIZE); + if (sizeValue != null) { + CACHE_SIZE = sizeValue; + } + } + //key: namespace value:FingerprintMetric protected Map metricMap = new HashMap<>(); - protected BitSetCache bitSetCache; protected int cacheSize; protected int reHashCount = 0; @@ -46,9 +55,82 @@ public class FingerprintCache { protected Long firstUpdateTime; protected double minHitCacheRate = 0.4; - public FingerprintCache(int cacheSize) { + private FingerprintCache(int cacheSize) { this.cacheSize = cacheSize; - this.bitSetCache = new BitSetCache(this.cacheSize); + if (this.cacheSize > 0) { + this.bitSetCache = new BitSetCache(this.cacheSize); + } + + } + + public static FingerprintCache getInstance() { + if (fingerprintCache == null) { + synchronized (FingerprintCache.class) { + if (fingerprintCache == null) { + fingerprintCache = new FingerprintCache(CACHE_SIZE); + } + } + } + return fingerprintCache; + } + + /** + * 创建代表日志指纹的字符串 + * + * @param message + * @return + */ + public static String creatFingerpringKey(IMessage message, String namespace, String logFingerprintFieldNames) { + String key = MapKeyUtil.createKey(namespace, logFingerprintFieldNames); + List logFingerprintFieldNameList = logFingerprintFieldNameListMap.get(key); + if (logFingerprintFieldNameList == null) { + synchronized (FingerprintCache.class) { + logFingerprintFieldNameList = logFingerprintFieldNameListMap.get(key); + if (logFingerprintFieldNameList == null) { + + if (logFingerprintFieldNames != null) { + List list = new ArrayList<>(); + for (String name : logFingerprintFieldNames.split(",")) { + list.add(name); + } + logFingerprintFieldNameList = list; + } + } + } + } + StringBuilder sb = new StringBuilder(); + boolean isFirst = true; + for (String value : logFingerprintFieldNameList) { + String msgValue = message.getMessageBody().getString(value); + if (StringUtil.isEmpty(msgValue)) { + msgValue = ""; + } + + if (isFirst) { + isFirst = false; + } else { + sb.append(FIELD_VALUE_SPLIT_SIGN); + } + sb.append(msgValue); + } + return sb.toString(); + + } + + public void put(String namespace, String key, boolean isTrue) { + BitSetCache.BitSet bitSetCache = new BitSetCache.BitSet(1); + if (isTrue) { + bitSetCache.set(0); + } + addLogFingerprint("default", key, bitSetCache); + } + + public Boolean get(String namespace, String key) { + BitSetCache.BitSet bitSet = getLogFingerprint("default", key); + if (bitSet == null) { + return null; + } + return bitSet.get(0); } public void addLogFingerprint(String namespace, String msgKey, BitSetCache.BitSet bitSet) { @@ -84,7 +166,8 @@ public void addLogFingerprint(String namespace, String msgKey, BitSetCache.BitSe protected void executeCloseStrategy() { this.rootFingerprintMetric.print(); - if (System.currentTimeMillis() - firstUpdateTime > 1000 * 60 * 60 * 4) { + + if (firstUpdateTime != null && System.currentTimeMillis() - firstUpdateTime > 1000 * 60 * 60 * 4) { return; } if (metricMap.size() == 1) { @@ -106,6 +189,9 @@ public BitSetCache.BitSet getLogFingerprint(String namespace, String msgKey) { if (msgKey == null) { return null; } + if (this.bitSetCache == null) { + return null; + } FingerprintMetric fingerprintMetric = getOrCreateMetric(namespace); if (fingerprintMetric.isCloseFingerprint()) { return null; @@ -145,69 +231,6 @@ public FingerprintMetric getOrCreateMetric(String namespace) { return fingerprintMetric; } - public static FingerprintCache getInstance() { - if (fingerprintCache == null) { - synchronized (FingerprintCache.class) { - if (fingerprintCache == null) { - fingerprintCache = new FingerprintCache(CACHE_SIZE); - } - } - } - return fingerprintCache; - } - - static { - String sizeValue = ComponentCreator.getProperties().getProperty("fingerprint.cache.size"); - if (StringUtil.isNotEmpty(sizeValue)) { - CACHE_SIZE = Integer.valueOf(sizeValue); - } - } - - protected static Map> logFingerprintFieldNameListMap = new HashMap<>(); - - /** - * 创建代表日志指纹的字符串 - * - * @param message - * @return - */ - public static String creatFingerpringKey(IMessage message, String namespace, String logFingerprintFieldNames) { - String key = MapKeyUtil.createKey(namespace, logFingerprintFieldNames); - List logFingerprintFieldNameList = logFingerprintFieldNameListMap.get(key); - if (logFingerprintFieldNameList == null) { - synchronized (FingerprintCache.class) { - logFingerprintFieldNameList = logFingerprintFieldNameListMap.get(key); - if (logFingerprintFieldNameList == null) { - - if (logFingerprintFieldNames != null) { - List list = new ArrayList<>(); - for (String name : logFingerprintFieldNames.split(",")) { - list.add(name); - } - logFingerprintFieldNameList = list; - } - } - } - } - StringBuilder sb = new StringBuilder(); - boolean isFirst = true; - for (String value : logFingerprintFieldNameList) { - String msgValue = message.getMessageBody().getString(value); - if (StringUtil.isEmpty(msgValue)) { - msgValue = ""; - } - - if (isFirst) { - isFirst = false; - } else { - sb.append(FIELD_VALUE_SPLIT_SIGN); - } - sb.append(msgValue); - } - return sb.toString(); - - } - public double getMinHitCacheRate() { return minHitCacheRate; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/FingerprintMetric.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/FingerprintMetric.java index 93399edd..717d9e4f 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/FingerprintMetric.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/FingerprintMetric.java @@ -22,48 +22,45 @@ public class FingerprintMetric { protected String namespace; - protected AtomicLong visitCount=new AtomicLong(0); - protected AtomicLong hitCacheCount=new AtomicLong(0); - protected AtomicLong cacheSize=new AtomicLong(0); - protected boolean isCloseFingerprint=false; + protected AtomicLong visitCount = new AtomicLong(0); + protected AtomicLong hitCacheCount = new AtomicLong(0); + protected AtomicLong cacheSize = new AtomicLong(0); + protected boolean isCloseFingerprint = false; - public FingerprintMetric(String namespace){ - this.namespace=namespace; + public FingerprintMetric(String namespace) { + this.namespace = namespace; } - public void addMetric(boolean isHitCache){ + public void addMetric(boolean isHitCache) { visitCount.incrementAndGet(); - if(isHitCache){ + if (isHitCache) { hitCacheCount.incrementAndGet(); } } - - public void addCaceSize(){ + public void addCaceSize() { cacheSize.incrementAndGet(); } - public Long getVisitCount(){ + public Long getVisitCount() { return this.visitCount.get(); } - public Long getHitCacheCount(){ + public Long getHitCacheCount() { return this.hitCacheCount.get(); } - - - public Long getCacheSize(){ + public Long getCacheSize() { return this.cacheSize.get(); } - public double getHitCacheRate(){ - double visitCount=getVisitCount(); - if(visitCount==0){ - visitCount=1; + public double getHitCacheRate() { + double visitCount = getVisitCount(); + if (visitCount == 0) { + visitCount = 1; } - double hitCacheCount=getHitCacheCount(); - return hitCacheCount/visitCount; + double hitCacheCount = getHitCacheCount(); + return hitCacheCount / visitCount; } public boolean isCloseFingerprint() { @@ -81,13 +78,13 @@ public void clear() { } public void print() { - JSONObject msg=new JSONObject(); - msg.put("visitCount",visitCount.get()); - msg.put("hitCacheCount",hitCacheCount.get()); - msg.put("cacheSize",cacheSize.get()); - msg.put("hitCacheRate",getHitCacheRate()); - msg.put("namespace",namespace); - msg.put("isClosed",isCloseFingerprint); + JSONObject msg = new JSONObject(); + msg.put("visitCount", visitCount.get()); + msg.put("hitCacheCount", hitCacheCount.get()); + msg.put("cacheSize", cacheSize.get()); + msg.put("hitCacheRate", getHitCacheRate()); + msg.put("namespace", namespace); + msg.put("isClosed", isCloseFingerprint); System.out.println(JsonableUtil.formatJson(msg)); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/PreFingerprint.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/PreFingerprint.java index 2c92249a..cead9a81 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/PreFingerprint.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/PreFingerprint.java @@ -56,18 +56,18 @@ public class PreFingerprint { protected transient AbstractStage filterChainStage; protected transient List> allPreviewFilterChainStage = new LinkedList<>(); - public PreFingerprint(String logFingerFieldNames, String filterStageIdentification, String sourceStageLable, String nextStageLable, int expressionCount, AbstractStage filterChainStage, FingerprintCache fingerprintCache) { + public PreFingerprint(String logFingerFieldNames, String filterStageIdentification, String sourceStageLable, String nextStageLable, int expressionCount, AbstractStage filterChainStage) { this.logFingerFieldNames = logFingerFieldNames; this.filterStageIdentification = filterStageIdentification; this.sourceStageLabel = sourceStageLable; this.nextStageLabel = nextStageLable; this.expressionCount = expressionCount; this.filterChainStage = filterChainStage; - this.fingerprintCache = fingerprintCache; + this.fingerprintCache = FingerprintCache.getInstance(); } - public PreFingerprint(String logFingerFieldNames, String filterStageIdentification, String sourceStageLable, String nextStageLable, AbstractStage filterChainStage, FingerprintCache fingerprintCache) { - this(logFingerFieldNames, filterStageIdentification, sourceStageLable, nextStageLable, -1, filterChainStage, fingerprintCache); + public PreFingerprint(String logFingerFieldNames, String filterStageIdentification, String sourceStageLable, String nextStageLable, AbstractStage filterChainStage) { + this(logFingerFieldNames, filterStageIdentification, sourceStageLable, nextStageLable, -1, filterChainStage); } /** @@ -77,9 +77,9 @@ public PreFingerprint(String logFingerFieldNames, String filterStageIdentificati * @return */ public boolean filterByLogFingerprint(IMessage message) { - if (logFingerFieldNames != null) { + if (logFingerFieldNames != null && !message.getHeader().isSystemMessage()) { String msgKey = FingerprintCache.creatFingerpringKey(message, filterStageIdentification, logFingerFieldNames); - if (msgKey != null) { + if (msgKey != null && !"".equals(msgKey)) { BitSetCache.BitSet bitSet = fingerprintCache.getLogFingerprint(filterStageIdentification, msgKey); if (bitSet != null && bitSet.get(0)) { return true; @@ -91,6 +91,11 @@ public boolean filterByLogFingerprint(IMessage message) { return false; } + public String createFieldMsg(IMessage message) { + String msgKey = FingerprintCache.creatFingerpringKey(message, filterStageIdentification, logFingerFieldNames); + return msgKey; + } + /** * 设置过滤指纹 * @@ -110,6 +115,12 @@ public String getLogFingerFieldNames() { return logFingerFieldNames; } + public void setLogFingerFieldNames(Set logFingerFieldNames) { + List fingers = new ArrayList<>(logFingerFieldNames); + Collections.sort(fingers); + this.logFingerFieldNames = MapKeyUtil.createKey(",", fingers); + } + public String getSourceStageLabel() { return sourceStageLabel; } @@ -130,12 +141,6 @@ public AbstractStage getFilterChainStage() { return filterChainStage; } - public void setLogFingerFieldNames(Set logFingerFieldNames) { - List fingers = new ArrayList<>(logFingerFieldNames); - Collections.sort(fingers); - this.logFingerFieldNames = MapKeyUtil.createKey(",", fingers); - } - public void addPreviwFilterChainStage(List> filterChainStages) { this.allPreviewFilterChainStage.addAll(filterChainStages); } @@ -144,6 +149,15 @@ public void addPreviwFilterChainStage(AbstractStage filterChainStage) { this.allPreviewFilterChainStage.add(filterChainStage); } + public FingerprintCache getFingerprintCache() { + return fingerprintCache; + } + + public void setFingerprintCache( + FingerprintCache fingerprintCache) { + this.fingerprintCache = fingerprintCache; + } + public List> getAllPreviewFilterChainStage() { return allPreviewFilterChainStage; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/schedule/ScheduleManager.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/schedule/ScheduleManager.java deleted file mode 100644 index 8da230e7..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/schedule/ScheduleManager.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * 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.schedule; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.rocketmq.streams.common.threadpool.ThreadPoolFactory; - -public class ScheduleManager { - protected List scheduleTasks = new ArrayList<>(); - protected AtomicBoolean isStart = new AtomicBoolean(false); - protected ScheduledExecutorService scheduledExecutorService = new ScheduledThreadPoolExecutor(5); - protected ExecutorService executorService = ThreadPoolFactory.createThreadPool(2, 50); - private static ScheduleManager scheduleManager = new ScheduleManager(); - - public static ScheduleManager getInstance() { - return scheduleManager; - } - - public void regist(ScheduleTask scheduleTask) { - start(); - if (scheduleTask == null) { - return; - } - scheduleTasks.add(scheduleTask); - } - - public void start() { - if (isStart.compareAndSet(false, true)) { - scheduledExecutorService.scheduleWithFixedDelay(new Runnable() { - @Override public void run() { - try { - List list = new ArrayList<>(scheduleTasks); - for (ScheduleTask scheduleTask : list) { - if (scheduleTask != null && scheduleTask.canExecute()) { - executeTask(scheduleTask); - } - } - } catch (Exception e) { - e.printStackTrace(); - } - - } - }, 0, 100, TimeUnit.MILLISECONDS); - } - } - - protected void executeTask(ScheduleTask task) { - if (task.getExecutorService() != null) { - task.getExecutorService().execute(task.getRunnable()); - } else { - executorService.execute(task.getRunnable()); - } - } -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/schedule/ScheduleTask.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/schedule/ScheduleTask.java deleted file mode 100644 index 165578f7..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/schedule/ScheduleTask.java +++ /dev/null @@ -1,101 +0,0 @@ -/* - * 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.schedule; - -import java.util.concurrent.ExecutorService; -import java.util.concurrent.atomic.AtomicBoolean; - -public class ScheduleTask { - protected int initialDelaySecond; - protected int delaySecond; - protected Runnable runnable; - protected ExecutorService executorService; - protected Long lastExecuteTime; - protected IScheduleCondition scheduleCondition; - protected AtomicBoolean isExecuting = new AtomicBoolean(false); - - public ScheduleTask(int initialDelaySecond, int delaySecond, Runnable runnable) { - this.initialDelaySecond = initialDelaySecond; - this.delaySecond = delaySecond; - this.runnable = runnable; - if (initialDelaySecond == 0) { - lastExecuteTime = 0L; - } else { - lastExecuteTime = System.currentTimeMillis(); - } - Runnable runnableProxy = new Runnable() { - @Override - public void run() { - runnable.run(); - lastExecuteTime = System.currentTimeMillis(); - isExecuting.set(false); - } - }; - this.runnable = runnableProxy; - } - - public ScheduleTask(IScheduleCondition scheduleCondition, Runnable runnable) { - this.scheduleCondition = scheduleCondition; - Runnable runnableProxy = new Runnable() { - @Override - public void run() { - runnable.run(); - lastExecuteTime = System.currentTimeMillis(); - isExecuting.set(false); - } - }; - this.runnable = runnableProxy; - } - - public boolean canExecute() { - if (!isExecuting.compareAndSet(false, true)) { - return false; - } - if (this.scheduleCondition != null) { - boolean canExecute = (this.scheduleCondition.canExecute()); - if (!canExecute) { - isExecuting.compareAndSet(true, false); - } - return canExecute; - } - if (initialDelaySecond == 0) { - initialDelaySecond = -1; - return true; - } - if (initialDelaySecond > 0 && (System.currentTimeMillis() - lastExecuteTime > initialDelaySecond*1000)) { - initialDelaySecond = -1; - return true; - } - if (System.currentTimeMillis() - lastExecuteTime > delaySecond*1000) { - return true; - } - isExecuting.compareAndSet(true, false); - return false; - } - - public ExecutorService getExecutorService() { - return executorService; - } - - public Runnable getRunnable() { - return runnable; - } - - public void setExecutorService(ExecutorService executorService) { - this.executorService = executorService; - } -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/threadpool/ScheduleFactory.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/threadpool/ScheduleFactory.java new file mode 100644 index 00000000..4b4573eb --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/threadpool/ScheduleFactory.java @@ -0,0 +1,43 @@ +package org.apache.rocketmq.streams.common.threadpool; + +import com.google.common.collect.Maps; +import java.util.Map; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import org.apache.commons.lang3.concurrent.BasicThreadFactory; + +public class ScheduleFactory { + + private final ScheduledExecutorService scheduledExecutorService; + + private final Map> scheduledFutureMap; + + private ScheduleFactory() { + this.scheduledExecutorService = new ScheduledThreadPoolExecutor(10, new BasicThreadFactory.Builder().namingPattern("rstream-schedule-%d").build()); + this.scheduledFutureMap = Maps.newConcurrentMap(); + } + + public static ScheduleFactory getInstance() { + return ScheduleManager.Instance; + } + + public void execute(String scheduleName, Runnable command, long initialDelay, long delay, TimeUnit unit) { + ScheduledFuture scheduledFuture = this.scheduledExecutorService.scheduleWithFixedDelay(command, initialDelay, delay, unit); + this.scheduledFutureMap.put(scheduleName, scheduledFuture); + } + + public void cancel(String scheduleName) { + ScheduledFuture scheduledFuture = this.scheduledFutureMap.get(scheduleName); + if (scheduledFuture != null && !scheduledFuture.isDone()) { + scheduledFuture.cancel(false); + this.scheduledFutureMap.remove(scheduleName); + } + } + + private static class ScheduleManager { + private static final ScheduleFactory Instance = new ScheduleFactory(); + } + +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/threadpool/ThreadPoolFactory.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/threadpool/ThreadPoolFactory.java index 5cbdc489..fc6eec51 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/threadpool/ThreadPoolFactory.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/threadpool/ThreadPoolFactory.java @@ -16,7 +16,10 @@ */ package org.apache.rocketmq.streams.common.threadpool; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; @@ -24,45 +27,52 @@ import java.util.concurrent.atomic.AtomicInteger; public class ThreadPoolFactory { - public static ExecutorService createThreadPool(int coreSize){ - ExecutorService executorService= new ThreadPoolExecutor(coreSize, coreSize, - 1000*60L, TimeUnit.MILLISECONDS, - new SynchronousQueue(), new DipperThreadFactory(), new ThreadPoolExecutor.CallerRunsPolicy()); - return executorService; + public static ExecutorService createThreadPool(int coreSize, String poolNamePrefix) { + return new ThreadPoolExecutor(coreSize, coreSize, 1000 * 60L, TimeUnit.MILLISECONDS, new SynchronousQueue(), new DipperThreadFactory(poolNamePrefix), new ThreadPoolExecutor.CallerRunsPolicy()); } + public static ExecutorService createThreadPool(int min, int max, String poolNamePrefix) { + return new ThreadPoolExecutor(min, max, 1000 * 60L, TimeUnit.MILLISECONDS, new SynchronousQueue(), new DipperThreadFactory(poolNamePrefix), new ThreadPoolExecutor.CallerRunsPolicy()); + } - public static ExecutorService createThreadPool(int min,int max){ - ExecutorService executorService= new ThreadPoolExecutor(min, max, - 1000*60L, TimeUnit.MILLISECONDS, - new SynchronousQueue(),new DipperThreadFactory(), new ThreadPoolExecutor.CallerRunsPolicy()); - return executorService; + public static ExecutorService createThreadPool(int min, int max, long keepAliveTime, TimeUnit timeUnit, BlockingQueue workQueue, String poolNamePrefix) { + return new ThreadPoolExecutor(min, max, keepAliveTime, timeUnit, workQueue, new DipperThreadFactory(poolNamePrefix), new ThreadPoolExecutor.CallerRunsPolicy()); } +// public static ScheduledExecutorService createScheduledThreadPool(int coreSize, String threadNamePrefix) { +// return new ScheduledThreadPoolExecutor(coreSize, new DipperThreadFactory(threadNamePrefix)); +// } + + public static ExecutorService createCachedThreadPool(String threadNamePrefix) { + return Executors.newCachedThreadPool(new DipperThreadFactory(threadNamePrefix)); + } + + public static ExecutorService createFixedThreadPool(int nThreads, String threadNamePrefix) { + return new ThreadPoolExecutor(nThreads, nThreads, + 0L, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue(), + new DipperThreadFactory(threadNamePrefix)); + } - static class DipperThreadFactory implements ThreadFactory { - private static final AtomicInteger poolNumber = new AtomicInteger(1); + public static class DipperThreadFactory implements ThreadFactory { private final ThreadGroup group; private final AtomicInteger threadNumber = new AtomicInteger(1); private final String namePrefix; - DipperThreadFactory() { + public DipperThreadFactory(String poolNamePrefix) { SecurityManager s = System.getSecurityManager(); - group = (s != null) ? s.getThreadGroup() : - Thread.currentThread().getThreadGroup(); - namePrefix = "dipper-pool-" + - poolNumber.getAndIncrement() + - "-thread-"; + group = (s != null) ? s.getThreadGroup() : Thread.currentThread().getThreadGroup(); + namePrefix = poolNamePrefix + "-dipper-thread-"; } - public Thread newThread(Runnable r) { - Thread t = new Thread(group, r, - namePrefix + threadNumber.getAndIncrement(), - 0); - if (t.isDaemon()) + @Override public Thread newThread(Runnable runnable) { + Thread t = new Thread(group, runnable, namePrefix + threadNumber.getAndIncrement(), 0); + if (t.isDaemon()) { t.setDaemon(false); - if (t.getPriority() != Thread.NORM_PRIORITY) + } + if (t.getPriority() != Thread.NORM_PRIORITY) { t.setPriority(Thread.NORM_PRIORITY); + } return t; } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/AbstractMutilPipelineChainPipline.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/AbstractMutilPipelineChainPipline.java deleted file mode 100644 index 3969ebf9..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/AbstractMutilPipelineChainPipline.java +++ /dev/null @@ -1,225 +0,0 @@ -/* - * 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.topology; - -import java.util.ArrayList; -import java.util.HashMap; -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.batchsystem.BatchFinishMessage; -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.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -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.topology.model.AbstractStage; -import org.apache.rocketmq.streams.common.topology.model.IStageHandle; -import org.apache.rocketmq.streams.common.topology.model.Pipeline; -import org.apache.rocketmq.streams.common.topology.stages.JoinChainStage; -import org.apache.rocketmq.streams.common.topology.stages.WindowChainStage; -import org.apache.rocketmq.streams.common.utils.CollectionUtil; - -/** - * 新的解析已经废弃,主要兼容老的规则数据 - */ -@Deprecated -public abstract class AbstractMutilPipelineChainPipline extends ChainStage implements IAfterConfigurableRefreshListener { - /** - * pipeline name,这是一个汇聚节点,会有多个pipline,这里存的是pipline name - */ - protected List piplineNames = new ArrayList<>(); - //每个pipline,对应一个消息来源,在消息头上会有消息来源的name,根据name转发数据 - protected Map> piplineName2MsgSourceName; - - /** - * piplineNames的对象表示 - */ - protected transient Map> piplines = null; - - protected transient IStageHandle handle = new IStageHandle() { - @Override - protected IMessage doProcess(IMessage message, AbstractContext context) { - if (CollectionUtil.isEmpty(piplines)) { - return message; - } - String msgSourceName = message.getHeader().getMsgRouteFromLable(); - if (piplines.size() > 0) { - List messages = new ArrayList<>(); - Iterator>> it = piplineName2MsgSourceName.entrySet().iterator(); - while (it.hasNext()) { - Entry> entry = it.next(); - String piplineName = entry.getKey(); - Set values = entry.getValue(); - for(String value:values){ - if (msgSourceName != null && msgSourceName.equals(value)) {//如果来源数据的标签和map中的相同,转发这条消息给对应的pipline - ChainPipeline pipline = piplines.get(piplineName); - IMessage copyMessage = message.deepCopy(); - //copyMessage.getMessageBody().put(ORI_MESSAGE_KEY,message.getMessageBody()); - // 保留一份最原始的数据,后续对字段的修改不影响这个字段 - Context newContext = new Context(copyMessage); - copyMessage.getHeader().setMsgRouteFromLable(msgSourceName); - boolean needReturn = executePipline(pipline, copyMessage, newContext, msgSourceName); - if (needReturn) { - return message; - } - if (newContext.isContinue()) { - if (newContext.isSplitModel()) { - messages.addAll(newContext.getSplitMessages()); - } else { - messages.add(copyMessage); - } - - } - } - } - - } - for (IMessage msg : messages) { - msg.getHeader().setMsgRouteFromLable(msgSourceName); - } - doMessageAfterFinishPipline(message, context, messages); - return message; - } - ; - return message; - } - - @Override - public String getName() { - return AbstractMutilPipelineChainPipline.class.getName(); - } - }; - - /** - * 找到对应的pipline,并完成执行。如果只要一个pipline满足就可以,返回true,否则返回false - * - * @param copyMessage - * @param newContext - * @param msgSourceName - * @return 不继续处理其他pipline 返回true,否则返回false - */ - protected abstract boolean executePipline(ChainPipeline pipline, IMessage copyMessage, Context newContext, String msgSourceName); - - /** - * 如果所有的pipline处理完,还需要继续处理pipline产生的消息,则实现这个方法 - * - * @param message - * @param context - * @param messages - */ - protected abstract void doMessageAfterFinishPipline(IMessage message, AbstractContext context, List messages); - - @Override - public boolean isAsyncNode() { - for (Pipeline pipline : piplines.values()) { - if (pipline.isAsynNode() == false) { - return false; - } - } - return true; - } - - @Override - public void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage) { - sendSystem(message, context, piplines.values()); - } - - @Override - public void addNewSplit(IMessage message, AbstractContext context, NewSplitMessage newSplitMessage) { - sendSystem(message, context, piplines.values()); - } - - @Override - public void removeSplit(IMessage message, AbstractContext context, RemoveSplitMessage removeSplitMessage) { - sendSystem(message, context, piplines.values()); - } - - @Override - public void batchMessageFinish(IMessage message, AbstractContext context, BatchFinishMessage checkPointMessage) { - sendSystem(message, context, piplines.values()); - } - - public void addPipline(ChainPipeline pipline) { - this.piplineNames.add(pipline.getConfigureName()); - } - - @Override - protected IStageHandle selectHandle(T t, AbstractContext context) { - return handle; - } - - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - if (piplineNames == null) { - return; - } - Map> piplineMap = new HashMap<>(); - for (String pipeLineName : piplineNames) { - ChainPipeline chainPipline = configurableService.queryConfigurable(Pipeline.TYPE, pipeLineName); - if (chainPipline != null) { - piplineMap.put(chainPipline.getConfigureName(), chainPipline); - } - List> stages= chainPipline.getStages(); - for(AbstractStage stage:stages){ - stage.setPipeline(getPipeline()); - if(WindowChainStage.class.isInstance(stage)){ - ((WindowChainStage)stage).getWindow().setFireReceiver(getReceiverAfterCurrentNode()); - }else if(JoinChainStage.class.isInstance(stage)){ - ((JoinChainStage) stage).doProcessAfterRefreshConfigurable(configurableService); - ((JoinChainStage)stage).getWindow().setFireReceiver(getReceiverAfterCurrentNode()); - } - } - - } - this.piplines = piplineMap; - } - - public List getPiplineNames() { - return piplineNames; - } - - public void setPiplineNames(List piplineNames) { - this.piplineNames = piplineNames; - } - - public List getPiplines() { - List piplines = new ArrayList<>(); - piplines.addAll(this.piplines.values()); - return piplines; - } - - public Map> getPiplineName2MsgSourceName() { - return piplineName2MsgSourceName; - } - - public void setPiplineName2MsgSourceName( - Map> piplineName2MsgSourceName) { - this.piplineName2MsgSourceName = piplineName2MsgSourceName; - } - - public ChainPipeline getPipeline(String pipelineName){ - return this.piplines.get(pipelineName); - } - - -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ChainPipeline.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ChainPipeline.java deleted file mode 100644 index 96416044..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ChainPipeline.java +++ /dev/null @@ -1,594 +0,0 @@ -/* - * 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.topology; - -import com.alibaba.fastjson.JSONObject; -import com.google.common.collect.Lists; -import java.io.File; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.ServiceLoader; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.rocketmq.streams.common.cache.compress.impl.LongValueKV; -import org.apache.rocketmq.streams.common.channel.source.ISource; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configurable.AbstractConfigurable; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.context.AbstractContext; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; -import org.apache.rocketmq.streams.common.metadata.MetaData; -import org.apache.rocketmq.streams.common.monitor.ConsoleMonitorManager; -import org.apache.rocketmq.streams.common.monitor.IMonitor; -import org.apache.rocketmq.streams.common.monitor.group.MonitorCommander; -import org.apache.rocketmq.streams.common.optimization.IHomologousOptimization; -import org.apache.rocketmq.streams.common.optimization.MessageGlobleTrace; -import org.apache.rocketmq.streams.common.optimization.fingerprint.PreFingerprint; -import org.apache.rocketmq.streams.common.schedule.ScheduleManager; -import org.apache.rocketmq.streams.common.schedule.ScheduleTask; -import org.apache.rocketmq.streams.common.topology.metric.StageGroup; -import org.apache.rocketmq.streams.common.topology.model.AbstractStage; -import org.apache.rocketmq.streams.common.topology.model.Pipeline; -import org.apache.rocketmq.streams.common.utils.DipperThreadLocalUtil; -import org.apache.rocketmq.streams.common.utils.FileUtil; -import org.apache.rocketmq.streams.common.utils.MapKeyUtil; -import org.apache.rocketmq.streams.common.utils.PipelineHTMLUtil; -import org.apache.rocketmq.streams.common.utils.PrintUtil; -import org.apache.rocketmq.streams.common.utils.StringUtil; - -/** - * 数据流拓扑结构,包含了source 算子,sink - */ -public class ChainPipeline extends Pipeline implements IAfterConfigurableRefreshListener, Serializable,Runnable { - - private static final long serialVersionUID = -5189371682717444347L; - - private final transient int duplicateCacheSize = 1000000; - private transient LongValueKV duplicateCache; - private transient List duplicateFields; - private transient int duplicateCacheExpirationTime; - - private transient int homologousExpressionCacheSize = 2000000; - private transient int preFingerprintCacheSize = 2000000; - /** - * 对pipeline执行预编译的优化 - */ - private transient IHomologousOptimization homologousOptimization; - - protected transient ISource source; - - /** - * channel对应后续的stageName - */ - protected List channelNextStageLabel; - - protected transient Map> stageMap = new HashMap<>(); - - private String channelName; - - /** - * 数据源输入格式,主要用于日志指纹过滤,如果没有则不做优化 - */ - protected MetaData channelMetaData; - - /** - * 是否发布,默认为true,关闭发布时,此字段为false,pipeline启动时应判断此字段是否为true,status默认都为1,status为0代表pipeline已被删除 - */ - - protected transient AtomicBoolean hasStart = new AtomicBoolean(false); - - /** - * 为了图形化拓扑和监控使用 - */ - protected List stageGroups=new ArrayList<>(); - protected transient List rootStageGroups=new ArrayList<>(); - protected String createTableSQL; - - - /** - * 启动一个channel,并给channel应用pipeline - */ - - public void startChannel() { - //定时生成拓扑页面 - ScheduleManager.getInstance().regist(new ScheduleTask(0,10,this)); - final String monitorName = createPipelineMonitorName(); - if (isInitSuccess()) { - if (!hasStart.compareAndSet(false, true)) { - return; - } - final IStreamOperator receiver = this; - IMonitor pipelineMonitorForChannel = DipperThreadLocalUtil.get(); - if (pipelineMonitorForChannel == null) { - //主要监控channel的启动 - pipelineMonitorForChannel = IMonitor.createMonitor(this); - } - String isOpenOptimizationStr = ComponentCreator.getProperties().getProperty("homologous.optimization.switch"); - boolean isOpenOptimization = true; - if (StringUtil.isNotEmpty(isOpenOptimizationStr)) { - isOpenOptimization = Boolean.parseBoolean(isOpenOptimizationStr); - } - if (this.homologousOptimization == null && isOpenOptimization) { - Iterable iterable = ServiceLoader.load(IHomologousOptimization.class); - Iterator it = iterable.iterator(); - String homologousExpressionCacheSizeStr = ComponentCreator.getProperties().getProperty("homologous.expression.cache.size"); - if (StringUtil.isNotEmpty(homologousExpressionCacheSizeStr)) { - this.homologousExpressionCacheSize = Integer.parseInt(homologousExpressionCacheSizeStr); - } - String preFingerprintCacheSizeStr = ComponentCreator.getProperties().getProperty("homologous.pre.fingerprint.cache.size"); - if (StringUtil.isNotEmpty(preFingerprintCacheSizeStr)) { - this.preFingerprintCacheSize = Integer.parseInt(preFingerprintCacheSizeStr); - } - if (it.hasNext()) { - this.homologousOptimization = it.next(); - this.homologousOptimization.optimizate(Lists.newArrayList(this), this.homologousExpressionCacheSize, this.preFingerprintCacheSize); - } - } - - try { - AtomicLong COUNT = new AtomicLong(0); - long startTime = System.currentTimeMillis(); - boolean isPrintPipelineQPS = ComponentCreator.getPropertyBooleanValue("pipeline.qps.print"); - source.start((IStreamOperator) (message, context) -> { - //每条消息一个,监控整个链路 - IMonitor pipelineMonitorForStage = context.startMonitor(monitorName); - pipelineMonitorForStage.setType(IMonitor.TYPE_DATAPROCESS); - if (!message.getHeader().isSystemMessage()) { - //如果没有前置数据源则从消息里面取延迟 - //msg.put("__time__",message.getHeader().getEventMsgTime()); - ConsoleMonitorManager.getInstance().reportChannel(ChainPipeline.this, source, message); - } - message.getHeader().setPipelineName(this.getConfigureName()); - //在正式执行逻辑之前, 基于同源的优化策略先进行计算 - if (this.homologousOptimization != null) { - this.homologousOptimization.calculate(message, context); - } - //然后再执行正式逻辑,测试正式逻辑遇到表达是计算,会先从头部信息上去找,如果找到就直接返回,如果没有才进行正式的计算 - T t = receiver.doMessage(message, context); - pipelineMonitorForStage.endMonitor(); - if (isPrintPipelineQPS) { - long count = COUNT.incrementAndGet(); - long gap = (System.currentTimeMillis() - startTime) / 1000; - if (gap == 0) { - gap = 1; - } - if (count % 1000 == 0) { - double qps = (double) count / (double) gap; - System.out.println("qps is " + qps + ",the count is " + COUNT.get()); - } - } - - MonitorCommander.getInstance().finishMonitor(pipelineMonitorForStage.getName(), pipelineMonitorForStage); - return t; - }); - } catch (Exception e) { - e.printStackTrace(); - //已经输出到sime的日志文件不需要再输出dipper.log - this.setInitSuccess(false); - //pipeline启动失败日志输出 - pipelineMonitorForChannel.occureError(e, pipelineMonitorForChannel.getName() + " pipeline startup error", e.getMessage()); - } - } else { - LOG.error("channel init failure, so can not start channel"); - } - } - - private String createDuplicateKey(IMessage message) { - List duplicateValues = Lists.newArrayList(); - for (String field : duplicateFields) { - duplicateValues.add(message.getMessageBody().getString(field)); - } - return StringUtil.createMD5Str(String.join("", duplicateValues)); - } - - private String createPipelineMonitorName() { - return MapKeyUtil.createKeyBySign(".", getType(), getNameSpace(), getConfigureName()); - } - - /** - * 可以替换某个阶段的阶段,而不用配置的阶段 - * - * @param t 数据 - * @param context 上下文 - * @param replaceStage stage - * @return stage执行后的结果 - */ - @Override - protected T doMessageInner(T t, AbstractContext context, AbstractStage... replaceStage) { - if (this.duplicateCache != null && this.duplicateFields != null && !this.duplicateFields.isEmpty() && !t.getHeader().isSystemMessage()) { - String duplicateKey = createDuplicateKey(t); - Long cacheTime = this.duplicateCache.get(duplicateKey); - Long currentTime = System.currentTimeMillis(); - if (cacheTime != null && currentTime - cacheTime < this.duplicateCacheExpirationTime) { - context.breakExecute(); - return t; - } else { - this.duplicateCache.put(duplicateKey, currentTime); - if (this.duplicateCache.getSize() > duplicateCacheSize) { - this.duplicateCache = new LongValueKV(this.duplicateCacheSize); - } - } - } - if (!t.getHeader().isSystemMessage()) { - MessageGlobleTrace.joinMessage(t);//关联全局监控器 - } - - if (!isTopology()) { - return super.doMessageInner(t, context, replaceStage); - } - context.setMessage(t); - doNextStages(context, getMsgSourceName(), channelName, this.channelNextStageLabel, null); - return t; - } - - protected boolean isTopology(List nextStageLabel) { - return nextStageLabel != null && nextStageLabel.size() != 0; - } - - public boolean isTopology() { - return isTopology(this.channelNextStageLabel); - } - - public void doNextStages(AbstractContext context, String msgPrevSourceName, String currentLabel, List nextStageLabel, String prevSQLNodeName) { - if (!isTopology(nextStageLabel)) { - return; - } - String oriMsgPrewSourceName = msgPrevSourceName; - int size = nextStageLabel.size(); - for (String lable : nextStageLabel) { - AbstractContext copyContext = context; - if (size > 1) { - copyContext = context.copy(); - } - T msg = (T) copyContext.getMessage(); - AbstractStage oriStage = stageMap.get(lable); - if (oriStage == null) { - if (stages != null && stages.size() > 0) { - synchronized (this) { - oriStage = stageMap.get(lable); - if (oriStage == null) { - createStageMap(); - oriStage = stageMap.get(lable); - } - } - } - if (oriStage == null) { - LOG.warn("expect stage named " + lable + ", but the stage is not exist"); - continue; - } - } - AbstractStage stage = oriStage; - if (filterByPreFingerprint(msg, copyContext, currentLabel, lable)) { - continue; - } - - if (!msg.getHeader().isSystemMessage()) { - ConsoleMonitorManager.getInstance().reportInput(stage, msg); - } - - //boolean needFlush = needFlush(msg); - if (StringUtil.isNotEmpty(oriMsgPrewSourceName)) { - msg.getHeader().setMsgRouteFromLable(oriMsgPrewSourceName); - } - boolean isContinue = executeStage(stage, msg, copyContext); - - if (!isContinue) { - //只要执行到了window分支都不应该被过滤 - if (stage.isAsyncNode() && !msg.getHeader().isSystemMessage()) { - MessageGlobleTrace.finishPipeline(msg); - } - } else { - if (!msg.getHeader().isSystemMessage()) { - ConsoleMonitorManager.getInstance().reportOutput(stage, msg, ConsoleMonitorManager.MSG_FLOWED, null); - } - if (stage instanceof ChainStage) { - ChainStage chainStage = (ChainStage) stage; - String msgSourceName = chainStage.getMsgSourceName(); - if (StringUtil.isNotEmpty(msgSourceName)) { - msgPrevSourceName = msgSourceName; - } - } - - if (copyContext.isSplitModel()) { - List messageList = copyContext.getSplitMessages(); - int splitMessageOffset = 0; - for (IMessage message : messageList) { - AbstractContext abstractContext = copyContext.copy(); - abstractContext.closeSplitMode(message); - message.getHeader().setMsgRouteFromLable(msg.getHeader().getMsgRouteFromLable()); - message.getHeader().addLayerOffset(splitMessageOffset); - splitMessageOffset++; - List labels = stage.doRoute(message); - if (labels == null || labels.size() == 0) { - if (!message.getHeader().isSystemMessage()) { - MessageGlobleTrace.finishPipeline(message); - } - continue; - } - doNextStages(abstractContext, msgPrevSourceName, stage.getLabel(), labels, stage.getOwnerSqlNodeTableName()); - } - } else { - List labels = stage.doRoute(msg); - if (labels == null || labels.size() == 0) { - if (!msg.getHeader().isSystemMessage()) { - MessageGlobleTrace.finishPipeline(msg); - } - continue; - } - doNextStages(copyContext, msgPrevSourceName, stage.getLabel(), labels, stage.getOwnerSqlNodeTableName()); - } - } - } - } - - /** - * 是否进入一个新的sql node - * - * @param stage - * @param prewSQLNodeName - * @return - */ - protected boolean isNewSQLNode(AbstractStage stage, String prewSQLNodeName) { - if (prewSQLNodeName == null) { - return true; - } - if (stage.getOwnerSqlNodeTableName().equals(prewSQLNodeName)) { - return false; - } - return true; - } - - @Override - protected boolean executeStage(final AbstractStage stage, T t, AbstractContext context) { - IMonitor stageMonitor = context.createChildrenMonitor(createPipelineMonitorName(), stage); - - try { - boolean isContinue = super.executeStage(stage, t, context); - stageMonitor.setResult(isContinue); - stageMonitor.endMonitor(); - if (stageMonitor.isSlow()) { - stageMonitor.setSampleData(context).put("stage_info", createStageInfo(stage)); - } - return isContinue; - } catch (Exception e) { - e.printStackTrace(); - //优化日志量 - //LOG.error("execute stage error " + stage.getConfigureName(), e); - stageMonitor.occureError(e, "execute stage error " + stage.getConfigureName(), e.getMessage()); - stageMonitor.setSampleData(context).put("stage_info", createStageInfo(stage)); - return false; - } - } - - protected boolean filterByPreFingerprint(IMessage t, AbstractContext context, String sourceName, String nextLable) { - PreFingerprint preFingerprint = getPreFingerprint(sourceName, nextLable); - if (preFingerprint != null) { - boolean isFilter = preFingerprint.filterByLogFingerprint(t); - if (isFilter) { - context.breakExecute(); - return true; - } - } - return false; - } - - protected JSONObject createStageInfo(AbstractStage stage) { - JSONObject jsonObject = null; - // if (stage instanceof ChainStage) { - // jsonObject = new JSONObject(); - // ChainStage chainStage = (ChainStage) stage; - // return chainStage.toJsonObject(); - // //String entityName = chainStage.getEntityName(); - // ////todo 需要改写 - // //if (creatorService != null && StringUtil.isNotEmpty(entityName)) { - // // IConfigurableCreator creator = creatorService.getCreator( - // // entityName); - // // if(creator!=null){ - // // String configures = creator.print(stage); - // // jsonObject.put("stageDetail", configures); - // // } - // // - // //} - // } - return jsonObject; - } - - public ChainPipeline addChainStage(ChainStage chainStage) { - addStage(chainStage); - return this; - } - - public ISource getSource() { - return source; - } - - public void setSource(ISource source) { - this.source = source; - if (getNameSpace() == null) { - setNameSpace(source.getNameSpace()); - } - channelName = source.getConfigureName(); - } - - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - createStageMap(); - Map stageGroupMap=createStageGroupMap(); - for(StageGroup stageGroup:this.stageGroups){ - stageGroup.init(this.stageMap,stageGroupMap); - if(stageGroup.getParent()==null&&!this.rootStageGroups.contains(stageGroup)){ - this.rootStageGroups.add(stageGroup); - } - } - - ISource source = configurableService.queryConfigurable(ISource.TYPE, channelName); - this.source = source; - for (AbstractStage stage : getStages()) { - stage.setPipeline(this); - if (stage instanceof IAfterConfigurableRefreshListener) { - if (!stage.isInitSuccess() && !this.isInitSuccess()) { - this.setInitSuccess(false); - return; - } - IAfterConfigurableRefreshListener afterConfigurableRefreshListener = (IAfterConfigurableRefreshListener) stage; - afterConfigurableRefreshListener.doProcessAfterRefreshConfigurable(configurableService); - } - } - - if (source instanceof AbstractConfigurable) { - AbstractConfigurable abstractConfigurable = (AbstractConfigurable) source; - if (!abstractConfigurable.isInitSuccess() && this.isInitSuccess()) { - this.setInitSuccess(false); - return; - } - } - - //增加去重的逻辑 - String duplicateFieldNameStr = ComponentCreator.getProperties().getProperty(getConfigureName() + ".duplicate.fields.names"); - if (duplicateFieldNameStr != null && !duplicateFieldNameStr.isEmpty()) { - this.duplicateFields = Lists.newArrayList(); - this.duplicateFields.addAll(Arrays.asList(duplicateFieldNameStr.split(";"))); - } - if (this.duplicateCache == null && this.duplicateFields != null) { - this.duplicateCache = new LongValueKV(this.duplicateCacheSize); - } - String duplicateCacheExpirationStr = ComponentCreator.getProperties().getProperty(getConfigureName() + ".duplicate.expiration.time"); - if (duplicateCacheExpirationStr != null && !duplicateCacheExpirationStr.isEmpty()) { - this.duplicateCacheExpirationTime = Integer.parseInt(duplicateCacheExpirationStr); - } else { - this.duplicateCacheExpirationTime = 86400000; - } - - } - - private Map createStageGroupMap() { - Map map=new HashMap<>(); - for(StageGroup stageGroup:stageGroups){ - map.put(stageGroup.getConfigureName(),stageGroup); - } - return map; - } - - public Map> createStageMap() { - for (AbstractStage stage : getStages()) { - stageMap.put(stage.getLabel(), stage); - stage.setPipeline(this); - } - return stageMap; - } - - public List getChannelNextStageLabel() { - return channelNextStageLabel; - } - - public void setChannelNextStageLabel(List channelNextStageLabel) { - this.channelNextStageLabel = channelNextStageLabel; - } - - @Override - public String toString() { - String LINE = PrintUtil.LINE; - StringBuilder sb = new StringBuilder(); - sb.append("###namespace=").append(getNameSpace()).append("###").append(LINE); - if (source != null) { - sb.append(source.toString()).append(LINE); - } - if (stages != null) { - for (AbstractStage stage : stages) { - sb.append(stage.toString()); - } - - } - return sb.toString(); - } - - @Override - public void destroy() { - if (source != null && hasStart.compareAndSet(true, false)) { - source.destroy(); - } - super.destroy(); - } - public void addStageGroup(StageGroup stageGroup){ - if(this.stageGroups!=null){ - this.stageGroups.add(stageGroup); - } - } - public Map> getStageMap() { - return stageMap; - } - - public Boolean getHasStart() { - return hasStart.get(); - } - - public MetaData getChannelMetaData() { - return channelMetaData; - } - - public void setChannelMetaData(MetaData channelMetaData) { - this.channelMetaData = channelMetaData; - } - - public String getChannelName() { - return channelName; - } - - public void setChannelName(String channelName) { - this.channelName = channelName; - } - - public List getStageGroups() { - return stageGroups; - } - - public void setStageGroups(List stageGroups) { - this.stageGroups = stageGroups; - } - - public List getRootStageGroups() { - return rootStageGroups; - } - - public String getCreateTableSQL() { - return createTableSQL; - } - - public void setCreateTableSQL(String createTableSQL) { - this.createTableSQL = createTableSQL; - } - - @Override public void run() { - String filePath=FileUtil.getJarPath(); - if(StringUtil.isEmpty(filePath)){ - filePath="/tmp"; - } - filePath=filePath+ File.separator+getConfigureName()+".html"; - String html=PipelineHTMLUtil.createHTML(this); - synchronized (this){ - FileUtil.write(filePath,html); - LOG.info(getConfigureName()+" create pipeline html success in "+filePath); - } - } -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/IJobGraph.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/IJobGraph.java new file mode 100644 index 00000000..bc605ff6 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/IJobGraph.java @@ -0,0 +1,13 @@ +package org.apache.rocketmq.streams.common.topology; + +import com.alibaba.fastjson.JSONObject; +import java.util.List; + +public interface IJobGraph { + + void start(); + + void stop(); + + List execute(List dataList); +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/ILifeCycle.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ILifeCycle.java similarity index 95% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/ILifeCycle.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ILifeCycle.java index 29e1cecb..f7f1ee42 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/ILifeCycle.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ILifeCycle.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.common.topology.model; +package org.apache.rocketmq.streams.common.topology; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/IReducer.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/IReducer.java similarity index 94% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/IReducer.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/IReducer.java index 074eec1d..aebf7d18 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/IReducer.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/IReducer.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.common.topology.stages.udf; +package org.apache.rocketmq.streams.common.topology; import com.alibaba.fastjson.JSONObject; import java.io.Serializable; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/shuffle/IShuffleKeyGenerator.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/IShuffleKeyGenerator.java similarity index 94% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/shuffle/IShuffleKeyGenerator.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/IShuffleKeyGenerator.java index d5b613c0..74ac221c 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/shuffle/IShuffleKeyGenerator.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/IShuffleKeyGenerator.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.common.topology.shuffle; +package org.apache.rocketmq.streams.common.topology; import org.apache.rocketmq.streams.common.context.IMessage; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/builder/IStageBuilder.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/IStageBuilder.java similarity index 90% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/builder/IStageBuilder.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/IStageBuilder.java index 0afd20f5..d796f24b 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/builder/IStageBuilder.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/IStageBuilder.java @@ -14,7 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.common.topology.builder; +package org.apache.rocketmq.streams.common.topology; + +import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; /** * 通过它构建pipeline中的chain stage。有各个子类实现 diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ISynchronousWindow.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ISynchronousWindow.java new file mode 100644 index 00000000..7d56157c --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ISynchronousWindow.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.topology; + +import org.apache.rocketmq.streams.common.context.AbstractContext; +import org.apache.rocketmq.streams.common.context.IMessage; + +public interface ISynchronousWindow { + + void accumulateDirectly(IMessage message, AbstractContext context); +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/IWindow.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/IWindow.java similarity index 53% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/IWindow.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/IWindow.java index b04d894d..35fc5d1a 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/IWindow.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/IWindow.java @@ -14,22 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.common.topology.model; +package org.apache.rocketmq.streams.common.topology; -import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; -import org.apache.rocketmq.streams.common.channel.sink.ISink; import org.apache.rocketmq.streams.common.configurable.IConfigurable; -import org.apache.rocketmq.streams.common.context.AbstractContext; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; -import org.apache.rocketmq.streams.common.topology.SectionPipeline; -import org.apache.rocketmq.streams.common.topology.shuffle.IShuffleKeyGenerator; /** * Window Definition 处理数据并输出 */ public interface IWindow - extends IStreamOperator>, IConfigurable, IShuffleKeyGenerator { + extends IConfigurable, IShuffleKeyGenerator { int DEFAULTFIRE_MODE = 0;// fire time=endtime+watermark int MULTI_WINDOW_INSTANCE_MODE = 1;// fire at window size interval, until event time >endtime+watermark, every window result is independent @@ -39,29 +32,6 @@ public interface IWindow * split char between function */ String SCRIPT_SPLIT_CHAR = ";"; - /** - * the default window size, 10 minutes - */ - Integer DEFAULT_WINDOW_SIZE = 10; - - /** - * the default window slide, 1 minute - */ - Integer DEFAULT_WINDOW_SLIDE = 1; - - Integer DEFAULT_WINDOW_SESSION_TIMEOUT = 10; - - // protected transient ExecutorService deleteService = Executors.newSingleThreadExecutor(); - - /** - * window start key used by TumbleFunction and HopFunction - */ - String WINDOW_START = "WINDOW_START_TIME"; - - /** - * window end key used by TumbleFunction and HopFunction - */ - String WINDOW_END = "WINDOW_END"; /** * tumble window type @@ -81,19 +51,17 @@ public interface IWindow String TYPE = "window"; - int FIRE_DELAY_SECOND = 1;//触发延迟 - /** * the delay time of system (ms) */ Integer SYS_DELAY_TIME = 3000; - /** - * 窗口触发后,需要执行的逻辑 - * - * @param receiver - */ - void setFireReceiver(SectionPipeline receiver); +// /** +// * 窗口触发后,需要执行的逻辑 +// * +// * @param receiver +// */ +// void setFireReceiver(SectionPipeline receiver); /** * 是否是同步操作,目前只有over 特殊场景会是true @@ -102,11 +70,13 @@ public interface IWindow */ boolean isSynchronous(); - IWindowCheckpoint getWindowCache(); - - void windowInit(); + void start(); - interface IWindowCheckpoint extends ISink { - void finishBatchMsg(BatchFinishMessage batchFinishMessage); - } +// IWindowCheckpoint getWindowCache(); +// +// void windowInit(); +// +// interface IWindowCheckpoint extends ISink { +// void finishBatchMsg(BatchFinishMessage batchFinishMessage); +// } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/task/TaskAssigner.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/assigner/TaskAssigner.java similarity index 78% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/task/TaskAssigner.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/assigner/TaskAssigner.java index 2026d3d5..3ab754a2 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/task/TaskAssigner.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/assigner/TaskAssigner.java @@ -14,17 +14,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.common.topology.task; +package org.apache.rocketmq.streams.common.topology.assigner; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; /** - * 帮助某个pipline 关联对应的数据源pipline。 主要用于多个同源任务自动装配的场景 namespace:source pipline的name space name:piplineName + * 帮助某个pipeline 关联对应的数据源pipeline。 主要用于多个同源任务自动装配的场景 namespace:source pipeline的name space name:pipelineName */ public class TaskAssigner extends BasedConfigurable { public static String TYPE = "assigner"; protected String taskName;//数据源对应的pipline name protected String pipelineName;//需要关联的pipline name + protected Integer weight = 1; public TaskAssigner() { setType(TYPE); @@ -45,4 +46,12 @@ public String getPipelineName() { public void setPipelineName(String pipelineName) { this.pipelineName = pipelineName; } + + public Integer getWeight() { + return weight; + } + + public void setWeight(Integer weight) { + this.weight = weight; + } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/builder/PipelineBuilder.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/builder/PipelineBuilder.java index b1e85672..70604025 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/builder/PipelineBuilder.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/builder/PipelineBuilder.java @@ -22,22 +22,21 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.rocketmq.streams.common.channel.impl.view.ViewSink; +import java.util.Properties; import org.apache.rocketmq.streams.common.channel.sink.ISink; import org.apache.rocketmq.streams.common.channel.source.ISource; -import org.apache.rocketmq.streams.common.configurable.AbstractConfigurable; import org.apache.rocketmq.streams.common.configurable.IConfigurable; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; import org.apache.rocketmq.streams.common.metadata.MetaData; -import org.apache.rocketmq.streams.common.model.NameCreator; import org.apache.rocketmq.streams.common.model.NameCreatorContext; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; -import org.apache.rocketmq.streams.common.topology.ChainStage; +import org.apache.rocketmq.streams.common.topology.IStageBuilder; import org.apache.rocketmq.streams.common.topology.metric.StageGroup; +import org.apache.rocketmq.streams.common.topology.model.AbstractChainStage; import org.apache.rocketmq.streams.common.topology.model.AbstractStage; -import org.apache.rocketmq.streams.common.topology.model.Pipeline; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; import org.apache.rocketmq.streams.common.topology.stages.OutputChainStage; -import org.apache.rocketmq.streams.common.topology.stages.ViewChainStage; +import org.apache.rocketmq.streams.common.utils.ENVUtil; +import org.apache.rocketmq.streams.common.utils.ReflectUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; public class PipelineBuilder implements Serializable { @@ -63,6 +62,11 @@ public class PipelineBuilder implements Serializable { */ protected String pipelineName; + /** + * 配置信息 + */ + protected Properties configuration; + /** * 数据源的格式,非必须 */ @@ -72,7 +76,7 @@ public class PipelineBuilder implements Serializable { /** * 如果需要制作拓扑结构,则保存当前构建的stage */ - protected ChainStage currentChainStage; + protected AbstractChainStage currentChainStage; /** * 在sql tree中,存储当前节点父节点的table name,主要用于双流join场景,用于判断是否是右流join @@ -85,8 +89,10 @@ public class PipelineBuilder implements Serializable { protected boolean isRightJoin = false; - - protected String rootTableName;//SQL Tree保存的是root tablename + /** + * SQL Tree保存的是root table name + */ + protected String rootTableName; /** * 主要用于把sql 按create view聚合,然后分层展开,便于监控和排错,主要用于sql解析时,完成stage的分组和层次关系 @@ -95,11 +101,12 @@ public class PipelineBuilder implements Serializable { protected StageGroup parentStageGroup; - public PipelineBuilder(String namespace, String pipelineName) { - pipeline.setNameSpace(namespace); - pipeline.setConfigureName(pipelineName); + public PipelineBuilder(String namespace, String pipelineName, Properties properties) { this.pipelineNameSpace = namespace; this.pipelineName = pipelineName; + this.configuration = properties; + pipeline.setNameSpace(namespace); + pipeline.setName(pipelineName); addConfigurables(pipeline); } @@ -116,13 +123,11 @@ public void setSource(ISource source) { /** * 创建chain stage - * - * @param stageBuilder - * @return */ - public ChainStage createStage(IStageBuilder stageBuilder) { - ChainStage chainStage = stageBuilder.createStageChain(this); - stageBuilder.addConfigurables(this);// 这句一定要在addChainStage前,会默认赋值namespace和name + public AbstractChainStage createStage(IStageBuilder> stageBuilder) { + AbstractChainStage chainStage = stageBuilder.createStageChain(this); + // 这句一定要在addChainStage前,会默认赋值namespace和name + stageBuilder.addConfigurables(this); if (StringUtil.isEmpty(chainStage.getLabel())) { chainStage.setLabel(createConfigurableName(chainStage.getType())); } @@ -130,44 +135,90 @@ public ChainStage createStage(IStageBuilder stageBuilder) { return chainStage; } - public List createSQL() { - List sqls = new ArrayList<>(); - for (IConfigurable configurable : configurables) { - sqls.add(AbstractConfigurable.createSQL(configurable)); + public ChainPipeline build() { + pipeline.setChannelMetaData(channelMetaData); + if (configurables != null) { + for (IConfigurable configurable : configurables) { + configurable.setJobName(pipelineName); + configurable.setConfiguration(this.configuration); + initAndENVReplace(configurable, this.configuration); + } + } - return sqls; + return pipeline; } - public ChainPipeline build(IConfigurableService configurableService) { - List configurableList = configurables; - pipeline.setChannelMetaData(channelMetaData); - if (configurableList != null) { - for (IConfigurable configurable : configurableList) { - configurableService.insert(configurable); + protected void initAndENVReplace(IConfigurable configurable, Properties configuration) { + try { + doEVNReplace(configurable, configuration); + } catch (Exception e) { + throw new RuntimeException("the configurable do evn replace error " + configurable.getClass().getName()); + } + try { + configurable.init(); + } catch (Exception e) { + throw new RuntimeException("the configurable do init error " + configurable.getClass().getName()); + } + + } + + protected void doEVNReplace(IConfigurable configurable, Properties configuration) { + ReflectUtil.scanConfiguableFields(configurable, (o, field) -> { + ENVDependence dependence = field.getAnnotation(ENVDependence.class); + if (dependence == null) { + return; + } + try { + field.setAccessible(true); + if (!field.getType().getName().equals(String.class.getName())) { + return; + } + String fieldValue = ReflectUtil.getBeanFieldValue(o, field.getName()); + String envValue = getENVVar(fieldValue, configuration); + if (StringUtil.isEmpty(envValue)) { + return; + } + ReflectUtil.setBeanFieldValue(o, field.getName(), envValue); + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); } + + }); + } + + protected String getENVVar(String fieldValue, Properties configuration) { + if (StringUtil.isEmpty(fieldValue)) { + return null; + } + String value = configuration.getProperty(fieldValue); + if (StringUtil.isNotEmpty(value)) { + return value; } - configurableService.refreshConfigurable(pipelineNameSpace); - return configurableService.queryConfigurable(Pipeline.TYPE, pipelineName); + return ENVUtil.getENVParameter(fieldValue); } public List getAllConfigurables() { - List configurableList = configurables; - pipeline.setChannelMetaData(channelMetaData); - return configurableList; + return configurables; + } + + public void addChainStage(AbstractChainStage chainStage) { + if (StringUtil.isEmpty(chainStage.getName())) { + chainStage.setName(createConfigurableName(chainStage.getType())); + chainStage.setLabel(chainStage.getName()); + } + pipeline.addChainStage(chainStage); } /** * 创建chain stage - * - * @param sink - * @return */ - public ChainStage createStage(ISink sink) { + public AbstractChainStage createStage(ISink sink) { OutputChainStage outputChainStage = new OutputChainStage(); sink.addConfigurables(this); outputChainStage.setSink(sink); - if (StringUtil.isEmpty(sink.getConfigureName())) { - sink.setConfigureName(createConfigurableName(sink.getType())); + if (StringUtil.isEmpty(sink.getName())) { + sink.setName(createConfigurableName(sink.getType())); } pipeline.addChainStage(outputChainStage); return outputChainStage; @@ -175,16 +226,9 @@ public ChainStage createStage(ISink sink) { /** * 增加输出 - * - * @param sink - * @return */ public OutputChainStage addOutput(ISink sink) { - OutputChainStage outputChainStage = new OutputChainStage<>();; - if(ViewSink.class.isInstance(sink)){ - - outputChainStage=new ViewChainStage(); - } + OutputChainStage outputChainStage = new OutputChainStage<>(); sink.addConfigurables(this); outputChainStage.setSink(sink); if (StringUtil.isEmpty(outputChainStage.getLabel())) { @@ -196,8 +240,6 @@ public OutputChainStage addOutput(ISink sink) { /** * 增加维表 - * - * @param configurable */ public void addNameList(IConfigurable configurable) { addConfigurables(configurable); @@ -205,21 +247,16 @@ public void addNameList(IConfigurable configurable) { /** * 增加中间chain stage - * - * @param stageChainBuilder */ - public ChainStage addChainStage(IStageBuilder stageChainBuilder) { - return createStage(stageChainBuilder); + public AbstractChainStage addChainStage(IStageBuilder> stageChainBuilder) { + return createStage(stageChainBuilder); } /** * 自动创建组建名称 - * - * @param type - * @return */ public String createConfigurableName(String type) { - return NameCreatorContext.get().createNewName(this.pipelineName, type); + return NameCreatorContext.get().createName(this.pipelineName, type); } /** @@ -230,12 +267,12 @@ public void addConfigurables(IConfigurable configurable) { if (StringUtil.isEmpty(configurable.getNameSpace())) { configurable.setNameSpace(getPipelineNameSpace()); } - if (StringUtil.isEmpty(configurable.getConfigureName())) { - configurable.setConfigureName(createConfigurableName(configurable.getType())); + if (StringUtil.isEmpty(configurable.getName())) { + configurable.setName(createConfigurableName(configurable.getType())); } //判断配置信息是否已经存在,如果不存在,则添加 for (IConfigurable config : this.configurables) { - if (config.getType().equals(configurable.getType()) && config.getConfigureName().equals(configurable.getConfigureName())) { + if (config.getType().equals(configurable.getType()) && config.getName().equals(configurable.getName())) { return; } } @@ -254,15 +291,13 @@ public void addConfigurables(Collection configurables) /** * 在当前拓扑基础上,增加下一层级的拓扑。如果需要做拓扑,需要设置标签 - * - * @param nextStages */ - public void setTopologyStages(ChainStage currentChainStage, List nextStages) { + public void setTopologyStages(AbstractChainStage currentChainStage, List> nextStages) { if (nextStages == null) { return; } List labelNames = new ArrayList<>(); - for (ChainStage stage : nextStages) { + for (AbstractChainStage stage : nextStages) { labelNames.add(stage.getLabel()); } @@ -270,7 +305,7 @@ public void setTopologyStages(ChainStage currentChainStage, List this.pipeline.setChannelNextStageLabel(labelNames); } else { currentChainStage.getNextStageLabels().addAll(labelNames); - for (ChainStage stage : nextStages) { + for (AbstractChainStage stage : nextStages) { stage.getPrevStageLabels().add(currentChainStage.getLabel()); } } @@ -278,11 +313,9 @@ public void setTopologyStages(ChainStage currentChainStage, List /** * 拓扑的特殊形式,下层只有单个节点 - * - * @param nextStage */ - public void setTopologyStages(ChainStage currentChainStage, ChainStage nextStage) { - List stages = new ArrayList<>(); + public void setTopologyStages(AbstractChainStage currentChainStage, AbstractChainStage nextStage) { + List> stages = new ArrayList<>(); stages.add(nextStage); setTopologyStages(currentChainStage, stages); } @@ -299,56 +332,51 @@ public ChainPipeline getPipeline() { return pipeline; } - public void setHorizontalStages(ChainStage stage) { -// if (isBreak) { -// return; -// } - List> stages = new ArrayList<>(); + public void setHorizontalStages(AbstractChainStage stage) { + List> stages = new ArrayList<>(); stages.add(stage); setHorizontalStages(stages); } /** * 如果需要做拓扑,需要设置标签 - * - * @param stages */ - public void setHorizontalStages(List> stages) { + public void setHorizontalStages(List> stages) { if (stages == null) { return; } - List lableNames = new ArrayList<>(); - Map> lableName2Stage = new HashMap(); - for (ChainStage stage : stages) { - if(stage==null){ + List labelNames = new ArrayList<>(); + Map> labelName2Stage = new HashMap(); + for (AbstractChainStage stage : stages) { + if (stage == null) { continue; } - lableNames.add(stage.getLabel()); - lableName2Stage.put(stage.getLabel(), stage); + labelNames.add(stage.getLabel()); + labelName2Stage.put(stage.getLabel(), stage); } if (currentChainStage == null) { - this.pipeline.setChannelNextStageLabel(lableNames); - for (String lableName : lableNames) { - ChainStage chainStage = lableName2Stage.get(lableName); - if(this.pipeline.getChannelName()!=null){ + this.pipeline.setChannelNextStageLabel(labelNames); + for (String lableName : labelNames) { + AbstractChainStage chainStage = labelName2Stage.get(lableName); + if (this.pipeline.getChannelName() != null) { chainStage.getPrevStageLabels().add(this.pipeline.getChannelName()); } } } else { - if(currentChainStage.getNextStageLabels()==null){ + if (currentChainStage.getNextStageLabels() == null) { currentChainStage.setNextStageLabels(new ArrayList<>()); } - for (String lableName : lableNames) { - if(StringUtil.isEmpty(lableName)){ + for (String labelName : labelNames) { + if (StringUtil.isEmpty(labelName)) { continue; } - if(!currentChainStage.getNextStageLabels().contains(lableName)){ - currentChainStage.getNextStageLabels().add(lableName); + if (!currentChainStage.getNextStageLabels().contains(labelName)) { + currentChainStage.getNextStageLabels().add(labelName); } - ChainStage chainStage = lableName2Stage.get(lableName); - List prewLables = chainStage.getPrevStageLabels(); - if (!prewLables.contains(this.currentChainStage.getLabel())) { + AbstractChainStage chainStage = labelName2Stage.get(labelName); + List prevLabels = chainStage.getPrevStageLabels(); + if (!prevLabels.contains(this.currentChainStage.getLabel())) { chainStage.getPrevStageLabels().add(this.currentChainStage.getLabel()); } @@ -356,26 +384,27 @@ public void setHorizontalStages(List> stages) { } } - public List> getFirstStages(){ - Map> stageMap= pipeline.createStageMap(); - List> stages=new ArrayList<>(); - List firstLables=pipeline.getChannelNextStageLabel(); - if(firstLables==null){ + public List> getFirstStages() { + Map> stageMap = pipeline.createStageMap(); + List> stages = new ArrayList<>(); + List labels = pipeline.getChannelNextStageLabel(); + if (labels == null) { return null; } - for(String lableName:firstLables){ - stages.add((ChainStage) stageMap.get(lableName)); + for (String labelName : labels) { + stages.add((AbstractChainStage) stageMap.get(labelName)); } return stages; } - public void setCurrentChainStage(ChainStage currentChainStage) { - this.currentChainStage = currentChainStage; - } - public ChainStage getCurrentChainStage() { + public AbstractChainStage getCurrentChainStage() { return currentChainStage; } + public void setCurrentChainStage(AbstractChainStage currentChainStage) { + this.currentChainStage = currentChainStage; + } + public List getConfigurables() { return configurables; } @@ -427,4 +456,12 @@ public StageGroup getParentStageGroup() { public void setParentStageGroup(StageGroup parentStageGroup) { this.parentStageGroup = parentStageGroup; } + + public Properties getConfiguration() { + return configuration; + } + + public void setConfiguration(Properties configuration) { + this.configuration = configuration; + } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/graph/JobGraph.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/graph/JobGraph.java new file mode 100644 index 00000000..46784bd2 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/graph/JobGraph.java @@ -0,0 +1,95 @@ +/* + * 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.topology.graph; + +import com.alibaba.fastjson.JSONObject; +import java.util.List; +import java.util.Properties; +import org.apache.rocketmq.streams.common.configuration.SystemContext; +import org.apache.rocketmq.streams.common.topology.IJobGraph; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; + +public class JobGraph implements IJobGraph { + + private String namespace; + private String jobName; + private List> pipelines; + private Properties jobConfiguration; + + public JobGraph(String namespace, String jobName, List> pipelines) { + this.namespace = namespace; + this.jobName = jobName; + this.pipelines = pipelines; + this.jobConfiguration = SystemContext.getProperties(); + } + + public JobGraph(String namespace, String jobName, List> pipelines, Properties jobConfiguration) { + this.namespace = namespace; + this.jobName = jobName; + this.pipelines = pipelines; + this.jobConfiguration = jobConfiguration; + } + + @Override public void start() { + for (ChainPipeline pipeline : pipelines) { + pipeline.startJob(); + } + } + + @Override public void stop() { + for (ChainPipeline pipeline : pipelines) { + pipeline.destroy(); + } + } + + @Override public List execute(List dataList) { + return pipelines.get(0).executeWithMsgs(dataList); + } + + public String getNamespace() { + return namespace; + } + + public void setNamespace(String namespace) { + this.namespace = namespace; + } + + public String getJobName() { + return jobName; + } + + public void setJobName(String jobName) { + this.jobName = jobName; + } + + public List> getPipelines() { + return pipelines; + } + + public void setPipelines(List> pipelines) { + this.pipelines = pipelines; + } + + public Properties getJobConfiguration() { + return jobConfiguration; + } + + public void setJobConfiguration(Properties jobConfiguration) { + this.jobConfiguration = jobConfiguration; + } + +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/NotFireReason.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/NotFireReason.java index cc7db515..6250eb37 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/NotFireReason.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/NotFireReason.java @@ -22,70 +22,77 @@ import java.util.List; import java.util.Map; import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.optimization.IHomologousOptimization; import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintCache; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; import org.apache.rocketmq.streams.common.utils.CollectionUtil; import org.apache.rocketmq.streams.common.utils.JsonableUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; public class NotFireReason { - protected List oriFilterFieldNames=new ArrayList<>();//参与过滤的字段名 - protected Map oriFilterFields=new HashMap<>();//原始 - - - protected List expressions=new ArrayList<>();//过滤失败的表达式 - protected Map filterFields=new HashMap<>();//参与过滤的字段名和值 - protected Map> filterFieldName2ETLScriptList=new HashMap<>();//过滤字段和ETL脚本 - private List filterFieldNames=new ArrayList<>();//过滤字段列表 + protected List oriFilterFieldNames = new ArrayList<>();//参与过滤的字段名 + protected Map oriFilterFields = new HashMap<>();//原始 + protected List expressions = new ArrayList<>();//过滤失败的表达式 + protected Map filterFields = new HashMap<>();//参与过滤的字段名和值 + protected Map> filterFieldName2ETLScriptList = new HashMap<>();//过滤字段和ETL脚本 protected transient ChainPipeline pipeline; protected transient FilterChainStage stage; - public NotFireReason(FilterChainStage stage,String fieldValues){ - this.stage=stage; - this.pipeline=(ChainPipeline) stage.getPipeline(); - String logFingerFieldNames=this.stage.getPreFingerprint().getLogFingerFieldNames(); - String[] values=logFingerFieldNames.split(","); - for(String oriFieldName:values){ + protected Map filterFieldName2OriFieldName; + private List filterFieldNames = new ArrayList<>();//过滤字段列表 + + public NotFireReason(FilterChainStage stage, String fieldValues) { + this.stage = stage; + this.pipeline = (ChainPipeline) stage.getPipeline(); + String logFingerFieldNames = this.stage.getPreFingerprint().getLogFingerFieldNames(); + if (logFingerFieldNames == null) { + return; + } + String[] values = logFingerFieldNames.split(","); + for (String oriFieldName : values) { oriFilterFieldNames.add(oriFieldName); } - values=fieldValues.split(FingerprintCache.FIELD_VALUE_SPLIT_SIGN); - for(int i=0;i> filterFieldName2ETLScriptList,Map filterFieldName2OriFieldName,List expressions,List filterFieldNames){ - this.expressions.addAll(expressions); - Map> etlScript=new HashMap<>(); - this.filterFieldNames.addAll(filterFieldNames); - Map oriFilterFields=new HashMap<>(); - List oriFilterFieldNames=new ArrayList<>(); - for(String filteFieldName:filterFieldNames){ - String oriFieldName=filterFieldName2OriFieldName.get(filteFieldName); - String oriFieldValue=this.oriFilterFields.get(oriFieldName); - if(oriFieldName!=null&&oriFieldValue!=null){ - oriFilterFields.put(oriFieldName,oriFieldValue); + + public void analysis(IMessage message) { + if (oriFilterFields.size() == 0) { + return; + } + Map> etlScript = new HashMap<>(); + Map oriFilterFields = new HashMap<>(); + List oriFilterFieldNames = new ArrayList<>(); + for (String filteFieldName : filterFieldNames) { + String oriFieldName = filterFieldName2OriFieldName.get(filteFieldName); + String oriFieldValue = this.oriFilterFields.get(oriFieldName); + if (oriFieldName != null && oriFieldValue != null) { + oriFilterFields.put(oriFieldName, oriFieldValue); } - String filterValue=message.getMessageBody().getString(filteFieldName); - if(filterValue!=null){ - filterFields.put(filteFieldName,filterValue); + String filterValue = message.getMessageBody().getString(filteFieldName); + if (filterValue != null) { + filterFields.put(filteFieldName, filterValue); } - if(oriFieldName!=null){ + if (oriFieldName != null) { oriFilterFieldNames.add(oriFieldName); } - List etl=filterFieldName2ETLScriptList.get(filteFieldName); - if(etl!=null){ - etlScript.put(oriFieldName,etl); + List etl = filterFieldName2ETLScriptList.get(filteFieldName); + if (etl != null) { + etlScript.put(oriFieldName, etl); } } - this.filterFieldName2ETLScriptList=etlScript; - this.oriFilterFields=oriFilterFields; - this.oriFilterFieldNames=oriFilterFieldNames; + this.filterFieldName2ETLScriptList = etlScript; + this.oriFilterFields = oriFilterFields; + this.oriFilterFieldNames = oriFilterFieldNames; } public List getOriFilterFieldNames() { @@ -144,33 +151,46 @@ public FilterChainStage getStage() { public ChainPipeline getPipeline() { return pipeline; } + @Override public String toString() { return JsonableUtil.formatJson(toJson()); } - public JSONObject toJson() { - JSONObject jsonObject=new JSONObject(); - jsonObject.put("expression", MapKeyUtil.createKey("\n",this.expressions)); - if(CollectionUtil.isNotEmpty(this.filterFields)){ - JSONObject filterFields=new JSONObject(); - filterFields.putAll(this.filterFields); - jsonObject.put("field current value",filterFields); - } + public JSONObject toJson() { + JSONObject jsonObject = new JSONObject(); + jsonObject.put("表达式和数据", MapKeyUtil.createKey("\n", this.expressions)); + + JSONObject values = new JSONObject(); + if (CollectionUtil.isNotEmpty(this.filterFields)) { + values.putAll(this.filterFields); + } - if(CollectionUtil.isNotEmpty(this.oriFilterFields)){ - JSONObject oriFilterFields=new JSONObject(); - oriFilterFields.putAll(this.oriFilterFields); - jsonObject.put("field original value",oriFilterFields); + if (CollectionUtil.isNotEmpty(this.oriFilterFields)) { + values.putAll(this.oriFilterFields); } - if(CollectionUtil.isNotEmpty(this.filterFieldName2ETLScriptList)){ - JSONObject etl=new JSONObject(); - etl.putAll(this.filterFieldName2ETLScriptList); - jsonObject.put("field etl",etl); + if (CollectionUtil.isNotEmpty(this.filterFieldName2ETLScriptList)) { + List strings = new ArrayList<>(); + for (List scripts : this.filterFieldName2ETLScriptList.values()) { + strings.add(MapKeyUtil.createKey("
", scripts)); + strings.add("

"); + } + jsonObject.put("etl", "
" + MapKeyUtil.createKeyFromCollection("
", strings)); + } + if (CollectionUtil.isNotEmpty(values)) { + jsonObject.put("field value", values); } return jsonObject; } + + public Map getFilterFieldName2OriFieldName() { + return filterFieldName2OriFieldName; + } + + public void setFilterFieldName2OriFieldName(Map filterFieldName2OriFieldName) { + this.filterFieldName2OriFieldName = filterFieldName2OriFieldName; + } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/SourceMetric.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/SourceMetric.java new file mode 100644 index 00000000..3a01eae5 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/SourceMetric.java @@ -0,0 +1,181 @@ +/* + * 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.topology.metric; + +import com.alibaba.fastjson.JSONObject; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.rocketmq.streams.common.channel.source.ISource; +import org.apache.rocketmq.streams.common.channel.source.SplitProgress; +import org.apache.rocketmq.streams.common.metadata.MetaData; +import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.common.utils.JsonableUtil; + +public class SourceMetric { + /** + * metric info + */ + protected AtomicLong outCount = new AtomicLong(0); + protected Long firstReceiveTime; + protected Long lastReceiveTime; + protected Long startTime; + protected volatile long avgCostTime = 0; + protected volatile long maxCostTime = 0; + protected AtomicLong sumCostTime = new AtomicLong(0); + protected Map splitProgresses; + protected ISource source; + protected MetaData metaData; + + public long endCalculate(long startTime) { + long cost = System.currentTimeMillis() - startTime; + long sum = this.sumCostTime.addAndGet(cost); + long count = outCount.incrementAndGet(); + this.avgCostTime = sum / count; + if (this.maxCostTime < cost) { + this.maxCostTime = cost; + } + return cost; + } + + public long incrementOut() { + return outCount.incrementAndGet(); + } + + public Long getOutCount() { + return outCount.get(); + } + + public void setOutCount(AtomicLong outCount) { + this.outCount = outCount; + } + + public String lastMsgReceive() { + if (lastReceiveTime == null) { + return null; + } + return DateUtil.longToString(lastReceiveTime); + } + + public String firstMsgReceive() { + if (firstReceiveTime == null) { + return null; + } + return DateUtil.longToString(firstReceiveTime); + } + + public String startTime() { + if (startTime == null) { + return null; + } + return DateUtil.longToString(startTime); + } + + public String toProgress() { + if (splitProgresses == null) { + return "还未采集到,请检查SQL Source with部分有没有配置sendLogTimeFieldName字段,这个字段是产生消息的时间,格式是时间戳或2022-09-01 12:12:12"; + } + JSONObject jsonObject = new JSONObject(); + for (SplitProgress progress : this.splitProgresses.values()) { + jsonObject.put("分片(" + progress.getSplitId() + ")延迟" + (progress.isTimeStamp() ? "时间(毫秒)" : "条数"), progress.getProgress() < 0 ? 0 : progress.getProgress() + "---(更新时间:" + DateUtil.getCurrentTimeString() + ")"); + } + return JsonableUtil.formatJson(jsonObject); + } + + public String toCost() { + JSONObject jsonObject = new JSONObject(); + jsonObject.put("平均消耗时间", avgCostTime); + jsonObject.put("最大消耗时间", maxCostTime); + jsonObject.put("总消耗时间", sumCostTime); + return JsonableUtil.formatJson(jsonObject); + } + + public void setMsgReceivTime(Long time) { + if (firstReceiveTime == null) { + firstReceiveTime = time; + } + lastReceiveTime = time; + } + + public Long getFirstReceiveTime() { + return firstReceiveTime; + } + + public void setFirstReceiveTime(Long firstReceiveTime) { + this.firstReceiveTime = firstReceiveTime; + } + + public Long getLastReceiveTime() { + return lastReceiveTime; + } + + public void setLastReceiveTime(Long lastReceiveTime) { + this.lastReceiveTime = lastReceiveTime; + } + + public Map getSplitProgresses() { + return splitProgresses; + } + + public void setSplitProgresses( + Map splitProgresses) { + this.splitProgresses = splitProgresses; + } + + public Long getStartTime() { + return startTime; + } + + public void setStartTime(Long startTime) { + this.startTime = startTime; + } + + public long getAvgCostTime() { + return avgCostTime; + } + + public void setAvgCostTime(long avgCostTime) { + this.avgCostTime = avgCostTime; + } + + public long getMaxCostTime() { + return maxCostTime; + } + + public void setMaxCostTime(long maxCostTime) { + this.maxCostTime = maxCostTime; + } + + public long getSumCostTime() { + return sumCostTime.get(); + } + + public ISource getSource() { + return source; + } + + public void setSource(ISource source) { + this.source = source; + } + + public MetaData getMetaData() { + return metaData; + } + + public void setMetaData(MetaData metaData) { + this.metaData = metaData; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/StageGroup.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/StageGroup.java index d1a1fbb4..8f4f56d4 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/StageGroup.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/StageGroup.java @@ -21,12 +21,11 @@ import java.util.List; import java.util.Map; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; - import org.apache.rocketmq.streams.common.model.NameCreatorContext; import org.apache.rocketmq.streams.common.topology.model.AbstractStage; public class StageGroup extends BasedConfigurable { - public static String TYPE="StageGroup"; + public static String TYPE = "StageGroup"; /** * 展示在拓扑中 @@ -35,19 +34,17 @@ public class StageGroup extends BasedConfigurable { /** * stage info */ - protected String startLable; - protected String endLable; - protected List allStageLables; + protected String startLabel; + protected String endLabel; + protected List allStageLabels; - protected transient AbstractStage startStage; - protected transient AbstractStage endStage; + protected transient AbstractStage startStage; + protected transient AbstractStage endStage; protected transient List> allStages; - - protected List childrenNames=new ArrayList<>(); + protected List childrenNames = new ArrayList<>(); protected String parentName; - - protected transient List children=new ArrayList<>(); + protected transient List children = new ArrayList<>(); protected transient StageGroup parent; /** @@ -67,76 +64,87 @@ public class StageGroup extends BasedConfigurable { protected long avgCost; protected long maxCost; - public StageGroup(){ + public StageGroup() { setType(TYPE); } - public StageGroup(AbstractStage startStage, AbstractStage endStage, List> allStages){ + public StageGroup(AbstractStage startStage, AbstractStage endStage, List> allStages) { setType(TYPE); - this.setConfigureName(NameCreatorContext.get().createName("StageGroup")); -// this.setNameSpace(namespace); + + this.setName(NameCreatorContext.get().createName("StageGroup")); + //this.setNameSpace(namespace); setStartStage(startStage); setEndStage(endStage); - if(allStages!=null){ - List allStageLables=new ArrayList<>(); - for(AbstractStage stage:allStages){ - allStageLables.add(stage.getLabel()); + if (allStages != null) { + List allStageLabels = new ArrayList<>(); + for (AbstractStage stage : allStages) { + allStageLabels.add(stage.getLabel()); } - this.allStageLables=allStageLables; - this.allStages=allStages; + this.allStageLabels = allStageLabels; + this.allStages = allStages; } } /** * 在序列化时做反序列化 + * * @param stageMap */ - public void init(Map> stageMap,Map stageGroupMap){ - this.startStage=stageMap.get(this.startLable); - this.endStage=stageMap.get(this.endLable); - this.parent=stageGroupMap.get(this.parentName); - List children=new ArrayList<>(); - for(String name:this.childrenNames){ + public void init(Map> stageMap, Map stageGroupMap) { + this.startStage = stageMap.get(this.startLabel); + this.endStage = stageMap.get(this.endLabel); + this.parent = stageGroupMap.get(this.parentName); + List children = new ArrayList<>(); + for (String name : this.childrenNames) { children.add(stageGroupMap.get(name)); } - this.children=children; - - List> allStages=new ArrayList<>(); - List tmpStageLables=new ArrayList<>(); - tmpStageLables.addAll(this.allStageLables); - Iterator it = tmpStageLables.iterator(); - while (it.hasNext()){ - String name=it.next(); - AbstractStage stage=stageMap.get(name); + this.children = children; + + List> allStages = new ArrayList<>(); + List tmpStageLabels = new ArrayList<>(); + tmpStageLabels.addAll(this.allStageLabels); + Iterator it = tmpStageLabels.iterator(); + while (it.hasNext()) { + String name = it.next(); + AbstractStage stage = stageMap.get(name); allStages.add(stage); stage.setStageGroup(this); } - this.allStages=allStages; - } - - public void calculateMetric(){ - inCount=startStage.getStageMetric().getInCount(); - qps=startStage.getStageMetric().getQps(); - outCount=endStage.getStageMetric().getOutCount(); + this.allStages = allStages; + } + + public void calculateMetric() { + inCount = startStage.calculateInCount(); + qps = startStage.calculateInQPS(); + if (children != null) { + for (StageGroup stageGroup : children) { + if (stageGroup.getStartStage().calculateInCount() > inCount) { + inCount = stageGroup.getStartStage().calculateInCount(); + } + if (stageGroup.getStartStage().calculateInQPS() > qps) { + qps = stageGroup.getStartStage().calculateInQPS(); + } + } + } + outCount = endStage.getStageMetric().getOutCount(); } - public String getStartLable() { - return startLable; + public String getStartLabel() { + return startLabel; } - public void setStartLable(String startLable) { - this.startLable = startLable; + public void setStartLabel(String startLabel) { + this.startLabel = startLabel; } - public String getEndLable() { - return endLable; + public String getEndLabel() { + return endLabel; } - public void setEndLable(String endLable) { - this.endLable = endLable; + public void setEndLabel(String endLabel) { + this.endLabel = endLabel; } - public String getSql() { return sql; } @@ -173,20 +181,6 @@ public long getMaxCost() { return maxCost; } - public void setStartStage(AbstractStage startStage) { - this.startStage = startStage; - this.startLable=startStage.getLabel(); - } - - public void setEndStage(AbstractStage endStage) { - this.endStage = endStage; - this.endLable=endStage.getLabel(); - } - - public void setChildren(List children) { - this.children = children; - } - public StageGroup getParent() { return parent; } @@ -194,8 +188,8 @@ public StageGroup getParent() { public void setParent(StageGroup parent) { parent.children.add(this); this.parent = parent; - this.parentName=this.parent.getConfigureName(); - parent.childrenNames.add(this.getConfigureName()); + this.parentName = this.parent.getName(); + parent.childrenNames.add(this.getName()); } public List getChildrenNames() { @@ -214,24 +208,38 @@ public void setParentName(String parentName) { this.parentName = parentName; } - public AbstractStage getStartStage() { + public AbstractStage getStartStage() { return startStage; } - public AbstractStage getEndStage() { + public void setStartStage(AbstractStage startStage) { + this.startStage = startStage; + this.startLabel = startStage.getLabel(); + } + + public AbstractStage getEndStage() { return endStage; } + public void setEndStage(AbstractStage endStage) { + this.endStage = endStage; + this.endLabel = endStage.getLabel(); + } + public List getChildren() { return children; } - public List getAllStageLables() { - return allStageLables; + public void setChildren(List children) { + this.children = children; + } + + public List getAllStageLabels() { + return allStageLabels; } - public void setAllStageLables(List allStageLables) { - this.allStageLables = allStageLables; + public void setAllStageLabels(List allStageLabels) { + this.allStageLabels = allStageLabels; } public String getViewName() { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/StageMetric.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/StageMetric.java index 046f826e..7d8c14b8 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/StageMetric.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/StageMetric.java @@ -27,52 +27,52 @@ public class StageMetric { /** * metric info */ - protected AtomicLong inCount=new AtomicLong(0); - protected AtomicLong outCount=new AtomicLong(0); + protected AtomicLong inCount = new AtomicLong(0); + protected AtomicLong outCount = new AtomicLong(0); protected transient Long firstReceiveTime; protected double qps; - protected long maxCostTime; - protected long avgCostTime; - protected transient long sumCostTime; + protected volatile long maxCostTime; + protected volatile long avgCostTime; + protected transient AtomicLong sumCostTime = new AtomicLong(0); - protected List notFireReasons=new ArrayList<>(); + protected List notFireReasons = new ArrayList<>(); - public long startCalculate(IMessage msg){ - if(firstReceiveTime==null){ - firstReceiveTime=System.currentTimeMillis(); + public long startCalculate(IMessage msg) { + if (firstReceiveTime == null) { + firstReceiveTime = System.currentTimeMillis(); } - long countValue=inCount.incrementAndGet(); - long timeGap=(System.currentTimeMillis()-firstReceiveTime)/1000; - if(timeGap<1) { + long countValue = inCount.incrementAndGet(); + long timeGap = (System.currentTimeMillis() - firstReceiveTime) / 1000; + if (timeGap < 1) { return System.currentTimeMillis(); } - qps=countValue/timeGap; + qps = countValue / timeGap; return System.currentTimeMillis(); } - public void endCalculate(long startTime){ - long cost=System.currentTimeMillis()-startTime; - this.sumCostTime+=cost; - this.avgCostTime=this.sumCostTime/inCount.get(); - if(this.maxCostTime100){ + while (notFireReasons.size() > 10) { notFireReasons.remove(0); } - } public Long getFirstReceiveTime() { @@ -107,17 +107,10 @@ public void setAvgCostTime(long avgCostTime) { this.avgCostTime = avgCostTime; } - public long getSumCostTime() { - return sumCostTime; - } - - public void setSumCostTime(long sumCostTime) { - this.sumCostTime = sumCostTime; - } - public Long getInCount() { return inCount.get(); } + public Long getOutCount() { return outCount.get(); } @@ -126,13 +119,12 @@ public List getNotFireReasons() { return notFireReasons; } - public String createNotFireReason() { - JSONArray jsonArray=new JSONArray(); - for(NotFireReason notFireReason:this.notFireReasons){ + public synchronized String createNotFireReason() { + JSONArray jsonArray = new JSONArray(); + for (NotFireReason notFireReason : this.notFireReasons) { jsonArray.add(notFireReason.toJson()); } return JsonableUtil.formatJson(jsonArray); } - } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractChainStage.java similarity index 90% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ChainStage.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractChainStage.java index ac867bf7..e029a6ae 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractChainStage.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.common.topology; +package org.apache.rocketmq.streams.common.topology.model; import java.util.Collection; import java.util.HashSet; @@ -22,13 +22,11 @@ import org.apache.rocketmq.streams.common.configurable.annotation.Changeable; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.topology.model.AbstractStage; -import org.apache.rocketmq.streams.common.topology.model.Pipeline; /** * 流拓扑结构的一个节点 */ -public abstract class ChainStage extends AbstractStage { +public abstract class AbstractChainStage extends AbstractStage { /** * 开发使用 @@ -36,7 +34,6 @@ public abstract class ChainStage extends AbstractStage { @Changeable protected String entityName; - //用于展示,辅助说明stage 在sql中的作用,如where函数解析,select 函数解析,select 字段排除等 protected String discription; @@ -86,12 +83,12 @@ public void sendSystem(IMessage message, AbstractContext context, Collection> set = new HashSet<>(); - for (Pipeline pipeline : pipelines) { + for (AbstractPipeline pipeline : pipelines) { if (pipeline != null) { set.add((ChainPipeline) pipeline); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractPipeline.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractPipeline.java new file mode 100644 index 00000000..4f7b6e3c --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractPipeline.java @@ -0,0 +1,165 @@ +/* + * 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.topology.model; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; +import org.apache.rocketmq.streams.common.interfaces.IUDF; +import org.apache.rocketmq.streams.common.optimization.IHomologousOptimization; +import org.apache.rocketmq.streams.common.optimization.fingerprint.PreFingerprint; +import org.apache.rocketmq.streams.common.utils.IdUtil; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 每个pipeline会有一个固定的处理流程,通过stage组成。每个stage可以决定是否需要中断执行,也可以决定下个stage的输入参数 + * + * @param pipeline初期流转的对象,在流转过程中可能会发生变化 + */ +public abstract class AbstractPipeline extends BasedConfigurable implements IStreamOperator { + + public static final Logger LOGGER = LoggerFactory.getLogger(AbstractPipeline.class); + + public static final String TYPE = "pipeline"; + + /** + * pipeline name,通过配置配 + */ + protected transient String name; + protected transient Map> stageMap = new HashMap<>(); + /** + * stage列表 + */ + protected List> stages = new ArrayList<>(); + /** + * 主要用于在join,union场景,标记上游节点用 + */ + protected String msgSourceName; + protected List udfs = new ArrayList<>(); + /** + * KEY: source stage label value: key:next stage label: value :PreFingerprint + */ + protected transient Map> preFingerprintExecutor = new HashMap<>(); + protected transient ExecutorService executeTasks; + protected IHomologousOptimization homologousOptimization; + /** + * 给数据源取个名字,主要用于同源任务归并,数据源名称,如果pipeline是数据源pipeline需要设置 + */ + private String sourceIdentification; + + public AbstractPipeline() { + setType(TYPE); + } + + public void addStage(AbstractStage stage) { + this.stages.add(stage); + } + + /** + * regist pre filter Fingerprint + * + * @param preFingerprint + */ + protected void registPreFingerprint(PreFingerprint preFingerprint) { + if (preFingerprint == null) { + return; + } + Map preFingerprintMap = this.preFingerprintExecutor.computeIfAbsent(preFingerprint.getSourceStageLabel(), k -> new HashMap<>()); + preFingerprintMap.put(preFingerprint.getNextStageLabel(), preFingerprint); + } + + protected PreFingerprint getPreFingerprint(String currentLable, String nextLable) { + Map preFingerprintMap = this.preFingerprintExecutor.get(currentLable); + if (preFingerprintMap == null) { + return null; + } + return preFingerprintMap.get(nextLable); + } + + @Override public void destroy() { + if (LOGGER.isInfoEnabled()) { + LOGGER.info("[{}][{}] Pipeline_Stop_Success", IdUtil.instanceId(), this.getName()); + } + if (executeTasks != null) { + this.executeTasks.shutdown(); + } + } + + protected String createPipelineMonitorName() { + return MapKeyUtil.createKeyBySign(".", getType(), getNameSpace(), this.getName()); + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public List> getStages() { + return stages; + } + + public void setStages(List> stages) { + this.stages = stages; + } + + public String getMsgSourceName() { + return msgSourceName; + } + + public void setMsgSourceName(String msgSourceName) { + this.msgSourceName = msgSourceName; + } + + public String getSourceIdentification() { + return sourceIdentification; + } + + public void setSourceIdentification(String sourceIdentification) { + this.sourceIdentification = sourceIdentification; + } + +// public Map> getPreFingerprintExecutor() { +// return preFingerprintExecutor; +// } + + public List getUdfs() { + return udfs; + } + + public void setUdfs(List udfs) { + this.udfs = udfs; + } + + public void setPreFingerprintExecutor( + Map> preFingerprintExecutor) { + this.preFingerprintExecutor = preFingerprintExecutor; + } + + public IHomologousOptimization getHomologousOptimization() { + return homologousOptimization; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractRule.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractRule.java index c53249e6..db53de8c 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractRule.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractRule.java @@ -30,57 +30,45 @@ public abstract class AbstractRule extends BasedConfigurable implements IStreamO public static final String TYPE = "rule"; public static final String FIRE_RULES = "fireRules"; - - /** - * 这个字段以前存在remark中,现在会在映射时做兼容转换,建议后续直接存储在这个字段 - */ - private String msgMetaDataName; - protected List varNames = new ArrayList(); - protected List actionNames = new ArrayList(); - @Deprecated protected List scriptNames = new ArrayList(); - protected String expressionName; - /** * 规则编号 */ protected String ruleCode; - /** * 规则描述 */ protected String ruleDesc; - /** * 规则标题 */ protected String ruleTitle; - /** * 规则发布状态 */ protected Integer ruleStatus = 0; - + protected boolean supportHyperscan = false;// support hyperscan optimization, if the rule has many regex, suggest open + /** + * 这个字段以前存在remark中,现在会在映射时做兼容转换,建议后续直接存储在这个字段 + */ + private String msgMetaDataName; /** * 风险级别 */ private String ruleLevel; - - protected boolean supportHyperscan=false;// support hyperscan optimization, if the rule has many regex, suggest open - - public abstract Set getDependentFields(); + public AbstractRule() { + setType(TYPE); + } // //public abstract Set getFunctionNames(); - public AbstractRule() { - setType(TYPE); - } + public abstract Set getDependentFields(); public String getMsgMetaDataName() { return msgMetaDataName; @@ -157,7 +145,7 @@ public void setRuleStatus(Integer ruleStatus) { public JSONObject toOutputJson() { JSONObject jsonObject = new JSONObject(); jsonObject.put("ruleNameSpace", getNameSpace()); - jsonObject.put("ruleName", getConfigureName()); + jsonObject.put("ruleName", getName()); if (StringUtil.isNotEmpty(ruleCode)) { jsonObject.put("ruleCode", ruleCode); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractScript.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractScript.java index 8555796a..c548b284 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractScript.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractScript.java @@ -49,6 +49,15 @@ public abstract class AbstractScript extends Based protected String value; protected String scriptType; + /** + * 存在私有空间,可变化 + */ + @Changeable + private String message; + + public AbstractScript() { + setType(TYPE); + } /** * 获取字段名对应的脚本列表 @@ -65,16 +74,6 @@ public abstract class AbstractScript extends Based */ public abstract Map> getDependentFields(); - /** - * 存在私有空间,可变化 - */ - @Changeable - private String message; - - public AbstractScript() { - setType(TYPE); - } - public String getLastUpdateTime() { return lastUpdateTime; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractStage.java index 2434b838..db034a74 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractStage.java @@ -16,36 +16,41 @@ */ package org.apache.rocketmq.streams.common.topology.model; +import com.alibaba.fastjson.JSONArray; import com.alibaba.fastjson.JSONObject; import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Set; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; -import org.apache.rocketmq.streams.common.component.ComponentCreator; +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.configurable.BasedConfigurable; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; -import org.apache.rocketmq.streams.common.interfaces.ISystemMessageProcessor; +import org.apache.rocketmq.streams.common.interfaces.IStage; +import org.apache.rocketmq.streams.common.interfaces.ISystemMessage; +import org.apache.rocketmq.streams.common.optimization.MessageTrace; import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintCache; import org.apache.rocketmq.streams.common.optimization.fingerprint.PreFingerprint; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; +import org.apache.rocketmq.streams.common.topology.metric.NotFireReason; import org.apache.rocketmq.streams.common.topology.metric.StageGroup; import org.apache.rocketmq.streams.common.topology.metric.StageMetric; +import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; +import org.apache.rocketmq.streams.common.utils.IdUtil; +import org.apache.rocketmq.streams.common.utils.JsonableUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.common.utils.TraceUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public abstract class AbstractStage extends BasedConfigurable implements IStreamOperator, ISystemMessageProcessor { - protected String filterFieldNames; - - private static final Log LOG = LogFactory.getLog(AbstractStage.class); - +public abstract class AbstractStage extends BasedConfigurable implements IStage { public static final String TYPE = "stage"; - + private static final Logger LOG = LoggerFactory.getLogger(AbstractStage.class); + protected String filterFieldNames; protected transient String name; /** @@ -55,7 +60,7 @@ public abstract class AbstractStage extends BasedConfigurabl protected String splitDataFieldName; - protected transient Pipeline pipeline; + protected transient AbstractPipeline pipeline; /** * 设置路由label,当需要做路由选择时需要设置 @@ -86,43 +91,149 @@ public abstract class AbstractStage extends BasedConfigurabl * 主要用于排错,把stage按sql分组,可以快速定位问题 */ protected String sql; - protected StageGroup stageGroup; + protected transient StageGroup stageGroup; /** * 前置指纹记录 */ protected transient PreFingerprint preFingerprint = null; //监控信息 - protected transient StageMetric stageMetric=new StageMetric(); + protected transient StageMetric stageMetric = new StageMetric(); public AbstractStage() { setType(TYPE); } - @Override public T doMessage(T t, AbstractContext context) { - long startTime=stageMetric.startCalculate(t); - try { - TraceUtil.debug(t.getHeader().getTraceId(), "AbstractStage", label, t.getMessageBody().toJSONString()); - } catch (Exception e) { - LOG.error("t.getMessageBody() parse error", e); + /** + * 执行一个stage + * + * @param t + * @param context + */ + @Override + public boolean executeStage(T t, AbstractContext context) { + if (t.getHeader().isSystemMessage()) { + ISystemMessage systemMessage = t.getSystemMessage(); + if (systemMessage instanceof CheckPointMessage) { + checkpoint(t, context, (CheckPointMessage) systemMessage); + } else if (systemMessage instanceof NewSplitMessage) { + addNewSplit(t, context, (NewSplitMessage) systemMessage); + } else if (systemMessage instanceof RemoveSplitMessage) { + removeSplit(t, context, (RemoveSplitMessage) systemMessage); + } else if (systemMessage instanceof BatchFinishMessage) { + batchMessageFinish(t, context, (BatchFinishMessage) systemMessage); + } else { + if (systemMessage == null) { + return true; + } + throw new RuntimeException("can not support this system message " + systemMessage.getClass().getName()); + } + if (isAsyncNode()) { + context.breakExecute(); + return false; + } + return true; } - IStageHandle handle = selectHandle(t, context); - if (handle == null) { - return t; + context.resetIsContinue(); + if (context.isSplitModel() && !isCloseSplitMode()) { + List oldSplits = context.getSplitMessages(); + List newSplits = new ArrayList(); + int splitMessageOffset = 0; + boolean isFinishTrace = MessageTrace.existFinishBranch(t); + for (T subT : oldSplits) { + context.closeSplitMode(subT); + subT.getHeader().setMsgRouteFromLable(t.getHeader().getMsgRouteFromLable()); + subT.getHeader().addLayerOffset(splitMessageOffset); + splitMessageOffset++; + Object result = this.doMessage(t, context); + boolean isContinue = (result != null && context.isContinue()); + if (!isContinue) { + context.removeSpliteMessage(subT); + context.cancelBreak(); + continue; + } + //lastMsg=subT; + if (context.isSplitModel()) { + newSplits.addAll(context.getSplitMessages()); + } else { + newSplits.add(subT); + } + } + MessageTrace.setResult(t, isFinishTrace);//因为某些stage可能会嵌套pipline,导致某个pipline执行完成,这里把局部pipline带来的成功清理掉,所以不参与整体的pipline触发逻辑 + //if (needFlush) { + // flushStage(stage, lastMsg, context); + //} + context.setSplitMessages(newSplits); + context.openSplitModel(); + + if (newSplits == null || newSplits.size() == 0) { + context.breakExecute(); + return false; + } + + } else { + if (isCloseSplitMode()) { + if (StringUtil.isNotEmpty(getSplitDataFieldName())) { + List msgs = context.getSplitMessages(); + JSONArray jsonArray = createJsonArray(msgs); + t.getMessageBody().put(getSplitDataFieldName(), jsonArray); + } + context.closeSplitMode(t); + } + Boolean isFinishTrace = MessageTrace.existFinishBranch(t); + Object result = this.doMessage(t, context); + boolean isContinue = (result != null && context.isContinue()); + MessageTrace.setResult(t, isFinishTrace);//因为某些stage可能会嵌套pipline,导致某个pipline执行完成,这里把局部pipline带来的成功清理掉,所以不参与整体的pipline触发逻辑 + return isContinue; } - Object result = handle.doMessage(t, context); - stageMetric.endCalculate(startTime); - if (!context.isContinue() || result == null) { - if(context.getNotFireReason()!=null){ - stageMetric.filterCalculate(context.getNotFireReason()); + return true; + } + + @Override + public T doMessage(T t, AbstractContext context) { + long startTime = stageMetric.startCalculate(t); + T result = null; + try { + TraceUtil.debug(t.getHeader().getTraceId(), "AbstractStage", label, t.getMessageBody().toJSONString()); + result = handleMessage(t, context); + long cost = stageMetric.endCalculate(startTime); + if (cost > 3000) { + LOG.warn("[{}][{}] Stage_Slow_On[{}]----[{}]", IdUtil.instanceId(), this.getPipeline().getName(), this.getSql(), cost); + } + if (!context.isContinue() || result == null) { + NotFireReason notFireReason = null; + if ("true".equals(getConfiguration().getProperty(ConfigurationKey.MONITOR_PIPELINE_HTML_SWITCH))) { + notFireReason = createNotReason(t, context); + if (notFireReason != null) { + stageMetric.filterCalculate(notFireReason); + } + } + if (LOG.isDebugEnabled()) { + + if (notFireReason == null) { + notFireReason = createNotReason(t, context); + } + if (notFireReason != null) { + String info = JsonableUtil.formatJson(notFireReason.toJson()).replace("

", "\n").replace("
", "\r"); + LOG.debug("[{}][{}] Filter_NotMatch_Reason[{}]----\n\r[{}]", IdUtil.instanceId(), this.getPipeline().getName(), this.getSql(), info); + } + + } + + return (T) context.breakExecute(); } - return (T) context.breakExecute(); + stageMetric.outCalculate(); + context.removeNotFireReason(); + } catch (Exception e) { + LOG.error("[{}][{}] Stage_Error_On({})-errorMsg({})", IdUtil.instanceId(), this.getPipeline().getName(), this.getSql(), e.getMessage(), e); + context.breakExecute(); } - stageMetric.outCalculate(); - context.removeNotFireReason(); - return (T) result; + + return result; } + protected abstract T handleMessage(T t, AbstractContext context); + /** * 是否是异步节点,流程会在此节点终止,启动新线程开启后续流程 * @@ -130,17 +241,12 @@ public AbstractStage() { */ public abstract boolean isAsyncNode(); - /** - * 复制一个新的stage,主要用于并发场景 - * - * @return - */ - protected abstract IStageHandle selectHandle(T t, AbstractContext context); - + @Override public String getName() { return name; } + @Override public void setName(String name) { this.name = name; } @@ -184,10 +290,9 @@ public List doRoute(T t) { return this.nextStageLabels; } - List labels = new ArrayList<>(this.nextStageLabels); if (StringUtil.isNotEmpty(routeLabel)) { - Set routeLabelSet=t.getHeader().createRouteLableSet(routeLabel); + Set routeLabelSet = t.getHeader().createRouteLabelSet(routeLabel); labels = new ArrayList<>(); for (String tempLabel : this.nextStageLabels) { if (routeLabelSet.contains(tempLabel)) { @@ -196,7 +301,7 @@ public List doRoute(T t) { } } if (StringUtil.isNotEmpty(filterLabel)) { - Set routeFilterLabelSet=t.getHeader().createRouteLableSet(filterLabel); + Set routeFilterLabelSet = t.getHeader().createRouteLabelSet(filterLabel); for (String tempLabel : this.nextStageLabels) { if (routeFilterLabelSet.contains(label)) { labels.remove(tempLabel); @@ -223,9 +328,9 @@ protected PreFingerprint loadLogFinger() { } filterName = i + ""; } - String stageIdentification = MapKeyUtil.createKeyBySign(".", pipeline.getNameSpace(), pipeline.getConfigureName(), filterName); + String stageIdentification = MapKeyUtil.createKeyBySign(".", pipeline.getNameSpace(), pipeline.getName(), filterName); if (this.filterFieldNames == null) { - this.filterFieldNames = ComponentCreator.getProperties().getProperty(stageIdentification); + this.filterFieldNames = getConfiguration().getProperty(stageIdentification); } if (this.filterFieldNames == null) { return null; @@ -286,20 +391,53 @@ protected PreFingerprint createPreFingerprint(String stageIdentification) { if (sourceLabel == null || nextLabel == null) { return null; } - return new PreFingerprint(this.filterFieldNames, stageIdentification, sourceLabel, nextLabel, -1, this, FingerprintCache.getInstance()); + PreFingerprint preFingerprint = new PreFingerprint(this.filterFieldNames, stageIdentification, sourceLabel, nextLabel, -1, this); + preFingerprint.setFingerprintCache(FingerprintCache.getInstance()); + return preFingerprint; } else { - return new PreFingerprint(this.filterFieldNames, stageIdentification, "0", "0", -1, this, FingerprintCache.getInstance()); + PreFingerprint preFingerprint = new PreFingerprint(this.filterFieldNames, stageIdentification, "0", "0", -1, this); + preFingerprint.setFingerprintCache(FingerprintCache.getInstance()); + return preFingerprint; } } - @Override public void batchMessageFinish(IMessage message, AbstractContext context, BatchFinishMessage checkPointMessage) { + /** + * 分析被过滤的原因 + * + * @param message + * @param context + * @return + */ + private NotFireReason createNotReason(T message, AbstractContext context) { + if (getPipeline().getHomologousOptimization() != null) { + PreFingerprint preFingerprint = getPreFingerprint(); + if (preFingerprint == null) { + return null; + } + String msgKey = preFingerprint.createFieldMsg(message); + NotFireReason notFireReason = getPipeline().getHomologousOptimization().analysisNotFireReason((FilterChainStage) this, msgKey, context.getNotFireExpressionMonitor()); + notFireReason.analysis(message); + return notFireReason; + } + return null; + } + private JSONArray createJsonArray(List msgs) { + JSONArray jsonArray = new JSONArray(); + for (T msg : msgs) { + jsonArray.add(msg.getMessageBody()); + } + return jsonArray; } public List getNextStageLabels() { return nextStageLabels; } + public void setNextStageLabels(List nextStageLabels) { + this.nextStageLabels = nextStageLabels; + } + public List getPrevStageLabels() { return prevStageLabels; } @@ -316,11 +454,11 @@ public void setMsgSourceName(String msgSourceName) { this.msgSourceName = msgSourceName; } - public Pipeline getPipeline() { + public AbstractPipeline getPipeline() { return pipeline; } - public void setPipeline(Pipeline pipeline) { + public void setPipeline(AbstractPipeline pipeline) { this.pipeline = pipeline; } @@ -336,6 +474,14 @@ public StageMetric getStageMetric() { return stageMetric; } + public Long calculateInCount() { + return stageMetric.getInCount(); + } + + public double calculateInQPS() { + return stageMetric.getQps(); + } + public String getSql() { return sql; } @@ -344,10 +490,6 @@ public void setSql(String sql) { this.sql = sql; } - public void setNextStageLabels(List nextStageLabels) { - this.nextStageLabels = nextStageLabels; - } - public String getFilterFieldNames() { return filterFieldNames; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/IStageHandle.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractStageHandle.java similarity index 80% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/IStageHandle.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractStageHandle.java index 78fb7687..9a54bd26 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/IStageHandle.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractStageHandle.java @@ -16,17 +16,14 @@ */ package org.apache.rocketmq.streams.common.topology.model; -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.IMessage; +import org.apache.rocketmq.streams.common.topology.ILifeCycle; -public abstract class IStageHandle implements ILifeCycle { - - private static final Log LOG = LogFactory.getLog(IStageHandle.class); +public abstract class AbstractStageHandle implements ILifeCycle { @Override - public T doMessage(T t, AbstractContext context) { + public T doMessage(T t, AbstractContext context) { return doProcess(t, context); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/ChainPipeline.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/ChainPipeline.java new file mode 100644 index 00000000..949858dd --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/ChainPipeline.java @@ -0,0 +1,516 @@ +/* + * 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.topology.model; + +import com.alibaba.fastjson.JSONObject; +import com.google.common.collect.Lists; +import java.io.File; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.ServiceLoader; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; +import org.apache.rocketmq.streams.common.channel.impl.memory.MemorySource; +import org.apache.rocketmq.streams.common.channel.source.ISource; +import org.apache.rocketmq.streams.common.configurable.IConfigurable; +import org.apache.rocketmq.streams.common.configurable.annotation.ConfigurableReference; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.context.AbstractContext; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.context.Message; +import org.apache.rocketmq.streams.common.interfaces.IConfigurablePropertySetter; +import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; +import org.apache.rocketmq.streams.common.interfaces.IUDF; +import org.apache.rocketmq.streams.common.metadata.MetaData; +import org.apache.rocketmq.streams.common.monitor.ConsoleMonitorManager; +import org.apache.rocketmq.streams.common.monitor.IMonitor; +import org.apache.rocketmq.streams.common.optimization.IHomologousCalculate; +import org.apache.rocketmq.streams.common.optimization.IHomologousOptimization; +import org.apache.rocketmq.streams.common.optimization.MessageTrace; +import org.apache.rocketmq.streams.common.optimization.fingerprint.PreFingerprint; +import org.apache.rocketmq.streams.common.threadpool.ScheduleFactory; +import org.apache.rocketmq.streams.common.topology.metric.StageGroup; +import org.apache.rocketmq.streams.common.topology.stages.OutputChainStage; +import org.apache.rocketmq.streams.common.utils.DipperThreadLocalUtil; +import org.apache.rocketmq.streams.common.utils.FileUtil; +import org.apache.rocketmq.streams.common.utils.IdUtil; +import org.apache.rocketmq.streams.common.utils.PipelineHTMLUtil; +import org.apache.rocketmq.streams.common.utils.PrintUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; + +; + +/** + * 数据流拓扑结构,包含了source 算子,sink + */ +public class ChainPipeline extends AbstractPipeline implements Serializable, Runnable, IConfigurablePropertySetter { + + private static final long serialVersionUID = -5189371682717444347L; + @ConfigurableReference protected ISource source; + + /** + * 是否发布,默认为true,关闭发布时,此字段为false,pipeline启动时应判断此字段是否为true,status默认都为1,status为0代表pipeline已被删除 + */ + protected transient AtomicBoolean hasStart = new AtomicBoolean(false); + protected transient List rootStageGroups = new ArrayList<>(); + /** + * channel对应后续的stageName + */ + protected List channelNextStageLabel; + /** + * 数据源输入格式,主要用于日志指纹过滤,如果没有则不做优化 + */ + protected MetaData channelMetaData; + /** + * 为了图形化拓扑和监控使用 + */ + protected transient List stageGroups = new ArrayList<>(); + protected String createTableSQL; + protected IHomologousCalculate homologousCalculate; + private String channelName; + + @Override protected boolean initConfigurable() { + createStageMap(); + Map stageGroupMap = createStageGroupMap(); + for (StageGroup stageGroup : this.stageGroups) { + stageGroup.init(this.stageMap, stageGroupMap); + if (stageGroup.getParent() == null && !this.rootStageGroups.contains(stageGroup)) { + this.rootStageGroups.add(stageGroup); + } + } + + return super.initConfigurable(); + } + + /** + * 启动一个channel,并给channel应用pipeline + */ + + public void startJob() { + + //可重入 + if (!hasStart.compareAndSet(false, true)) { + return; + } + String instanceId = IdUtil.instanceId(); + try { + startPipeline(); + + final IStreamOperator receiver = this; + + source.start((IStreamOperator) (message, context) -> { + if (!message.getHeader().isSystemMessage()) { + //如果没有前置数据源则从消息里面取延迟 + //msg.put("__time__",message.getHeader().getEventMsgTime()); + ConsoleMonitorManager.getInstance().reportChannel(ChainPipeline.this, source, message); + } + message.getHeader().setPipelineName(this.getName()); + + //然后再执行正式逻辑,测试正式逻辑遇到表达是计算,会先从头部信息上去找,如果找到就直接返回,如果没有才进行正式的计算 + return receiver.doMessage(message, context); + }); + + if ("true".equals(configuration.getProperty(ConfigurationKey.MONITOR_PIPELINE_HTML_SWITCH))) { + ScheduleFactory.getInstance().execute(getNameSpace() + "-" + getName() + "-monitor_switch_schedule", this, 0, 10, TimeUnit.SECONDS); + } + LOGGER.info("[{}][{}] Pipeline_Start_Success", instanceId, this.getName()); + } catch (Exception e) { + this.hasStart.set(false); + //pipeline启动失败日志输出 + LOGGER.error("[{}][{}] Pipeline_Start_Error", instanceId, this.getName(), e); + throw e; + } + + } + + /** + * 可以替换某个阶段的阶段,而不用配置的阶段 + * + * @param t 数据 + * @param context 上下文 + * @return stage执行后的结果 + */ + @Override public T doMessage(T t, AbstractContext context) { + if (homologousCalculate != null) { + homologousCalculate.calculate(t, context); + } + + if (!t.getHeader().isSystemMessage()) { + MessageTrace.joinMessage(t);//关联全局监控器 + } + context.setMessage(t); + doNextStages(context, getMsgSourceName(), channelName, this.channelNextStageLabel); + return t; + } + + public boolean isTopology() { + return isTopology(this.channelNextStageLabel); + } + + public void doNextStages(AbstractContext context, String msgPrevSourceName, String currentLabel, List nextStageLabel) { + if (!isTopology(nextStageLabel)) { + return; + } + String oriMsgPreSourceName = msgPrevSourceName; + int size = nextStageLabel.size(); + for (String label : nextStageLabel) { + AbstractContext copyContext = context; + if (size > 1) { + copyContext = context.copy(); + } + T msg = (T) copyContext.getMessage(); + AbstractStage oriStage = stageMap.get(label); + if (oriStage == null) { + if (stages != null && stages.size() > 0) { + synchronized (this) { + oriStage = stageMap.get(label); + if (oriStage == null) { + createStageMap(); + oriStage = stageMap.get(label); + } + } + } + if (oriStage == null) { + LOGGER.warn("[{}][{}] expect stage named {}, but the stage is not exist", IdUtil.instanceId(), this.getName(), label); + continue; + } + } + AbstractStage stage = oriStage; + if (filterByPreFingerprint(msg, copyContext, currentLabel, label)) { + continue; + } + + if (!msg.getHeader().isSystemMessage()) { + ConsoleMonitorManager.getInstance().reportInput(stage, msg); + } + + //boolean needFlush = needFlush(msg); + if (StringUtil.isNotEmpty(oriMsgPreSourceName)) { + msg.getHeader().setMsgRouteFromLable(oriMsgPreSourceName); + } + boolean isContinue = stage.executeStage(msg, copyContext); + + if (!isContinue) { + //只要执行到了window分支都不应该被过滤 + if (stage.isAsyncNode() && !msg.getHeader().isSystemMessage()) { + MessageTrace.finishPipeline(msg); + } + } else { + if (!msg.getHeader().isSystemMessage()) { + ConsoleMonitorManager.getInstance().reportOutput(stage, msg, ConsoleMonitorManager.MSG_FLOWED, null); + } + if (stage instanceof AbstractChainStage) { + AbstractChainStage chainStage = (AbstractChainStage) stage; + String msgSourceName = chainStage.getMsgSourceName(); + if (StringUtil.isNotEmpty(msgSourceName)) { + msgPrevSourceName = msgSourceName; + } + } + + if (copyContext.isSplitModel()) { + List messageList = copyContext.getSplitMessages(); + int splitMessageOffset = 0; + for (IMessage message : messageList) { + AbstractContext abstractContext = copyContext.copy(); + abstractContext.closeSplitMode(message); + message.getHeader().setMsgRouteFromLable(msg.getHeader().getMsgRouteFromLable()); + message.getHeader().addLayerOffset(splitMessageOffset); + splitMessageOffset++; + List labels = stage.doRoute(message); + if (labels == null || labels.size() == 0) { + if (!message.getHeader().isSystemMessage()) { + MessageTrace.finishPipeline(message); + } + continue; + } + doNextStages(abstractContext, msgPrevSourceName, stage.getLabel(), labels); + } + } else { + List labels = stage.doRoute(msg); + if (labels == null || labels.size() == 0) { + if (!msg.getHeader().isSystemMessage()) { + MessageTrace.finishPipeline(msg); + } + continue; + } + doNextStages(copyContext, msgPrevSourceName, stage.getLabel(), labels); + } + } + } + } + + protected boolean filterByPreFingerprint(IMessage t, AbstractContext context, String sourceName, String nextLable) { + PreFingerprint preFingerprint = getPreFingerprint(sourceName, nextLable); + if (preFingerprint != null) { + boolean isFilter = preFingerprint.filterByLogFingerprint(t); + if (isFilter) { + context.breakExecute(); + return true; + } + } + return false; + } + + public List executeWithMsgs(List msgs) { + MemorySource memorySource = new MemorySource(); + memorySource.setReceiver(this); + memorySource.setNameSpace(this.getNameSpace()); + memorySource.setName(this.getName()); + memorySource.init(); + this.setSource(memorySource); + if (msgs == null) { + return null; + } + int sinkCount = 0; + for (AbstractStage stage : getStages()) { + if (stage instanceof OutputChainStage) { + OutputChainStage outputChainStage = (OutputChainStage) stage; + outputChainStage.setCallbackModel(true); + sinkCount++; + } + } + + startPipeline(); + MessageFinishCallBack messageFinishCallBack = new MessageFinishCallBack(); + messageFinishCallBack.setSinkCount(sinkCount); + for (JSONObject jsonObject : msgs) { + memorySource.doReceiveMessage(jsonObject); + } + + try { + memorySource.sendCheckpoint(memorySource.getQueueId()); + memorySource.executeMessage((Message) BatchFinishMessage.create(messageFinishCallBack)); + List result = messageFinishCallBack.get(); + if ("true".equals(configuration.getProperty(ConfigurationKey.MONITOR_PIPELINE_HTML_SWITCH))) { + this.run(); + } + return result; + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + destroy(); + } + } + + public ChainPipeline addChainStage(AbstractChainStage chainStage) { + addStage(chainStage); + return this; + } + + public ISource getSource() { + return source; + } + + public void setSource(ISource source) { + this.source = source; + if (getNameSpace() == null) { + setNameSpace(source.getNameSpace()); + } + if (getChannelName() == null) { + channelName = source.getName(); + } + + } + + public void startPipeline() { + + if (this.udfs != null) { + for (IUDF iudf : this.udfs) { + iudf.loadUDF(); + } + } + + List> stages = this.getStages(); + for (AbstractStage stage : stages) { + stage.setConfiguration(this.configuration); + stage.startJob(); + } + + Iterable iterable = ServiceLoader.load(IHomologousOptimization.class); + Iterator it = iterable.iterator(); + if (it.hasNext()) { + IHomologousOptimization homologousOptimization = it.next(); + homologousOptimization.optimizate(Lists.newArrayList(this)); + this.homologousOptimization = homologousOptimization; + this.homologousOptimization.setPeFingerprintForPipeline(this); + this.homologousCalculate = homologousOptimization.createHomologousCalculate(); + } + + } + + private Map createStageGroupMap() { + Map map = new HashMap<>(); + for (StageGroup stageGroup : stageGroups) { + map.put(stageGroup.getName(), stageGroup); + } + return map; + } + + public Map> createStageMap() { + for (AbstractStage stage : getStages()) { + stage.init(); + stageMap.put(stage.getLabel(), stage); + stage.setPipeline(this); + } + return stageMap; + } + + public List getChannelNextStageLabel() { + return channelNextStageLabel; + } + + public void setChannelNextStageLabel(List channelNextStageLabel) { + this.channelNextStageLabel = channelNextStageLabel; + } + + private IMonitor createPipelineMonitor() { + + IMonitor pipelineMonitorForChannel = DipperThreadLocalUtil.get(); + final IStreamOperator receiver = this; + if (pipelineMonitorForChannel == null) { + //主要监控channel的启动 + pipelineMonitorForChannel = IMonitor.createMonitor(this); + } + return pipelineMonitorForChannel; + + } + + @Override public String toString() { + String LINE = PrintUtil.LINE; + StringBuilder sb = new StringBuilder(); + sb.append("###namespace=").append(getNameSpace()).append("###").append(LINE); + if (source != null) { + sb.append(source.toString()).append(LINE); + } + if (stages != null) { + for (AbstractStage stage : stages) { + sb.append(stage.toString()); + } + } + return sb.toString(); + } + + @Override public void destroy() { + super.destroy(); + if (source != null) { + source.destroy(); + hasStart.set(false); + } + if (this.udfs != null) { + for (IUDF udf : this.udfs) { + udf.destroy(); + } + } + for (Map.Entry> entry : stageMap.entrySet()) { + entry.getValue().stopJob(); + } + if ("true".equals(configuration.getProperty(ConfigurationKey.MONITOR_PIPELINE_HTML_SWITCH))) { + ScheduleFactory.getInstance().cancel(getNameSpace() + "-" + getName() + "-monitor_switch_schedule"); + } + } + + public void addStageGroup(StageGroup stageGroup) { + if (this.stageGroups != null) { + this.stageGroups.add(stageGroup); + } + } + + protected boolean isTopology(List nextStageLabel) { + return nextStageLabel != null && nextStageLabel.size() != 0; + } + + public Map> getStageMap() { + return stageMap; + } + + public void setStageMap(Map> stageMap) { + this.stageMap = stageMap; + } + + public Boolean getHasStart() { + return hasStart.get(); + } + + public MetaData getChannelMetaData() { + return channelMetaData; + } + + public void setChannelMetaData(MetaData channelMetaData) { + this.channelMetaData = channelMetaData; + } + + public String getChannelName() { + return channelName; + } + + public void setChannelName(String channelName) { + this.channelName = channelName; + } + + public List getStageGroups() { + return stageGroups; + } + + public void setStageGroups(List stageGroups) { + this.stageGroups = stageGroups; + } + + public List getRootStageGroups() { + return rootStageGroups; + } + + public void setRootStageGroups(List rootStageGroups) { + this.rootStageGroups = rootStageGroups; + } + + public String getCreateTableSQL() { + return createTableSQL; + } + + public void setCreateTableSQL(String createTableSQL) { + this.createTableSQL = createTableSQL; + } + + @Override public void run() { + String filePath = FileUtil.getJarPath(); + if (StringUtil.isEmpty(filePath)) { + filePath = "/tmp"; + } + filePath = filePath + File.separator + getName() + ".html"; + try { + String html = PipelineHTMLUtil.createHTML(this); + FileUtil.write(filePath, html); + LOGGER.info("[{}][{}] create pipeline html success in {}", IdUtil.instanceId(), getName(), filePath); + } catch (Exception e) { + LOGGER.info("[{}][{}] create pipeline html error on {}", IdUtil.instanceId(), getName(), e.getMessage(), e); + } + + } + + @Override public void setConfigurableProperty(IConfigurable newConfigurable) { + if (newConfigurable.getClass().isAssignableFrom(ChainPipeline.class)) { + ChainPipeline newPipeline = (ChainPipeline) newConfigurable; + this.toObject(newPipeline.toJson()); + } + } + +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/JobGraphCopy.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/JobGraphCopy.java new file mode 100644 index 00000000..a466716a --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/JobGraphCopy.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.topology.model; + +import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; +import java.util.List; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; +import org.apache.rocketmq.streams.common.configurable.IConfigurable; +import org.apache.rocketmq.streams.common.enums.StageType; +import org.apache.rocketmq.streams.common.model.JobConfigure; +import org.apache.rocketmq.streams.common.model.JobStage; +import org.apache.rocketmq.streams.common.model.StageRelation; +import org.apache.rocketmq.streams.common.topology.metric.StageGroup; +import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; +import org.apache.rocketmq.streams.common.topology.stages.ScriptChainStage; +import org.apache.rocketmq.streams.common.utils.PrintUtil; + +public class JobGraphCopy extends BasedConfigurable { + public static final String DEFAULT_NAMESPACE = "dipper.private.blink.rules"; + public static String TYPE = "JobGraph"; + + protected transient List pipelineNamesBySource; + protected transient List> pipelinesBySource;//一个数据源一个pipeline + + /** + * 为了监控使用,正常使用可以暂时忽略 + */ + protected List jobConfigures; + protected List stageRelations; + protected List jobStages; + + public JobGraphCopy() { + setType(TYPE); + } + + private static void deepBuild(List children, List stageRelationDOList, String jobName) { + if (CollectionUtils.isEmpty(children)) { + return; + } + for (StageGroup child : children) { + StageRelation stageRelationDO = buildStageRelation(child, jobName); + stageRelationDOList.add(stageRelationDO); + deepBuild(child.getChildren(), stageRelationDOList, jobName); + } + } + + private static StageRelation buildStageRelation(StageGroup stageGroup, String jobName) { + // TODO group中需要加入pos信息 + String sqlContent = new JSONObject() + .fluentPut("sql", stageGroup.getSql()) + // .fluentPut("pos", stageGroup.getPos()) + .toJSONString(); + StageRelation stageRelation = new StageRelation(); + stageRelation.setJobName(jobName); + stageRelation.setGroupName(stageGroup.getName()); + stageRelation.setSqlContent(sqlContent); + stageRelation.setViewName(stageGroup.getViewName()); + stageRelation.setStageLabels(JSON.toJSONString(stageGroup.getAllStageLabels())); + stageRelation.setStartLabel(stageGroup.getStartLabel()); + stageRelation.setEndLabel(stageGroup.getEndLabel()); + stageRelation.setParentName(stageGroup.getParentName()); + stageRelation.setChildrenNames(JSON.toJSONString(stageGroup.getChildrenNames())); + return stageRelation; + } + + private static String getSqlContent(AbstractStage stage) { + return new JSONObject() + .fluentPut("sql", stage.getSql()) + // .fluentPut("pos", stage.getPos()) + .toJSONString(); + } + + protected static String getStageContent(AbstractStage stage) { + String type = StageType.getTypeForStage(stage); + StringBuilder sb = new StringBuilder(); + switch (type) { + case "filter": + FilterChainStage filterChainStage = (FilterChainStage) stage; + AbstractRule rule = filterChainStage.getRule(); + sb.append(rule.toString() + PrintUtil.LINE); + break; + case "script": + ScriptChainStage scriptChainStage = (ScriptChainStage) stage; + AbstractScript functionScript = scriptChainStage.getScript(); + functionScript.init(); + sb.append(functionScript.toString()); + break; + } + return sb.toString(); + } + + /** + * 创建监控相关的信息 + * + * @param pipelines + * @param configurables + */ + public void createJobInfo(List> pipelines, List configurables) { + createJobConfiguables(configurables); + createJobStages(pipelines); + createStageRelation(pipelines); + } + + private void createJobStages(List> pipelines) { + if (pipelines == null) { + return; + } + for (ChainPipeline pipeline : pipelines) { + List stageDOList = new ArrayList<>(); + int i = 0; + List> stages = pipeline.getStages(); + for (AbstractStage stage : stages) { + List prevStageLabels = stage.getPrevStageLabels(); + if (CollectionUtils.isEmpty(prevStageLabels) || StringUtils.equals(prevStageLabels.get(0), getName())) { + //上游为null 虚拟一个source + String sqlContent = new JSONObject() + .fluentPut("sql", pipeline.getCreateTableSQL()) + // .fluentPut("pos", pipeline.getCreateTableSQLPos()) + .toJSONString(); + JobStage source = new JobStage(); + source.setJobName(getName()); + source.setStageName(getName() + "_source_" + i++); + source.setMachineName(""); + source.setStageType(StageType.SOURCE.getType()); + source.setStageContent(""); + source.setSqlContent(sqlContent); + source.setPrevStageLables("[]"); + source.setNextStageLables("[\"" + stage.getLabel() + "\"]"); + stageDOList.add(source); + } + JobStage stageDO = new JobStage(); + stageDO.setJobName(getName()); + stageDO.setStageName(stage.getLabel()); + stageDO.setMachineName(""); + stageDO.setStageType(StageType.getTypeForStage(stage)); + stageDO.setStageContent(getStageContent(stage)); + stageDO.setSqlContent(getSqlContent(stage)); + stageDO.setPrevStageLables(JSON.toJSONString(prevStageLabels)); + stageDO.setNextStageLables(JSON.toJSONString(stage.getNextStageLabels())); + stageDOList.add(stageDO); + } + this.jobStages = stageDOList; + } + } + + public void createStageRelation(List> pipelines) { + if (pipelines == null) { + return; + } + List stageRelations = new ArrayList<>(); + for (ChainPipeline chainPipeline : pipelines) { + List groups = chainPipeline.getRootStageGroups(); + for (StageGroup group : groups) { + StageRelation stageRelationDO = buildStageRelation(group, getName()); + stageRelationDO.setParentName(null); + stageRelations.add(stageRelationDO); + deepBuild(group.getChildren(), stageRelations, getName()); + } + + } + this.stageRelations = stageRelations; + + } + + protected void createJobConfiguables(List configurables) { + if (configurables == null) { + return; + } + List jobConfigures = new ArrayList<>(); + for (IConfigurable configurable : configurables) { + JobConfigure jobConfigure = new JobConfigure(); + jobConfigure.setJobName(getName()); + jobConfigure.setName(configurable.getName()); + jobConfigure.setConfigureNamespace(configurable.getNameSpace()); + jobConfigure.setConfigureType(configurable.getType()); + jobConfigures.add(jobConfigure); + } + JobConfigure relationDO = new JobConfigure(); + relationDO.setJobName(getName()); + relationDO.setName(getName()); + relationDO.setConfigureNamespace(DEFAULT_NAMESPACE); + relationDO.setConfigureType("stream_task"); + jobConfigures.add(relationDO); + this.jobConfigures = jobConfigures; + } + + public List getJobConfigures() { + return jobConfigures; + } + + public void setJobConfigures(List jobConfigures) { + this.jobConfigures = jobConfigures; + } + + public List getStageRelations() { + return stageRelations; + } + + public void setStageRelations(List stageRelations) { + this.stageRelations = stageRelations; + } + + public List getJobStages() { + return jobStages; + } + + public void setJobStages(List jobStages) { + this.jobStages = jobStages; + } + + public List getPipelineNamesBySource() { + return pipelineNamesBySource; + } + + public void setPipelineNamesBySource(List pipelineNamesBySource) { + this.pipelineNamesBySource = pipelineNamesBySource; + } + + public void setPipelines(List> pipelines) { + if (pipelines == null) { + return; + } + this.pipelinesBySource = pipelines; + List pipelineNames = new ArrayList<>(); + for (ChainPipeline chainPipeline : pipelines) { + pipelineNames.add(chainPipeline.getName()); + } + setPipelineNamesBySource(pipelineNames); + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/MessageFinishCallBack.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/MessageFinishCallBack.java new file mode 100644 index 00000000..20a6e231 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/MessageFinishCallBack.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.topology.model; + +import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; + +public class MessageFinishCallBack implements Future> { + protected Set result = new HashSet<>(); + protected volatile boolean isDone = false; + protected int sinkCount = 1; + protected AtomicInteger finishSinkCount = new AtomicInteger(0); + + @Override public boolean cancel(boolean mayInterruptIfRunning) { + return false; + } + + @Override public boolean isCancelled() { + return false; + } + + @Override public boolean isDone() { + return isDone; + } + + @Override public List get() throws InterruptedException, ExecutionException { + if (isDone) { + return new ArrayList<>(result); + } + synchronized (this) { + this.wait(); + } + return new ArrayList<>(result); + } + + @Override public List get(long timeout, + TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + if (isDone) { + return new ArrayList<>(result); + } + synchronized (this) { + this.wait(createMilliseconds(timeout, unit)); + } + return new ArrayList<>(result); + } + + public synchronized void finish(List result) { + int count = finishSinkCount.incrementAndGet(); + if (result == null) { + return; + } + this.result.addAll(result); + if (count == sinkCount) { + isDone = true; + this.notifyAll(); + } + + } + + protected long createMilliseconds(long timeout, TimeUnit unit) { + if (TimeUnit.MILLISECONDS.equals(unit)) { + return timeout; + } else if (TimeUnit.SECONDS.equals(unit)) { + return timeout * 1000; + } else if (TimeUnit.MINUTES.equals(unit)) { + return timeout * 1000 * 60; + } else if (TimeUnit.HOURS.equals(unit)) { + return timeout * 1000 * 60 * 60; + } else { + throw new RuntimeException("can not support the unit " + unit + ", can use SECONDS,MINUTES,HOURS"); + } + } + + public int getSinkCount() { + return sinkCount; + } + + public void setSinkCount(int sinkCount) { + this.sinkCount = sinkCount; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/Pipeline.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/Pipeline.java deleted file mode 100644 index c79c18ee..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/Pipeline.java +++ /dev/null @@ -1,345 +0,0 @@ -/* - * 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.topology.model; - -import com.alibaba.fastjson.JSONArray; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; -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.configurable.BasedConfigurable; -import org.apache.rocketmq.streams.common.context.AbstractContext; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; -import org.apache.rocketmq.streams.common.interfaces.ISystemMessage; -import org.apache.rocketmq.streams.common.optimization.MessageGlobleTrace; -import org.apache.rocketmq.streams.common.optimization.fingerprint.PreFingerprint; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; -import org.apache.rocketmq.streams.common.utils.StringUtil; - -/** - * 每个pipline会有一个固定的处理流程,通过stage组成。每个stage可以决定是否需要中断执行,也可以决定下个stage的输入参数 - * - * @param pipline初期流转的对象,在流转过程中可能会发生变化 - */ -public class Pipeline extends BasedConfigurable implements IStreamOperator { - - public static final Log LOG = LogFactory.getLog(Pipeline.class); - - public static final String TYPE = "pipeline"; - - /** - * pipeline name,通过配置配 - */ - protected transient String name; - - /** - * stage列表 - */ - protected List> stages = new ArrayList<>(); - - /** - * 给数据源取个名字,主要用于同源任务归并,数据源名称,如果pipeline是数据源pipeline需要设置 - */ - private String sourceIdentification; - /** - * 主要用于在join,union场景,标记上游节点用 - */ - protected String msgSourceName; - - /** - * KEY: source stage label value: key:next stage label: value :PreFingerprint - */ - protected transient Map> preFingerprintExecutor = new HashMap<>(); - - public Pipeline() { - setType(TYPE); - } - - @Override public T doMessage(T t, AbstractContext context) { - T message = doMessage(t, context, null); - return message; - } - - public T doMessage(T t, AbstractContext context, AbstractStage... replaceStage) { - T message = doMessageInner(t, context, replaceStage); - context.setMessage(message); - return message; - } - - /** - * 可以替换某个阶段的阶段,而不用配置的阶段 - * - * @param t - * @param context - * @param replaceStage - * @return - */ - protected T doMessageInner(T t, AbstractContext context, AbstractStage... replaceStage) { - return doMessageFromIndex(t, context, 0, replaceStage); - } - - public T doMessageFromIndex(T t, AbstractContext context, int index, AbstractStage... replaceStage) { - context.setMessage(t); - //boolean needFlush = needFlush(t); - for (int i = index; i < stages.size(); i++) { - AbstractStage oriStage = stages.get(i); - AbstractStage stage = chooseReplaceStage(oriStage, replaceStage); - boolean isContinue = executeStage(stage, t, context); - if (!isContinue) { - if (stage.isAsyncNode()) { - MessageGlobleTrace.finishPipeline(t); - ; - } - return t; - } - } - MessageGlobleTrace.finishPipeline(t); - return t; - } - - @Override protected boolean initConfigurable() { - for(AbstractStage stage:stages){ - stage.init(); - } - return super.initConfigurable(); - } - - /** - * regist pre filter Fingerprint - * - * @param preFingerprint - */ - protected void registPreFingerprint(PreFingerprint preFingerprint) { - if (preFingerprint == null) { - return; - } - Map preFingerprintMap = this.preFingerprintExecutor.computeIfAbsent(preFingerprint.getSourceStageLabel(), k -> new HashMap<>()); - preFingerprintMap.put(preFingerprint.getNextStageLabel(), preFingerprint); - } - - protected PreFingerprint getPreFingerprint(String currentLable, String nextLable) { - Map preFingerprintMap = this.preFingerprintExecutor.get(currentLable); - if (preFingerprintMap == null) { - return null; - } - return preFingerprintMap.get(nextLable); - } - - /** - * 执行一个stage - * - * @param stage - * @param t - * @param context - */ - protected boolean executeStage(AbstractStage stage, T t, AbstractContext context) { - if (t.getHeader().isSystemMessage()) { - ISystemMessage systemMessage = t.getSystemMessage(); - if (systemMessage instanceof CheckPointMessage) { - stage.checkpoint(t, context, (CheckPointMessage) systemMessage); - } else if (systemMessage instanceof NewSplitMessage) { - stage.addNewSplit(t, context, (NewSplitMessage) systemMessage); - } else if (systemMessage instanceof RemoveSplitMessage) { - stage.removeSplit(t, context, (RemoveSplitMessage) systemMessage); - } else if (systemMessage instanceof BatchFinishMessage) { - stage.batchMessageFinish(t, context, (BatchFinishMessage) systemMessage); - } else { - if (systemMessage == null) { - return true; - } - throw new RuntimeException("can not support this system message " + systemMessage.getClass().getName()); - } - if (stage.isAsyncNode()) { - context.breakExecute(); - return false; - } - return true; - } - context.resetIsContinue(); - if (context.isSplitModel() && stage.isCloseSplitMode() == false) { - List oldSplits = context.getSplitMessages(); - List newSplits = new ArrayList(); - int splitMessageOffset = 0; - T lastMsg = null; - for (T subT : oldSplits) { - context.closeSplitMode(subT); - if(ChainPipeline.class.isInstance(this)&&!((ChainPipeline)this).isTopology()&&StringUtil.isNotEmpty(this.msgSourceName)){ - subT.getHeader().setMsgRouteFromLable(t.getHeader().getMsgRouteFromLable()); - }else { - subT.getHeader().setMsgRouteFromLable(t.getHeader().getMsgRouteFromLable()); - } - - subT.getHeader().addLayerOffset(splitMessageOffset); - splitMessageOffset++; - boolean isContinue = doMessage(subT, stage, context); - lastMsg = subT; - if (!isContinue) { - context.removeSpliteMessage(subT); - context.cancelBreak(); - continue; - } - //lastMsg=subT; - if (context.isSplitModel()) { - newSplits.addAll(context.getSplitMessages()); - } else { - newSplits.add(subT); - } - } - MessageGlobleTrace.clear(t);//因为某些stage可能会嵌套pipline,导致某个pipline执行完成,这里把局部pipline带来的成功清理掉,所以不参与整体的pipline触发逻辑 - //if (needFlush) { - // flushStage(stage, lastMsg, context); - //} - context.setSplitMessages(newSplits); - context.openSplitModel(); - - if (newSplits == null || newSplits.size() == 0) { - context.breakExecute(); - return false; - } - - } else { - if (stage.isCloseSplitMode()) { - if (StringUtil.isNotEmpty(stage.getSplitDataFieldName())) { - List msgs = context.getSplitMessages(); - JSONArray jsonArray = createJsonArray(msgs); - t.getMessageBody().put(stage.getSplitDataFieldName(), jsonArray); - } - context.closeSplitMode(t); - } - boolean isContinue = doMessage(t, stage, context); - MessageGlobleTrace.clear(t);//因为某些stage可能会嵌套pipline,导致某个pipline执行完成,这里把局部pipline带来的成功清理掉,所以不参与整体的pipline触发逻辑 - //if (needFlush) { - // flushStage(stage, t, context); - //} - if (!isContinue) { - return false; - } - } - return true; - } - - public boolean isAsynNode() { - //for(AbstractStage stage:stages){ - // if(stage.supportRepeateMessageFilter()==false){ - // return false; - // } - //} - return false; - } - - //private void flushStage(AbstractStage stage, IMessage message, AbstractContext context) { - // stage.checkpoint(message, context); - //} - - //protected boolean needFlush(T msg) { - // return msg.getHeader().isNeedFlush(); - //} - - private JSONArray createJsonArray(List msgs) { - JSONArray jsonArray = new JSONArray(); - for (T msg : msgs) { - jsonArray.add(msg.getMessageBody()); - } - return jsonArray; - } - - /** - * 可以给指定的stage,替换掉已有的stage - * - * @param oriStage - * @param replaceStage - * @return - */ - protected AbstractStage chooseReplaceStage(AbstractStage oriStage, AbstractStage... replaceStage) { - if (replaceStage == null) { - return oriStage; - } - for (AbstractStage stage : replaceStage) { - if (stage != null && stage.getName().equals(oriStage.getName())) { - return stage; - } - } - return oriStage; - } - - private boolean doMessage(T t, AbstractStage stage, AbstractContext context) { - Object result = null; - result = stage.doMessage(t, context); - if (result == null || !context.isContinue()) { - return false; - } - return true; - } - - public void addStage(AbstractStage stage) { - this.stages.add(stage); - } - - public void setStageLable(AbstractStage stage, String lable) { - stage.setLabel(lable); - } - - public void setStages(List> stages) { - this.stages = stages; - } - - @Override public void destroy() { - if (LOG.isInfoEnabled()) { - LOG.info(getName() + " is destroy, release pipline " + stages.size()); - } - stages.clear(); - } - - public String getName() { - return name; - } - - public void setName(String name) { - this.name = name; - } - - public List> getStages() { - return stages; - } - - public String getMsgSourceName() { - return msgSourceName; - } - - public String getSourceIdentification() { - return sourceIdentification; - } - - public void setSourceIdentification(String sourceIdentification) { - this.sourceIdentification = sourceIdentification; - } - - public Map> getPreFingerprintExecutor() { - return preFingerprintExecutor; - } - - public void setMsgSourceName(String msgSourceName) { - this.msgSourceName = msgSourceName; - } -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/SectionPipeline.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/SectionPipeline.java similarity index 76% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/SectionPipeline.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/SectionPipeline.java index 2ad92e2e..57a2a3d7 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/SectionPipeline.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/SectionPipeline.java @@ -14,25 +14,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.common.topology; +package org.apache.rocketmq.streams.common.topology.model; import java.util.List; import org.apache.rocketmq.streams.common.configurable.IConfigurableIdentification; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; -import org.apache.rocketmq.streams.common.topology.model.AbstractStage; -import org.apache.rocketmq.streams.common.topology.model.Pipeline; /** * A part of the pipeline, starting from a stage and going back */ -public class SectionPipeline implements IStreamOperator>, - IConfigurableIdentification { +public class SectionPipeline implements IStreamOperator>, IConfigurableIdentification { protected ChainPipeline pipeline; protected AbstractStage currentStage; - public SectionPipeline(Pipeline pipeline, AbstractStage currentStage) { + public SectionPipeline(AbstractPipeline pipeline, AbstractStage currentStage) { this((ChainPipeline) pipeline, currentStage); } @@ -41,18 +38,14 @@ public SectionPipeline(ChainPipeline pipeline, AbstractStage currentStage) { this.currentStage = currentStage; } - @Override - public AbstractContext doMessage(IMessage message, AbstractContext context) { + @Override public AbstractContext doMessage(IMessage message, AbstractContext context) { //设置window触发后需要执行的逻辑 if (pipeline.isTopology()) { - pipeline.doNextStages(context, currentStage.getMsgSourceName(), currentStage.getLabel(), currentStage.getNextStageLabels(), currentStage.getOwnerSqlNodeTableName()); + pipeline.doNextStages(context, currentStage.getMsgSourceName(), currentStage.getLabel(), currentStage.getNextStageLabels()); return context; } else { - final int index = chooseWindowStageNextIndex(pipeline); - - pipeline.doMessageFromIndex(message, context, index); - return context; + throw new RuntimeException("expect Topology, but not " + pipeline.getName()); } } @@ -88,18 +81,15 @@ public ChainPipeline getPipeline() { return pipeline; } - @Override - public String getConfigureName() { - return pipeline.getConfigureName(); + @Override public String getName() { + return pipeline.getName(); } - @Override - public String getNameSpace() { + @Override public String getNameSpace() { return pipeline.getNameSpace(); } - @Override - public String getType() { + @Override public String getType() { return pipeline.getType(); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/StreamGraph.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/StreamGraph.java new file mode 100644 index 00000000..c090c935 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/StreamGraph.java @@ -0,0 +1,156 @@ +/* + * 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.topology.model; + +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; +import org.apache.rocketmq.streams.common.configurable.IConfigurable; +import org.apache.rocketmq.streams.common.configurable.IConfigurableQuery; +import org.apache.rocketmq.streams.common.configurable.IFieldProcessor; +import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; +import org.apache.rocketmq.streams.common.configuration.SystemContext; +import org.apache.rocketmq.streams.common.utils.ENVUtil; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; +import org.apache.rocketmq.streams.common.utils.ReflectUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; + +public class StreamGraph extends BasedConfigurable implements IConfigurableQuery { + + protected List configurables; + protected ChainPipeline mainPipeline; + protected List subStreamGraphs; + + protected Map> type2Configurables = new HashMap<>(); + protected Map name2Configurables = new HashMap<>(); + + public StreamGraph(ChainPipeline mainPipeline, List configurables) { + this.mainPipeline = mainPipeline; + this.configurables = configurables; + this.setNameSpace(this.mainPipeline.getNameSpace()); + this.setName(this.mainPipeline.getName()); + for (IConfigurable configurable : configurables) { + setENVVar(configurable);//替换环境变量 + configurable.init();//初始化对象 + registConfigurable(configurable);//实现IConfigurableQuery接口的数据结构 + + } +// executeAfterConfigurableRefreshListener(configurables);//实现IAfterConfigurableRefreshListener接口的回调 + + } + + @Override public List queryConfigurableByType(String type) { + List result = new ArrayList<>(); + for (IConfigurable configurable : queryConfigurable(type)) { + result.add((T) configurable); + } + return result; + } + + @Override public T queryConfigurable(String configurableType, String name) { + return (T) name2Configurables.get(MapKeyUtil.createKey(configurableType, name)); + } + + @Override public List queryConfigurable(String type) { + return type2Configurables.get(type); + } + + @Override public Map queryConfigurableMapByType(String type) { + Map result = new HashMap<>(); + for (IConfigurable configurable : queryConfigurable(type)) { + result.put(configurable.getName(), (T) configurable); + } + return result; + } + + @Override public Collection findAll() { + return this.configurables; + } + + protected void setENVVar(IConfigurable configurable) { + ReflectUtil.scanConfiguableFields(configurable, new IFieldProcessor() { + @Override public void doProcess(Object o, Field field) { + ENVDependence dependence = field.getAnnotation(ENVDependence.class); + if (dependence == null) { + return; + } + String fieldValue = ReflectUtil.getBeanFieldValue(o, field.getName()); + if (fieldValue == null) { + return; + } + String value = getENVVar(fieldValue); + if (StringUtil.isNotEmpty(value)) { + ReflectUtil.setBeanFieldValue(o, field.getName(), value); + } + } + }); + } + + private void registConfigurable(IConfigurable configurable) { + name2Configurables.put(MapKeyUtil.createKey(configurable.getType(), configurable.getName()), configurable); + List configurableList = type2Configurables.get(configurable.getType()); + if (configurableList == null) { + configurableList = new ArrayList<>(); + type2Configurables.put(configurable.getType(), configurableList); + } + configurableList.add(configurable); + + } + + protected String getENVVar(String fieldValue) { + if (StringUtil.isEmpty(fieldValue)) { + return null; + } + String value = SystemContext.getStringParameter(fieldValue); + if (StringUtil.isNotEmpty(value)) { + return value; + } + return ENVUtil.getENVParameter(fieldValue); + } + + @Override public void destroy() { + super.destroy(); + if (subStreamGraphs != null) { + for (StreamGraph streamGraph : subStreamGraphs) { + streamGraph.destroy(); + } + } + for (IConfigurable configurable : this.configurables) { + configurable.destroy(); + } + } + + public List getConfigurables() { + return configurables; + } + + public ChainPipeline getMainPipeline() { + return mainPipeline; + } + + public List getSubStreamGraphs() { + return subStreamGraphs; + } + + public void setSubStreamGraphs(List subStreamGraphs) { + this.subStreamGraphs = subStreamGraphs; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/Union.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/Union.java index 53079e07..f5157f2e 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/Union.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/Union.java @@ -20,13 +20,12 @@ import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; -import org.apache.rocketmq.streams.common.topology.ChainStage; -import org.apache.rocketmq.streams.common.topology.builder.IStageBuilder; +import org.apache.rocketmq.streams.common.topology.IStageBuilder; import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; import org.apache.rocketmq.streams.common.topology.stages.udf.UDFUnionChainStage; public class Union extends BasedConfigurable implements IStreamOperator>, - IStageBuilder { + IStageBuilder> { public static String TYPE = "union"; protected transient IStreamOperator> receiver; @@ -49,7 +48,7 @@ public AbstractContext doMessage(IMessage message, AbstractContext con } @Override - public ChainStage createStageChain(PipelineBuilder pipelineBuilder) { + public AbstractChainStage createStageChain(PipelineBuilder pipelineBuilder) { pipelineBuilder.addConfigurables(this); UDFUnionChainStage chainStage = new UDFUnionChainStage(); chainStage.setUnion(this); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/shuffle/ShuffleMQCreator.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/shuffle/ShuffleMQCreator.java deleted file mode 100644 index 68bd088d..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/shuffle/ShuffleMQCreator.java +++ /dev/null @@ -1,406 +0,0 @@ -/* - * 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.topology.shuffle; - -import java.lang.reflect.Field; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; - -import java.util.concurrent.ConcurrentHashMap; -import org.apache.rocketmq.streams.common.cache.softreference.ICache; -import org.apache.rocketmq.streams.common.cache.softreference.impl.SoftReferenceCache; -import org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder; -import org.apache.rocketmq.streams.common.channel.builder.IShuffleChannelBuilder; -import org.apache.rocketmq.streams.common.channel.impl.memory.MemoryCache; -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.AbstractSupportShuffleSink; -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.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; -import org.apache.rocketmq.streams.common.utils.MapKeyUtil; -import org.apache.rocketmq.streams.common.utils.ReflectUtil; -import org.apache.rocketmq.streams.common.utils.ServiceLoadUtil; -import org.apache.rocketmq.streams.common.utils.StringUtil; - -/** - * create shuffle source and producer - */ -public class ShuffleMQCreator { - private static ICache cache=new SoftReferenceCache<>(); - public static String SHUFFLE_BUILTIN="built-in"; - /** - * input parameters - */ - protected String namespace; - protected String pipelineName; - protected String shuffleOwnerName; - protected int splitCount; - protected transient ISource pipelineSource; - - protected transient List> queueList;//shuffle mq's split list - protected transient Map> queueMap = new ConcurrentHashMap<>(); - - - /** - * need create - */ - protected transient AbstractSupportShuffleSink producer; - protected ISource consumer; - - /** - * - */ - protected volatile boolean hasCreateShuffleChannel = false; - - private ShuffleMQCreator(ISource source,String namespace,String pipelineName,String shuffleOwnerName, int splitCount){ - this.pipelineSource=source; - this.namespace=namespace; - this.pipelineName=pipelineName; - this.shuffleOwnerName=shuffleOwnerName; - this.splitCount=splitCount; - } - - public static ShuffleMQCreator createShuffleCreator(ISource source,String namespace,String pipelineName,String shuffleOwnerName, int splitCount){ - String key= MapKeyUtil.createKey(namespace,pipelineName,shuffleOwnerName); - ShuffleMQCreator shuffleMQCreator=cache.get(key); - if(shuffleMQCreator!=null){ - return shuffleMQCreator; - } - shuffleMQCreator=new ShuffleMQCreator(source,namespace,pipelineName,shuffleOwnerName,splitCount); - shuffleMQCreator.init(); - cache.put(key,shuffleMQCreator); - return shuffleMQCreator; - } - - - public static ISource getSource(String namespace,String pipelineName,String shuffleOwnerName ){ - String key= MapKeyUtil.createKey(namespace,pipelineName,shuffleOwnerName); - ShuffleMQCreator shuffleMQCreator=cache.get(key); - if(shuffleMQCreator==null){ - throw new RuntimeException("expect get ShuffleMQCreator in cache, but not. Check whether it is created in the shuffleproducerchainstage class "); - } - return shuffleMQCreator.getConsumer(); - } - - - - - - /** - * init shuffle channel - */ - private void init() { - String channelType =this.pipelineSource.getClass().getSimpleName(); - if(SHUFFLE_BUILTIN.equals(channelType)){ - ISource source=createSourceByProperty(namespace, pipelineName); - source.setNameSpace(namespace); - autoCreateShuffleChannel(source); - }else{ - consumer= createSourceByProperty(namespace, pipelineName); - producer = createSinkByProperty(namespace, pipelineName); - if (consumer == null || producer== null) { - autoCreateShuffleChannel(pipelineSource); - } - if (consumer == null) { - return; - } - if (consumer instanceof AbstractSource) { - ((AbstractSource) consumer).setJsonData(true); - } - if(producer!=null){ - producer.init(); - } - if (producer != null && (queueList == null || queueList.size() == 0)) { - this.producer.init(); - queueList = producer.getSplitList(); - Map> tmp = new ConcurrentHashMap<>(); - for (ISplit queue : queueList) { - tmp.put(queue.getQueueId(), queue); - } - this.queueMap = tmp; - } - } - - - } - /** - * choose shuffle split - * @param key - * @return - */ - public int hash(Object key) { - int mValue = queueList.size(); - int h = 0; - if (key != null) { - h = key.hashCode() ^ (h >>> 16); - if (h < 0) { - h = -h; - } - } - return h % mValue; - } - - - /** - * 创建channel,根据配置文件配置channel的连接信息 - * - * @return - */ - protected ISource createSourceByProperty(String namespace, String name) { - IChannelBuilder builder = createBuilder(); - if (builder == null) { - return null; - } - Properties properties = createChannelProperties(namespace,this.shuffleOwnerName); - ISource source = builder.createSource(namespace, name, properties, null); - if (source instanceof MemorySource) { - MemorySource memorySource = (MemorySource) source; - MemoryCache memoryCache = new MemoryCache(); - memorySource.setMemoryCache(memoryCache); - memoryCache.init(); - } - source.init(); - return source; - } - - - /** - * 创建channel,根据配置文件配置channel的连接信息 - * - * @return - */ - protected AbstractSupportShuffleSink createSinkByProperty(String namespace, String name) { - - IChannelBuilder builder = createBuilder(); - if (builder == null) { - return null; - } - Properties properties = createChannelProperties(namespace,this.shuffleOwnerName); - - ISink sink = builder.createSink(namespace, name, properties, null); - if (!(sink instanceof AbstractSupportShuffleSink)) { - throw new RuntimeException("can not support shuffle " + sink.toJson()); - } - if (sink instanceof MemorySink) { - MemorySink memorySink = (MemorySink) sink; - if (!(this.consumer instanceof MemorySource)) { - throw new RuntimeException("shuffle cosumer need memory, real is " + this.consumer); - } - MemorySource memorySource = (MemorySource) this.consumer; - MemoryCache memoryCache = memorySource.getMemoryCache(); - memorySink.setMemoryCache(memoryCache); - } - - sink.init(); - return (AbstractSupportShuffleSink) sink; - } - - - /** - * 如果用户未配置shuffle channel,根据pipeline数据源动态创建 - * - */ - public void autoCreateShuffleChannel(ISource pipelineSource) { - if (!hasCreateShuffleChannel) { - synchronized (this) { - if (!hasCreateShuffleChannel) { - String channelType =this.pipelineSource.getClass().getSimpleName(); - IChannelBuilder builder = ServiceLoadUtil.loadService(IChannelBuilder.class,channelType); - if (builder == null) { - throw new RuntimeException("can not create shuffle channel, not find channel builder by the type" + channelType); - } - if (!(builder instanceof IShuffleChannelBuilder)) { - throw new RuntimeException("can not create shuffle channel, builder not impl IShuffleChannelBuilder " + channelType); - } - IShuffleChannelBuilder shuffleChannelBuilder = (IShuffleChannelBuilder) builder; - ISink sink = shuffleChannelBuilder.createBySource(pipelineSource); - sink.init(); - if (!(sink instanceof MemoryChannel) && !(sink instanceof AbstractSupportShuffleSink)) { - throw new RuntimeException("can not create shuffle channel, sink not extends AbstractSupportShuffleSink " + channelType); - } - ISource source = null; - if (sink instanceof MemoryChannel) { - MemoryCache memoryCache = new MemoryCache(); - memoryCache.setNameSpace(namespace); - memoryCache.setConfigureName(shuffleOwnerName); - - sink = new MemorySink(); - source = new MemorySource(); - ((MemorySink) sink).setMemoryCache(memoryCache); - ((MemorySource) source).setMemoryCache(memoryCache); - memoryCache.init(); - } - - Properties properties = new Properties(); - - properties.put("groupName", shuffleOwnerName); - properties.put("tags", shuffleOwnerName); - - AbstractSupportShuffleSink shuffleSink = (AbstractSupportShuffleSink) sink; - shuffleSink.setSplitNum(getShuffleSplitCount(shuffleSink)); - shuffleSink.setNameSpace(namespace); - shuffleSink.setConfigureName(shuffleOwnerName); - String topicFiledName = shuffleSink.getShuffleTopicFieldName(); - String shuffleTopic = null; - //内存模式,是无topic的 - if (StringUtil.isNotEmpty(topicFiledName)) { - String topic = ReflectUtil.getDeclaredField(shuffleSink, topicFiledName); - shuffleTopic = createShuffleTopic(topic); - ReflectUtil.setBeanFieldValue(shuffleSink, topicFiledName, shuffleTopic); - } - - //修改和window有关的属性,如groupname,tags - List fields = ReflectUtil.getDeclaredFieldsContainsParentClass(sink.getClass()); - for (Field field : fields) { - String fieldName = field.getName(); - String value = properties.getProperty(fieldName); - if (StringUtil.isNotEmpty(value)) { - ReflectUtil.setBeanFieldValue(sink, fieldName, value); - } - } - shuffleSink.setHasInit(false); - shuffleSink.init();//在这里完成shuffle channel的创建 - if (source == null) { - source = shuffleChannelBuilder.copy(pipelineSource); - } - - //修改和window有关的属性,如groupname,tags - - fields = ReflectUtil.getDeclaredFieldsContainsParentClass(source.getClass()); - for (Field field : fields) { - String fieldName = field.getName(); - String value = properties.getProperty(fieldName); - if (StringUtil.isNotEmpty(value)) { - ReflectUtil.setBeanFieldValue(source, fieldName, value); - } - } - - source.setNameSpace(sink.getNameSpace()); - source.setConfigureName(sink.getConfigureName()); - //修改主题 - if (shuffleTopic != null && topicFiledName != null) { - ReflectUtil.setBeanFieldValue(source, topicFiledName, shuffleTopic); - } - if (source instanceof AbstractSource) { - AbstractSource abstractSource = (AbstractSource) source; - abstractSource.setHasInit(false); - } - source.init(); - - this.producer = shuffleSink; - this.consumer = source; - } - } - } - } - /** - * create channel builder - * - * @return - */ - protected IChannelBuilder createBuilder() { - String type = ComponentCreator.getProperties().getProperty(ConfigureFileKey.WINDOW_SHUFFLE_CHANNEL_TYPE); - if (StringUtil.isEmpty(type)) { - return null; - } - return ServiceLoadUtil.loadService(IChannelBuilder.class,type); - } - - - /** - * 根据属性文件配置 - * - * @return 资源文件 - */ - protected Properties createChannelProperties(String namespace,String shuffleOwnerName) { - Properties properties = new Properties(); - for (Map.Entry entry : ComponentCreator.getProperties().entrySet()) { - String key = (String) entry.getKey(); - String value = (String) entry.getValue(); - if (key.startsWith(ConfigureFileKey.WINDOW_SHUFFLE_CHANNEL_PROPERTY_PREFIX)) { - String channelKey = key.replace(ConfigureFileKey.WINDOW_SHUFFLE_CHANNEL_PROPERTY_PREFIX, ""); - if (channelKey.startsWith(namespace)) {//支持基于namespace 做shuffle window共享 - channelKey = channelKey.replace(namespace, ""); - properties.put(channelKey, value); - } else { - if (!properties.containsKey(channelKey)) { - properties.put(channelKey, value); - } - } - } - } - Set multiPropertySet = new HashSet<>(); - String dynamicProperty = properties.getProperty("dynamic.property"); - if (dynamicProperty != null) { - - String dynamicPropertyValue = shuffleOwnerName; - String[] mutilPropertys = dynamicProperty.split(","); - - for (String properyKey : mutilPropertys) { - properties.put(properyKey, dynamicPropertyValue); - multiPropertySet.add(properyKey); - } - - } - String groupName = "groupName"; - if (!multiPropertySet.contains(groupName)) { - properties.put(groupName, shuffleOwnerName); - } - if (!multiPropertySet.contains("tags")) { - properties.put("tags", shuffleOwnerName); - } - return properties; - } - - protected int getShuffleSplitCount(AbstractSupportShuffleSink shuffleSink) { - if(this.splitCount>0){ - return splitCount; - } - int splitNum = shuffleSink.getSplitNum(); - return splitNum > 0 ? splitNum : 32; - } - - public List> getQueueList() { - return queueList; - } - - /** - * 1个pipeline一个 shuffle topic - * - * @param topic - * @return - */ - protected String createShuffleTopic(String topic) { - return "shuffle_" + topic + "_" + namespace.replaceAll("\\.", "_") + "_" + pipelineName.replaceAll("\\.", "_").replaceAll(";", "_"); - } - - public AbstractSupportShuffleSink getProducer() { - return producer; - } - - public ISource getConsumer() { - return consumer; - } -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/AbstractStatelessChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/AbstractStatelessChainStage.java index 5fd17a5a..870693ae 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/AbstractStatelessChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/AbstractStatelessChainStage.java @@ -16,14 +16,15 @@ */ package org.apache.rocketmq.streams.common.topology.stages; +import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; 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; -import org.apache.rocketmq.streams.common.topology.ChainStage; +import org.apache.rocketmq.streams.common.topology.model.AbstractChainStage; -public abstract class AbstractStatelessChainStage extends ChainStage { +public abstract class AbstractStatelessChainStage extends AbstractChainStage { @Override public void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage) { checkPointMessage.replyAnyone(); @@ -38,4 +39,17 @@ public void addNewSplit(IMessage message, AbstractContext context, NewSplitMessa public void removeSplit(IMessage message, AbstractContext context, RemoveSplitMessage removeSplitMessage) { } + + @Override public void startJob() { + + } + + @Override public void stopJob() { + + } + + @Override + public void batchMessageFinish(IMessage message, AbstractContext context, BatchFinishMessage checkPointMessage) { + + } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/AbstractWindowStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/AbstractWindowStage.java deleted file mode 100644 index 4385855c..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/AbstractWindowStage.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * 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.topology.stages; - -import java.util.HashSet; -import java.util.Set; -import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; -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.checkpoint.CheckPointState; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; -import org.apache.rocketmq.streams.common.context.AbstractContext; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; -import org.apache.rocketmq.streams.common.topology.ChainStage; -import org.apache.rocketmq.streams.common.topology.model.IWindow; - -public abstract class AbstractWindowStage extends ChainStage implements - IAfterConfigurableRefreshListener { - protected String windowName; - protected transient IWindow window; - - @Override - public void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage) { - if (window.getWindowCache() == null) {//over window windowcache is null - return; - } - if (message.getHeader().isNeedFlush()) { - if (window.getWindowCache() != null && message.getHeader().getCheckpointQueueIds() != null && message.getHeader().getCheckpointQueueIds().size() > 0) { - window.getWindowCache().checkpoint(message.getHeader().getCheckpointQueueIds()); - } else { - if (window.getWindowCache() != null) { - Set queueIds = new HashSet<>(); - queueIds.add(message.getHeader().getQueueId()); - window.getWindowCache().checkpoint(queueIds); - } - - } - - } - CheckPointState checkPointState = new CheckPointState(); - checkPointState.setQueueIdAndOffset(window.getWindowCache().getFinishedQueueIdAndOffsets(checkPointMessage)); - checkPointMessage.reply(checkPointState); - } - - @Override - public void addNewSplit(IMessage message, AbstractContext context, NewSplitMessage newSplitMessage) { - - } - - @Override - public void removeSplit(IMessage message, AbstractContext context, RemoveSplitMessage removeSplitMessage) { - - } - - @Override - public void batchMessageFinish(IMessage message, AbstractContext context, BatchFinishMessage checkPointMessage) { - window.getWindowCache().finishBatchMsg(checkPointMessage); - } - - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - window = configurableService.queryConfigurable(IWindow.TYPE, windowName); - if (((ChainPipeline)getPipeline()).isTopology()) { - window.setFireReceiver(getReceiverAfterCurrentNode()); - } - if (Boolean.TRUE.equals(Boolean.valueOf(ComponentCreator.getProperties().getProperty(ConfigureFileKey.DIPPER_RUNNING_STATUS, ConfigureFileKey.DIPPER_RUNNING_STATUS_DEFAULT)))) { - window.windowInit(); - } - } - - public String getWindowName() { - return windowName; - } - - public void setWindowName(String windowName) { - this.windowName = windowName; - } - -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/DimChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/DimChainStage.java new file mode 100644 index 00000000..a6400e85 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/DimChainStage.java @@ -0,0 +1,180 @@ +/* + * 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.topology.stages; + +import com.alibaba.fastjson.JSONArray; +import com.alibaba.fastjson.JSONObject; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.rocketmq.streams.common.context.AbstractContext; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.interfaces.IDim; +import org.apache.rocketmq.streams.common.utils.StringUtil; + +public class DimChainStage extends ScriptChainStage { + protected IDim dim; + protected String expressionStr; + protected String alias; + protected String dimScript; + protected String[] fieldNames; + protected boolean isInnerJoin; + protected boolean isLeftJoin; + protected String splitFieldName; + + @Override public void startJob() { + dim.startLoadDimData(); + } + + @Override protected IMessage handleMessage(IMessage message, AbstractContext context) { + JSONArray rows = getRows(message.getMessageBody(), expressionStr, alias, dimScript, fieldNames); + if (isInnerJoin) { + if (rows == null || rows.size() == 0) { + context.breakExecute(); + return null; + } + } else if (isLeftJoin) { + if (rows == null || rows.size() == 0) { + return message; + } + } + message.getMessageBody().put(splitFieldName, rows); + + return super.handleMessage(message, context); + } + + /** + * 根据表达式,从namelist中获取符合条件的数据 + * + * @param expressionStr (varname,functionName,value)&(varname,functionName,value) + * @param fieldNames 需要返回的字段名 + * @return + */ + protected JSONArray getRows(JSONObject msg, + String expressionStr, String alias, String script, String... fieldNames) { + + if (StringUtil.isEmpty(script)) { + script = null; + } + List> rows = this.dim.matchExpression(expressionStr, msg, true, script); + if (rows == null || rows.size() == 0) { + return null; + } + JSONArray jsonArray = new JSONArray(); + for (Map row : rows) { + JSONObject jsonObject = new JSONObject(); + if (fieldNames == null || fieldNames.length == 0) { + if (StringUtil.isEmpty(alias)) { + jsonObject.putAll(row); + } else { + Iterator> it = row.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry entry = it.next(); + String fieldName = entry.getKey(); + if (alias != null) { + fieldName = alias + "." + fieldName; + } + jsonObject.put(fieldName, entry.getValue()); + } + } + + } else { + for (String fieldName : fieldNames) { + String tmp = fieldName; + if (alias != null) { + fieldName = alias + "." + fieldName; + } + jsonObject.put(fieldName, row.get(tmp)); + } + } + jsonArray.add(jsonObject); + } + return jsonArray; + } + + @Override public void stopJob() { + dim.destroy(); + } + + public IDim getDim() { + return dim; + } + + public void setDim(IDim dim) { + this.dim = dim; + } + + public String getExpressionStr() { + return expressionStr; + } + + public void setExpressionStr(String expressionStr) { + this.expressionStr = expressionStr; + } + + public String getAlias() { + return alias; + } + + public void setAlias(String alias) { + this.alias = alias; + } + + public String getDimScript() { + return dimScript; + } + + public void setDimScript(String dimScript) { + this.dimScript = dimScript; + } + + public String[] getFieldNames() { + return fieldNames; + } + + public void setFieldNames(String[] fieldNames) { + this.fieldNames = fieldNames; + } + + public boolean isInnerJoin() { + return isInnerJoin; + } + + public void setInnerJoin(boolean innerJoin) { + isInnerJoin = innerJoin; + } + + public boolean isLeftJoin() { + return isLeftJoin; + } + + public void setLeftJoin(boolean leftJoin) { + isLeftJoin = leftJoin; + } + + public String getSplitFieldName() { + return splitFieldName; + } + + public void setSplitFieldName(String splitFieldName) { + this.splitFieldName = splitFieldName; + } + + @Override public boolean isAsyncNode() { + return false; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/EmptyChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/EmptyChainStage.java index 13b24ed2..d78da007 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/EmptyChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/EmptyChainStage.java @@ -18,28 +18,16 @@ import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.topology.model.IStageHandle; public class EmptyChainStage extends AbstractStatelessChainStage { - protected transient IStageHandle handle = new IStageHandle() { - @Override public String getName() { - return getClass().getName(); - } - @Override - protected IMessage doProcess(IMessage message, AbstractContext context) { - return proccessMessage(message,context); - } - }; - - protected IMessage proccessMessage(IMessage message, AbstractContext context) { + @Override + protected IMessage handleMessage(IMessage message, AbstractContext context) { return message; } + @Override public boolean isAsyncNode() { return false; } - @Override protected IStageHandle selectHandle(T t, AbstractContext context) { - return handle ; - } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/FilterChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/FilterChainStage.java index 222c4b68..b03aedc5 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/FilterChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/FilterChainStage.java @@ -16,53 +16,19 @@ */ package org.apache.rocketmq.streams.common.topology.stages; -import com.alibaba.fastjson.JSONObject; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.concurrent.atomic.AtomicInteger; -import javax.swing.ImageIcon; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.component.IComponent; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.configurable.annotation.ConfigurableReference; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.interfaces.IFilterService; import org.apache.rocketmq.streams.common.monitor.ConsoleMonitorManager; -import org.apache.rocketmq.streams.common.monitor.TopologyFilterMonitor; import org.apache.rocketmq.streams.common.optimization.fingerprint.PreFingerprint; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; import org.apache.rocketmq.streams.common.topology.metric.NotFireReason; import org.apache.rocketmq.streams.common.topology.model.AbstractRule; -import org.apache.rocketmq.streams.common.topology.model.AbstractStage; -import org.apache.rocketmq.streams.common.topology.model.IStageHandle; -import org.apache.rocketmq.streams.common.utils.MapKeyUtil; -import org.apache.rocketmq.streams.common.utils.PrintUtil; -import org.apache.rocketmq.streams.common.utils.ReflectUtil; -import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.common.utils.TraceUtil; -public class FilterChainStage extends AbstractStatelessChainStage implements IAfterConfigurableRefreshListener { - protected transient AtomicInteger count = new AtomicInteger(0); - protected transient Map map = new HashMap<>(); - private List names; - /** - * 通过名字匹配模式,加载一批规则,避免大批量输入规则名称 - */ - protected String nameRegex; - private transient List rules; - private transient Map ruleName2JsonObject = new HashMap<>(); - public static transient Class componentClass = ReflectUtil.forClass("org.apache.rocketmq.streams.filter.FilterComponent"); - protected boolean openHyperscan = false; - protected static transient IComponent component; - protected transient FilterChainStage SELF; +public class FilterChainStage extends AbstractStatelessChainStage { + @ConfigurableReference private R rule; public FilterChainStage() { - SELF=this; setEntityName("filter"); } @@ -72,158 +38,48 @@ public boolean isAsyncNode() { return false; } - protected transient IStageHandle handle = new IStageHandle() { - @Override - protected IMessage doProcess(IMessage message, AbstractContext context) { - boolean isTrace = TraceUtil.hit(message.getHeader().getTraceId()); - - if (component == null) { - component = ComponentCreator.getComponent(null, componentClass); - } - String fieldValue=message.getHeader().getLogFingerprintValue(); - NotFireReason notFireReason=null; - if(preFingerprint!=null){ - notFireReason=new NotFireReason(SELF,fieldValue); - context.setNotFireReason(notFireReason); - } + @Override + protected IMessage handleMessage(IMessage message, AbstractContext context) { - List fireRules = component.getService().executeRule(message, context, rules); + boolean isFire = rule.doMessage(message, context);//component.getService().executeRule(message, context, rule); + if (!isFire) { + context.breakExecute(); - //not match rules - if (fireRules == null || fireRules.size() == 0) { - context.breakExecute(); - if (preFingerprint != null) { - preFingerprint.addLogFingerprintToSource(message); - } - notFireReason=context.getNotFireReason(); - if(isTrace&¬FireReason!=null){ - traceFailExpression(message,notFireReason); - } + if (preFingerprint != null) { + preFingerprint.addLogFingerprintToSource(message); } - return message; } + return message; + } - @Override - public String getName() { - return FilterChainStage.class.getName(); - } - }; + protected void setNotReasonToContext(AbstractContext context) { - protected void traceFailExpression(IMessage message,NotFireReason notFireReason) { - ConsoleMonitorManager.getInstance().reportOutput(FilterChainStage.this, message, ConsoleMonitorManager.MSG_FILTERED, notFireReason.toString()); - TraceUtil.debug(message.getHeader().getTraceId(), "break rule", notFireReason.toString()); } - - @Override - protected IStageHandle selectHandle(T t, AbstractContext context) { - return handle; + protected void traceFailExpression(IMessage message, NotFireReason notFireReason) { + ConsoleMonitorManager.getInstance().reportOutput(FilterChainStage.this, message, ConsoleMonitorManager.MSG_FILTERED, notFireReason.toString()); + TraceUtil.debug(message.getHeader().getTraceId(), "break rule", notFireReason.toString()); } - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - if (names == null || names.size() == 0) { - List ruleList = configurableService.queryConfigurableByType(AbstractRule.TYPE); - if (ruleList != null && ruleList.size() > 0) { - // rules= (R[]) Array.newInstance(ruleList.get(0).getClass(), ruleList.size()); - List matchedRules = new ArrayList<>(); - for (int i = 0; i < ruleList.size(); i++) { - if (StringUtil.isNotEmpty(nameRegex)) { - if (!StringUtil.matchRegex(ruleList.get(i).getConfigureName(), nameRegex)) { - continue; - } - } - matchedRules.add(ruleList.get(i)); - //rules[i]=(R)ruleList.get(i); - ruleName2JsonObject.put(ruleList.get(i).getConfigureName(), ruleList.get(i).toOutputJson()); - } - rules = new ArrayList<>(); - for (int i = 0; i < rules.size(); i++) { - rules.add((R) matchedRules.get(i)); - } - } - } else { - if (names != null && names.size() > 0) { - rules = new ArrayList<>(); - } - int i = 0; - ChainPipeline pipline = (ChainPipeline) getPipeline(); - String filterName = getLabel(); - for (String name : names) { - AbstractRule rule = configurableService.queryConfigurable(AbstractRule.TYPE, name); - if(rule==null){ - throw new RuntimeException("the rule expect exist, but not. the rule name is "+name); - } - rules.add((R) rule); - if (!this.isOpenHyperscan()) { - /** - * open hyperscan to optimaztion mutil regex - */ - String key = MapKeyUtil.createKeyBySign(".", pipline.getNameSpace(), pipline.getConfigureName(), filterName, "open_hyperscan"); - String openHyperscan = ComponentCreator.getProperties().getProperty(key); - if (openHyperscan != null && Boolean.valueOf(openHyperscan)) { - this.openHyperscan = true; - } - } - if (isOpenHyperscan()) { - rule.setSupportHyperscan(true); - } - ruleName2JsonObject.put(rules.get(i).getConfigureName(), rules.get(i).toOutputJson()); - i++; - } - } + @Override public void startJob() { + super.startJob(); if (this.preFingerprint == null) { this.preFingerprint = loadLogFinger(); } + } + public R getRule() { + return rule; } - public void setRule(AbstractRule... rules) { - if (rules == null || rules.length == 0) { + public void setRule(R rule) { + if (rule == null) { return; } - this.rules = new ArrayList<>(); - if (names == null) { - names = new ArrayList<>(); - } - int i = 0; - for (AbstractRule rule : rules) { - this.rules.add((R) rules[i]); - names.add(rules[i].getConfigureName()); - ruleName2JsonObject.put(rules[i].getConfigureName(), rules[i].toOutputJson()); - i++; - } - setNameSpace(rules[0].getNameSpace()); - - } - - public List getNames() { - return names; - } - - public void setNames(List names) { - this.names = names; - } - - public String getNameRegex() { - return nameRegex; - } - - public void setNameRegex(String nameRegex) { - this.nameRegex = nameRegex; - } - - public List getRules() { - return rules; - } - - public boolean isOpenHyperscan() { - return openHyperscan; - } + this.rule = rule; + setNameSpace(rule.getNameSpace()); - public void setOpenHyperscan(boolean openHyperscan) { - this.openHyperscan = openHyperscan; } @Override diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/JoinChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/JoinChainStage.java deleted file mode 100644 index 9cb9df35..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/JoinChainStage.java +++ /dev/null @@ -1,161 +0,0 @@ -/* - * 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.topology.stages; - -import org.apache.rocketmq.streams.common.configurable.IConfigurable; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.context.AbstractContext; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.context.MessageHeader; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; -import org.apache.rocketmq.streams.common.topology.model.IStageHandle; -import org.apache.rocketmq.streams.common.topology.model.IWindow; -import org.apache.rocketmq.streams.common.topology.model.Pipeline; - -/** - * 新的解析已经废弃,主要兼容老的规则数据 - */ -@Deprecated -public class JoinChainStage extends AbstractWindowStage { - - protected String leftPipelineName; - protected String rightPipelineName; - protected String rightDependentTableName; - - protected transient ChainPipeline leftPipeline; - protected transient ChainPipeline rightPipeline; - - protected transient IStageHandle handle = new IStageHandle() { - - @Override - protected IMessage doProcess(IMessage message, AbstractContext context) { - String lable = message.getHeader().getMsgRouteFromLable(); - String joinFlag = null; - if (lable != null) { - if (lable.equals(rightDependentTableName)) { - joinFlag = MessageHeader.JOIN_RIGHT; - } else { - joinFlag = MessageHeader.JOIN_LEFT; - } - message.getHeader().setMsgRouteFromLable(joinFlag); - } else { - throw new RuntimeException("can not dipatch message, need route label " + toJson()); - } - window.setFireReceiver(getReceiverAfterCurrentNode()); - if (MessageHeader.JOIN_LEFT.equals(joinFlag)) { - leftPipeline.doMessage(message, context); - } else { - rightPipeline.doMessage(message, context); - } - //if(!MessageGloableTrace.existFinshBranch(message)){ - // context.setBreak(true); - //} - context.breakExecute(); - return message; - } - - @Override - public String getName() { - return JoinChainStage.class.getName(); - } - - }; - - public JoinChainStage() { - super.entityName = "join"; - } - - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - super.doProcessAfterRefreshConfigurable(configurableService); - leftPipeline = configurableService.queryConfigurable(Pipeline.TYPE, leftPipelineName); - rightPipeline = configurableService.queryConfigurable(Pipeline.TYPE, rightPipelineName); - } - - @Override - protected IStageHandle selectHandle(T t, AbstractContext context) { - return handle; - } - - public IWindow getWindow() { - return window; - } - - public void setWindow(IWindow window) { - this.window = window; - if (window instanceof IConfigurable) { - setWindowName(window.getConfigureName()); - setLabel(window.getConfigureName()); - } - } - - @Override - public boolean isAsyncNode() { - return true; - } - - @Override - public String getEntityName() { - return super.entityName; - } - - public String getLeftPipelineName() { - return leftPipelineName; - } - - public void setLeftPipelineName(String leftPipelineName) { - this.leftPipelineName = leftPipelineName; - } - - public String getRightPipelineName() { - return rightPipelineName; - } - - public void setRightPipelineName(String rightPipelineName) { - this.rightPipelineName = rightPipelineName; - } - - public String getRightDependentTableName() { - return rightDependentTableName; - } - - public void setRightDependentTableName(String rightDependentTableName) { - this.rightDependentTableName = rightDependentTableName; - } - - public ChainPipeline getLeftPipeline() { - return leftPipeline; - } - - public void setLeftPipeline(ChainPipeline leftPipeline) { - if (leftPipeline != null) { - this.leftPipelineName = leftPipeline.getConfigureName(); - } - this.leftPipeline = leftPipeline; - } - - public ChainPipeline getRightPipeline() { - return rightPipeline; - } - - public void setRightPipeline(ChainPipeline rightPipeline) { - if (rightPipeline != null) { - this.rightPipelineName = rightPipeline.getConfigureName(); - } - this.rightPipeline = rightPipeline; - } -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/JoinStartChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/JoinStartChainStage.java index c7422ee8..798fc3fb 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/JoinStartChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/JoinStartChainStage.java @@ -16,23 +16,26 @@ */ package org.apache.rocketmq.streams.common.topology.stages; +import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; +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; -import org.apache.rocketmq.streams.common.context.MessageHeader; -import org.apache.rocketmq.streams.common.topology.model.IStageHandle; public class JoinStartChainStage extends EmptyChainStage { protected String rightDependentTableName; - protected String leftLableName; - protected String rightLableName; + protected String leftLabelName; + protected String rightLabelName; + @Override - protected IMessage proccessMessage(IMessage message, AbstractContext context) { + protected IMessage handleMessage(IMessage message, AbstractContext context) { String lable = message.getHeader().getMsgRouteFromLable(); if (lable != null) { if (lable.equals(rightDependentTableName)) { - message.getHeader().addRouteLabel(rightLableName); + message.getHeader().addRouteLabel(rightLabelName); } else { - message.getHeader().addRouteLabel(leftLableName); + message.getHeader().addRouteLabel(leftLabelName); } } else { @@ -41,6 +44,28 @@ protected IMessage proccessMessage(IMessage message, AbstractContext context) { return message; } + @Override public void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage) { + handleMessage(message, context); + super.checkpoint(message, context, checkPointMessage); + } + + @Override public void addNewSplit(IMessage message, AbstractContext context, NewSplitMessage newSplitMessage) { + handleMessage(message, context); + super.addNewSplit(message, context, newSplitMessage); + } + + @Override + public void removeSplit(IMessage message, AbstractContext context, RemoveSplitMessage removeSplitMessage) { + handleMessage(message, context); + super.removeSplit(message, context, removeSplitMessage); + } + + @Override + public void batchMessageFinish(IMessage message, AbstractContext context, BatchFinishMessage checkPointMessage) { + handleMessage(message, context); + super.batchMessageFinish(message, context, checkPointMessage); + } + public String getRightDependentTableName() { return rightDependentTableName; } @@ -49,19 +74,19 @@ public void setRightDependentTableName(String rightDependentTableName) { this.rightDependentTableName = rightDependentTableName; } - public String getLeftLableName() { - return leftLableName; + public String getLeftLabelName() { + return leftLabelName; } - public void setLeftLableName(String leftLableName) { - this.leftLableName = leftLableName; + public void setLeftLabelName(String leftLabelName) { + this.leftLabelName = leftLabelName; } - public String getRightLableName() { - return rightLableName; + public String getRightLabelName() { + return rightLabelName; } - public void setRightLableName(String rightLableName) { - this.rightLableName = rightLableName; + public void setRightLabelName(String rightLabelName) { + this.rightLabelName = rightLabelName; } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/NewSQLChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/NewSQLChainStage.java index 213fd9aa..99c749f1 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/NewSQLChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/NewSQLChainStage.java @@ -16,62 +16,28 @@ */ package org.apache.rocketmq.streams.common.topology.stages; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurable; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.configurable.annotation.ConfigurableReference; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; -import org.apache.rocketmq.streams.common.topology.model.IStageHandle; -public class NewSQLChainStage extends AbstractStatelessChainStage implements IAfterConfigurableRefreshListener { - protected String sqlMessageProcessorName; - protected transient IStreamOperator messageProcessor; +public class NewSQLChainStage extends AbstractStatelessChainStage { + @ConfigurableReference protected IStreamOperator messageProcessor; public NewSQLChainStage() { setEntityName("SQL"); } - protected transient IStageHandle handle = new IStageHandle() { - @Override - protected IMessage doProcess(IMessage message, AbstractContext context) { - IMessage msg = messageProcessor.doMessage(message, context); - message.setMessageBody(msg.getMessageBody()); - context.setMessage(message); - return message; - } - - @Override - public String getName() { - return NewSQLChainStage.class.getName(); - } - }; - - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - this.messageProcessor = configurableService.queryConfigurable(IStreamOperator.TYPE, - sqlMessageProcessorName); - } - @Override - protected IStageHandle selectHandle(T t, AbstractContext context) { - return handle; - } - - public String getSqlMessageProcessorName() { - return sqlMessageProcessorName; - } - - public void setSqlMessageProcessorName(String sqlMessageProcessorName) { - this.sqlMessageProcessorName = sqlMessageProcessorName; + protected IMessage handleMessage(IMessage message, AbstractContext context) { + IMessage msg = messageProcessor.doMessage(message, context); + message.setMessageBody(msg.getMessageBody()); + context.setMessage(message); + return message; } public void setMessageProcessor(IStreamOperator messageProcessor) { this.messageProcessor = messageProcessor; - if (IConfigurable.class.isInstance(messageProcessor)) { - IConfigurable configurable = (IConfigurable)messageProcessor; - this.sqlMessageProcessorName = configurable.getConfigureName(); - } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/OpenAPIChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/OpenAPIChainStage.java index f3561a7a..3a861456 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/OpenAPIChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/OpenAPIChainStage.java @@ -17,49 +17,21 @@ package org.apache.rocketmq.streams.common.topology.stages; import org.apache.rocketmq.streams.common.channel.IChannel; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.configurable.annotation.ConfigurableReference; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; -import org.apache.rocketmq.streams.common.topology.model.IStageHandle; -public class OpenAPIChainStage extends AbstractStatelessChainStage implements IAfterConfigurableRefreshListener { - protected String openApiChannelName; - protected transient IChannel channel; - protected transient IStageHandle handle = new IStageHandle() { - @Override - protected IMessage doProcess(IMessage message, AbstractContext context) { - IStreamOperator receiver = (IStreamOperator)channel; - IMessage msg = receiver.doMessage(message, context); - message.setMessageBody(msg.getMessageBody()); - context.setMessage(message); - return message; - } - - @Override - public String getName() { - return OpenAPIChainStage.class.getName(); - } - }; - - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - IChannel channel = configurableService.queryConfigurable(IChannel.TYPE, openApiChannelName); - this.channel = channel; - } +public class OpenAPIChainStage extends AbstractStatelessChainStage { + @ConfigurableReference protected IChannel channel; @Override - protected IStageHandle selectHandle(T t, AbstractContext context) { - return handle; - } - - public String getOpenApiChannelName() { - return openApiChannelName; - } - - public void setOpenApiChannelName(String openApiChannelName) { - this.openApiChannelName = openApiChannelName; + protected IMessage handleMessage(IMessage message, AbstractContext context) { + IStreamOperator receiver = (IStreamOperator) channel; + IMessage msg = receiver.doMessage(message, context); + message.setMessageBody(msg.getMessageBody()); + context.setMessage(message); + return message; } public IChannel getChannel() { @@ -69,7 +41,6 @@ public IChannel getChannel() { public void setChannel(IChannel channel) { this.channel = channel; this.setNameSpace(channel.getNameSpace()); - this.setOpenApiChannelName(channel.getConfigureName()); } @Override diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/OutputChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/OutputChainStage.java index 05f18833..778b9843 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/OutputChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/OutputChainStage.java @@ -16,72 +16,56 @@ */ package org.apache.rocketmq.streams.common.topology.stages; +import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; import java.util.HashSet; +import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; -import org.apache.rocketmq.streams.common.channel.IChannel; import org.apache.rocketmq.streams.common.channel.sink.ISink; 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.checkpoint.CheckPointState; import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.configurable.annotation.ConfigurableReference; import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.metadata.MetaData; -import org.apache.rocketmq.streams.common.topology.ChainStage; -import org.apache.rocketmq.streams.common.topology.model.IStageHandle; +import org.apache.rocketmq.streams.common.topology.model.AbstractChainStage; import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class OutputChainStage extends ChainStage implements IAfterConfigurableRefreshListener { - public static final String OUT_MOCK_SWITCH = "out.mock.switch";//在配置文件中,是否打开mock的开关 - - private String sinkName; - - private String metaDataName; - +public class OutputChainStage extends AbstractChainStage { + private static final Logger LOGGER = LoggerFactory.getLogger(OutputChainStage.class); /** * 可以关闭输出,在测试场景有效果.可以通过配置文件配置 */ @ENVDependence protected String closeOutput; - protected transient ISink sink; + @ConfigurableReference protected ISink sink; - protected transient MetaData metaData; - - /** - * 如果需要把输出关闭或mock到一个其他channel,可以通过配置一个mockchannel。同时通过配置文件 - */ - protected transient ISink mockSink; - protected transient AtomicInteger count = new AtomicInteger(0); - protected transient IStageHandle handle = new IStageHandle() { - @Override - protected IMessage doProcess(IMessage message, AbstractContext context) { - return doSink(message,context); - } + @ConfigurableReference protected MetaData metaData; - @Override - public String getName() { - return OutputChainStage.class.getName(); - } - }; + protected transient List callbackMsgs; + protected transient boolean isCallbackModel;//启动callback模式,输出直接给callback,不再写sink + protected transient AtomicInteger count; - protected IMessage doSink(IMessage message, AbstractContext context){ + @Override protected IMessage handleMessage(IMessage message, AbstractContext context) { if (StringUtil.isNotEmpty(closeOutput)) { String tmp = closeOutput.toLowerCase(); if ("true".equals(tmp) || "false".equals(tmp)) { - Boolean value = Boolean.valueOf(tmp); + boolean value = Boolean.parseBoolean(tmp); if (value) { return message; } } else { - tmp = getENVVar(closeOutput); + tmp = closeOutput; if (StringUtil.isNotEmpty(tmp)) { if ("true".equals(tmp.toLowerCase())) { return message; @@ -91,30 +75,23 @@ protected IMessage doSink(IMessage message, AbstractContext context){ } boolean isWindowTest = ComponentCreator.getPropertyBooleanValue("window.fire.isTest"); if (isWindowTest) { - System.out.println("output count is " + count.incrementAndGet()); + LOGGER.info("[{}] output count is {}", this.getName(), count.incrementAndGet()); } - /** + /* * 主要是输出可能影响线上数据,可以通过配置文件的开关,把所有的输出,都指定到一个其他输出中 */ - if (openMockChannel()) { - if (mockSink != null) { - mockSink.batchAdd(message); - return message; - } - return message; + if (isCallbackModel) { + callbackMsgs.add(message.getMessageBody()); + } else { + sink.batchAdd(message); } - sink.batchAdd(message); return message; } @Override public void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage) { - ISink realSink = null; - if (openMockChannel() && mockSink != null) { - realSink = mockSink; - } else { - realSink = sink; - } + ISink realSink = sink; + if (message.getHeader().isNeedFlush()) { Set queueIds = new HashSet<>(); if (message.getHeader().getCheckpointQueueIds() != null) { @@ -142,97 +119,40 @@ public void removeSplit(IMessage message, AbstractContext context, RemoveSplitMe } - - @Override public void batchMessageFinish(IMessage message, AbstractContext context, BatchFinishMessage checkPointMessage) { - ISink realSink = null; - if (openMockChannel() && mockSink != null) { - realSink = mockSink; + @Override public void batchMessageFinish(IMessage message, AbstractContext context, BatchFinishMessage batchFinishMessage) { + ISink realSink = sink; + if (batchFinishMessage.getMessageFinishCallBack(message) != null && isCallbackModel) { + batchFinishMessage.getMessageFinishCallBack(message).finish(callbackMsgs); } else { - realSink = sink; + realSink.flush(); } - realSink.flush(); - - } - @Override - protected IStageHandle selectHandle(T t, AbstractContext context) { - return handle; - } - - protected IChannel queryChannel() { - return configurableService.queryConfigurable(IChannel.TYPE, sinkName); - } - - public String getSinkName() { - return sinkName; - } - - public void setSinkName(String sinkName) { - this.sinkName = sinkName; - } - public String getMetaDataName() { - return metaDataName; } - public void setMetaDataName(String metaDataName) { - this.metaDataName = metaDataName; + @Override public void startJob() { + this.callbackMsgs = new ArrayList<>(); + this.count = new AtomicInteger(0); } - public IStageHandle getHandle() { - return handle; - } - - public void setHandle(IStageHandle handle) { - this.handle = handle; - } + @Override public void stopJob() { + if (this.sink != null) { + this.sink.destroy(); + } - public void setSink(ISink channel) { - this.sink = channel; - this.setNameSpace(channel.getNameSpace()); - this.setSinkName(channel.getConfigureName()); - this.setLabel(channel.getConfigureName()); } public void setMetaData(MetaData metaData) { this.metaData = metaData; } - public ISink getSink() { + public ISink getSink() { return sink; } - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - sink = configurableService.queryConfigurable(ISink.TYPE, sinkName); - if (sink == null) { - sink = configurableService.queryConfigurable(IChannel.TYPE, sinkName); - } - - metaData = configurableService.queryConfigurable(MetaData.TYPE, metaDataName); - mockSink = getMockChannel(configurableService, sink.getNameSpace()); - } - - protected ISink getMockChannel(IConfigurableService configurableService, String nameSpace) { - String type = ComponentCreator.getProperties().getProperty("out.mock.type"); - if (type == null) { - return null; - } - ISink mockSink = configurableService.queryConfigurable(ISink.TYPE, OUT_MOCK_SWITCH + "_" + type); - if (mockSink == null) { - mockSink = configurableService.queryConfigurable(IChannel.TYPE, OUT_MOCK_SWITCH + "_" + type); - } - return mockSink; - } - - protected boolean openMockChannel() { - String swtich = ComponentCreator.getProperties().getProperty(OUT_MOCK_SWITCH); - if (swtich == null) { - return false; - } - if ("true".equals(swtich)) { - return true; - } - return false; + public void setSink(ISink channel) { + this.sink = channel; + this.setNameSpace(channel.getNameSpace()); + this.setLabel(channel.getName()); } public String getCloseOutput() { @@ -248,4 +168,11 @@ public boolean isAsyncNode() { return false; } + public boolean isCallbackModel() { + return isCallbackModel; + } + + public void setCallbackModel(boolean callbackModel) { + isCallbackModel = callbackModel; + } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/PythonChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/PythonChainStage.java index ba7b44fb..bb49252c 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/PythonChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/PythonChainStage.java @@ -17,49 +17,21 @@ package org.apache.rocketmq.streams.common.topology.stages; import org.apache.rocketmq.streams.common.channel.IChannel; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; -import org.apache.rocketmq.streams.common.topology.model.IStageHandle; -public class PythonChainStage extends AbstractStatelessChainStage implements IAfterConfigurableRefreshListener { - protected String pythonChannelName; - protected transient IChannel channel; - protected transient IStageHandle handle = new IStageHandle() { - @Override - protected IMessage doProcess(IMessage message, AbstractContext context) { - IStreamOperator receiver = (IStreamOperator)channel; - IMessage msg = receiver.doMessage(message, context); - message.setMessageBody(msg.getMessageBody()); - context.setMessage(message); - return message; - } - - @Override - public String getName() { - return PythonChainStage.class.getName(); - } - }; - - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - IChannel channel = configurableService.queryConfigurable(IChannel.TYPE, pythonChannelName); - this.channel = channel; - } +public class PythonChainStage extends AbstractStatelessChainStage { + @ENVDependence protected IChannel channel; @Override - protected IStageHandle selectHandle(T t, AbstractContext context) { - return handle; - } - - public String getPythonChannelName() { - return pythonChannelName; - } - - public void setPythonChannelName(String pythonChannelName) { - this.pythonChannelName = pythonChannelName; + protected IMessage handleMessage(IMessage message, AbstractContext context) { + IStreamOperator receiver = (IStreamOperator) channel; + IMessage msg = receiver.doMessage(message, context); + message.setMessageBody(msg.getMessageBody()); + context.setMessage(message); + return message; } public IChannel getChannel() { @@ -69,7 +41,6 @@ public IChannel getChannel() { public void setChannel(IChannel channel) { this.channel = channel; this.setNameSpace(channel.getNameSpace()); - this.setPythonChannelName(channel.getConfigureName()); } @Override diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/RightJoinChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/RightJoinChainStage.java deleted file mode 100644 index e587700b..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/RightJoinChainStage.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * 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.topology.stages; - -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.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.context.AbstractContext; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.context.MessageHeader; -import org.apache.rocketmq.streams.common.topology.ChainStage; -import org.apache.rocketmq.streams.common.topology.model.IStageHandle; -import org.apache.rocketmq.streams.common.topology.model.Pipeline; - -/** - * 双流join的右流的处理逻辑 - */ -public class RightJoinChainStage extends ChainStage implements IAfterConfigurableRefreshListener { - protected String pipelineName; - - protected transient Pipeline pipline; - - protected transient IStageHandle handle = new IStageHandle() { - @Override - protected IMessage doProcess(IMessage message, AbstractContext context) { - String joinFlag = MessageHeader.JOIN_RIGHT; - message.getHeader().setMsgRouteFromLable(joinFlag); - pipline.doMessage(message, context); - context.breakExecute(); - return message; - } - - @Override - public String getName() { - return RightJoinChainStage.class.getName(); - } - }; - - @Override - public boolean isAsyncNode() { - return true; - } - - @Override - protected IStageHandle selectHandle(IMessage message, AbstractContext context) { - return handle; - } - - public String getPipelineName() { - return pipelineName; - } - - public void setPipelineName(String pipelineName) { - this.pipelineName = pipelineName; - } - - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - pipline = configurableService.queryConfigurable(Pipeline.TYPE, pipelineName); - } - - @Override - public void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage) { - sendSystem(message, context, pipline); - } - - @Override - public void addNewSplit(IMessage message, AbstractContext context, NewSplitMessage newSplitMessage) { - sendSystem(message, context, pipline); - } - - @Override - public void removeSplit(IMessage message, AbstractContext context, RemoveSplitMessage removeSplitMessage) { - sendSystem(message, context, pipline); - } -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/SQLChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/SQLChainStage.java deleted file mode 100644 index e126173c..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/SQLChainStage.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * 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.topology.stages; - -import org.apache.rocketmq.streams.common.channel.IChannel; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.context.AbstractContext; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; -import org.apache.rocketmq.streams.common.topology.model.IStageHandle; - -@Deprecated -public class SQLChainStage extends AbstractStatelessChainStage implements IAfterConfigurableRefreshListener { - protected String dbChannelName; - protected transient IChannel channel; - protected transient IStageHandle handle = new IStageHandle() { - @Override - protected IMessage doProcess(IMessage message, AbstractContext context) { - IStreamOperator receiver = (IStreamOperator)channel; - IMessage msg = receiver.doMessage(message, context); - message.setMessageBody(msg.getMessageBody()); - context.setMessage(message); - return message; - } - - @Override - public String getName() { - return SQLChainStage.class.getName(); - } - }; - - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - IChannel channel = configurableService.queryConfigurable(IChannel.TYPE, dbChannelName); - this.channel = channel; - } - - @Override - public boolean isAsyncNode() { - return false; - } - - @Override - protected IStageHandle selectHandle(T t, AbstractContext context) { - return handle; - } - - public String getDbChannelName() { - return dbChannelName; - } - - public void setDbChannelName(String dbChannelName) { - this.dbChannelName = dbChannelName; - } - - public IChannel getChannel() { - return channel; - } - - public void setChannel(IChannel channel) { - this.channel = channel; - this.dbChannelName = channel.getConfigureName(); - - } -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ScriptChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ScriptChainStage.java index 3edfa7b4..07fa4cf1 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ScriptChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ScriptChainStage.java @@ -17,80 +17,40 @@ package org.apache.rocketmq.streams.common.topology.stages; import java.util.List; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.configurable.annotation.ConfigurableReference; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; import org.apache.rocketmq.streams.common.topology.model.AbstractScript; -import org.apache.rocketmq.streams.common.topology.model.IStageHandle; -import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.common.utils.TraceUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class ScriptChainStage extends AbstractStatelessChainStage implements IAfterConfigurableRefreshListener { - private static final Log LOG = LogFactory.getLog(ScriptChainStage.class); - protected String scriptName; +public class ScriptChainStage extends AbstractStatelessChainStage { + private static final Logger LOGGER = LoggerFactory.getLogger(ScriptChainStage.class); - protected transient AbstractScript script; + @ConfigurableReference protected AbstractScript script; public ScriptChainStage() { setEntityName("operator"); } - protected transient IStageHandle handle = new IStageHandle() { - @Override - protected IMessage doProcess(IMessage message, AbstractContext context) { - IStreamOperator> receiver = (IStreamOperator) script; - List messages = receiver.doMessage(message, context); - TraceUtil.debug(message.getHeader().getTraceId(), "ScriptChainStage", script.getValue(), - message.getMessageBody().toJSONString()); - if (messages == null || messages.size() == 0) { - context.breakExecute(); - return message; - } else if (messages.size() == 1) { - context.setMessage(messages.get(0)); - } else { - context.setSplitModel(true); - context.setSplitMessages(messages); - } - if (ComponentCreator.getPropertyBooleanValue("debug.message.parse")) { - LOG.info("message size is " + messages.size() + " split mode is " + context.isSplitModel()); - } - return message; - } - - @Override - public String getName() { - return ScriptChainStage.class.getName(); - } - }; - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - if (StringUtil.isEmpty(scriptName)) { - return; + protected IMessage handleMessage(IMessage message, AbstractContext context) { + IStreamOperator> receiver = (IStreamOperator) script; + List messages = receiver.doMessage(message, context); + TraceUtil.debug(message.getHeader().getTraceId(), "ScriptChainStage", script.getValue(), + message.getMessageBody().toJSONString()); + if (messages == null || messages.size() == 0) { + context.breakExecute(); + return message; + } else if (messages.size() == 1) { + context.setMessage(messages.get(0)); + } else { + context.setSplitModel(true); + context.setSplitMessages(messages); } - AbstractScript script = queryScript(configurableService, AbstractScript.TYPE, scriptName); - this.script = script; - } - - @Override - protected IStageHandle selectHandle(T t, AbstractContext context) { - return handle; - } - - protected AbstractScript queryScript(IConfigurableService configurableService, String type, String scriptName) { - return configurableService.queryConfigurable(type, scriptName); - } - - public void setScript(AbstractScript script) { - this.script = script; - this.setNameSpace(script.getNameSpace()); - this.setScriptName(script.getConfigureName()); - this.setLabel(script.getConfigureName()); + return message; } public List getDependentScripts(String fieldName) { @@ -112,11 +72,10 @@ public AbstractScript getScript() { return script; } - public String getScriptName() { - return scriptName; + public void setScript(AbstractScript script) { + this.script = script; + this.setNameSpace(script.getNameSpace()); + this.setLabel(script.getName()); } - public void setScriptName(String scriptName) { - this.scriptName = scriptName; - } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/UnionChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ShuffleChainStage.java similarity index 51% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/UnionChainStage.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ShuffleChainStage.java index 944fea61..54ddd5aa 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/UnionChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ShuffleChainStage.java @@ -16,36 +16,36 @@ */ package org.apache.rocketmq.streams.common.topology.stages; -import java.util.List; 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.topology.AbstractMutilPipelineChainPipline; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; +import org.apache.rocketmq.streams.common.topology.model.AbstractChainStage; -/** - * 新的解析已经废弃,主要兼容老的规则数据 - */ -@Deprecated -public class UnionChainStage extends AbstractMutilPipelineChainPipline { +public class ShuffleChainStage extends AbstractStatelessChainStage { - private static final long serialVersionUID = -6448769339534974034L; + protected AbstractChainStage outputChainStage; + protected AbstractChainStage consumeChainStage; - @Override - protected boolean executePipline(ChainPipeline pipline, IMessage copyMessage, Context newContext, String msgSourceName) { + @Override protected IMessage handleMessage(IMessage message, AbstractContext context) { + return null; + } - pipline.doMessage(copyMessage, newContext); + @Override public boolean isAsyncNode() { return false; } - @Override - protected void doMessageAfterFinishPipline(IMessage message, AbstractContext context, List messages) { - if (messages.size() == 0) { - context.breakExecute(); - } else { - context.openSplitModel(); - context.setSplitMessages(messages); - } + public AbstractChainStage getOutputChainStage() { + return outputChainStage; + } + + public void setOutputChainStage(AbstractChainStage outputChainStage) { + this.outputChainStage = outputChainStage; } + public AbstractChainStage getConsumeChainStage() { + return consumeChainStage; + } + + public void setConsumeChainStage(AbstractChainStage consumeChainStage) { + this.consumeChainStage = consumeChainStage; + } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ShuffleConsumerChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ShuffleConsumerChainStage.java deleted file mode 100644 index af1d6109..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ShuffleConsumerChainStage.java +++ /dev/null @@ -1,193 +0,0 @@ -/* - * 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.topology.stages; - -import com.alibaba.fastjson.JSONArray; -import com.alibaba.fastjson.JSONObject; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; -import org.apache.rocketmq.streams.common.channel.source.ISource; -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.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -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.interfaces.IStreamOperator; -import org.apache.rocketmq.streams.common.interfaces.ISystemMessage; -import org.apache.rocketmq.streams.common.topology.SectionPipeline; -import org.apache.rocketmq.streams.common.topology.model.AbstractRule; -import org.apache.rocketmq.streams.common.topology.model.IStageHandle; -import org.apache.rocketmq.streams.common.topology.shuffle.ShuffleMQCreator; -import org.apache.rocketmq.streams.common.utils.CompressUtil; -import org.apache.rocketmq.streams.common.utils.StringUtil; -import org.apache.rocketmq.streams.common.utils.TraceUtil; - -public class ShuffleConsumerChainStage extends AbstractStatelessChainStage implements IAfterConfigurableRefreshListener { - /** - * 消息所属的window - */ - protected transient String MSG_OWNER = "MSG_OWNER"; - protected static final String SHUFFLE_MESSAGES = "SHUFFLE_MESSAGES"; - private static final String SHUFFLE_TRACE_ID = "SHUFFLE_TRACE_ID"; - public static final String SHUFFLE_OFFSET = "SHUFFLE_OFFSET"; - - - private transient SectionPipeline lastStages; - - - protected transient ISource consumer; - protected String shuffleOwnerName;//shuffle 拥有者到名子,如是窗口,则是windowname+groupname+updateflag - - - - - protected transient AtomicBoolean hasStart = new AtomicBoolean(false); - public void startSource() { - if (consumer == null) { - return; - } - if (hasStart.compareAndSet(false, true)) { - consumer.start(new IStreamOperator() { - @Override public Object doMessage(IMessage message, AbstractContext context) { - sendMessage(message,context); - return message; - } - }); - } - - } - - - /** - * 接收到分片信息,如果是系统消息,做缓存刷新,否则把消息放入缓存,同时计算存储的有效性 - * - * @param oriMessage - * @param context - * @return - */ - - protected transient AtomicLong COUNT = new AtomicLong(0); - - /** - * send shuffle msg 2 last node - * @param oriMessage - * @param context - */ - protected void sendMessage(IMessage oriMessage, AbstractContext context) { - if (oriMessage.getHeader().isSystemMessage()) { - lastStages.doMessage(oriMessage,context); - return ; - - } - if (oriMessage.getMessageBody().getBooleanValue(ShuffleProducerChainStage.IS_COMPRESSION_MSG)) { - byte[] bytes = oriMessage.getMessageBody().getBytes(ShuffleProducerChainStage.COMPRESSION_MSG_DATA); - String msgStr = CompressUtil.unGzip(bytes); - oriMessage.setMessageBody(JSONObject.parseObject(msgStr)); - } - /** - * 过滤不是这个window的消息,一个shuffle通道,可能多个window共享,这里过滤掉非本window的消息 - */ - boolean isFilter = filterNotOwnerMessage(oriMessage); - if (isFilter) { - return; - } - String queueId = oriMessage.getHeader().getQueueId(); - JSONArray messages = oriMessage.getMessageBody().getJSONArray(SHUFFLE_MESSAGES); - if (messages == null) { - return; - } - - String traceId = oriMessage.getMessageBody().getString(SHUFFLE_TRACE_ID); - if (!StringUtil.isEmpty(traceId)) { - TraceUtil.debug(traceId, "shuffle message in", "received message size:" + messages.size()); - } - int i=0; - for (Object obj : messages) { - IMessage message = new Message((JSONObject) obj); - message.getHeader().setQueueId(queueId); - message.getHeader().setOffset(oriMessage.getHeader().getOffset()); - message.getHeader().addLayerOffset(i++); - /** - * create new offset - */ - message.getMessageBody().put(SHUFFLE_OFFSET, oriMessage.getHeader().getOffset()); - lastStages.doMessage(message,new Context(message)); - } - } - - /** - * When the producer receives the first piece of data, it will notify the consumer to start consumption - * @param t - * @param context - * @return - */ - @Override protected IStageHandle selectHandle(T t, AbstractContext context) { - return new IStageHandle() { - @Override protected T doProcess(T t, AbstractContext context) { - startSource(); - return null; - } - - @Override public String getName() { - return ShuffleConsumerChainStage.class.getName(); - } - }; - } - /** - * 过滤掉不是这个window的消息 - * - * @param oriMessage - * @return - */ - protected boolean filterNotOwnerMessage(IMessage oriMessage) { - String owner = oriMessage.getMessageBody().getString(MSG_OWNER); - if (owner != null && owner.equals(this.shuffleOwnerName)) { - return false; - } - return true; - } - - - - - @Override public boolean isAsyncNode() { - return false; - } - - public String getShuffleOwnerName() { - return shuffleOwnerName; - } - - public void setShuffleOwnerName(String shuffleOwnerName) { - this.shuffleOwnerName = shuffleOwnerName; - } - - @Override public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - if(this.consumer==null){ - this.consumer=ShuffleMQCreator.getSource(getPipeline().getNameSpace(),getPipeline().getConfigureName(),shuffleOwnerName); - this.consumer.init(); - lastStages = getReceiverAfterCurrentNode(); - } - - - } -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ShuffleProducerChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ShuffleProducerChainStage.java deleted file mode 100644 index b7d759b0..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ShuffleProducerChainStage.java +++ /dev/null @@ -1,345 +0,0 @@ -/* - * 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.topology.stages; - -import com.alibaba.fastjson.JSONArray; -import com.alibaba.fastjson.JSONObject; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.lang3.tuple.MutablePair; -import org.apache.commons.lang3.tuple.Pair; -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.sink.AbstractSupportShuffleSink; -import org.apache.rocketmq.streams.common.channel.sinkcache.impl.AbstractMultiSplitMessageCache; -import org.apache.rocketmq.streams.common.channel.source.ISource; -import org.apache.rocketmq.streams.common.channel.split.ISplit; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -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.topology.ChainPipeline; -import org.apache.rocketmq.streams.common.topology.model.AbstractRule; -import org.apache.rocketmq.streams.common.topology.model.AbstractStage; -import org.apache.rocketmq.streams.common.topology.model.IWindow; -import org.apache.rocketmq.streams.common.topology.shuffle.ShuffleMQCreator; -import org.apache.rocketmq.streams.common.utils.CompressUtil; -import org.apache.rocketmq.streams.common.utils.StringUtil; -import org.apache.rocketmq.streams.common.utils.TraceUtil; - -public class ShuffleProducerChainStage extends OutputChainStage { - private static final Log LOG = LogFactory.getLog(ShuffleProducerChainStage.class); - public static final String IS_COMPRESSION_MSG = "_is_compress_msg"; - public static final String COMPRESSION_MSG_DATA = "_compress_msg"; - public static final String MSG_FROM_SOURCE = "msg_from_source"; - public static final String ORIGIN_OFFSET = "origin_offset"; - - public static final String ORIGIN_QUEUE_ID = "origin_queue_id"; - - public static final String ORIGIN_QUEUE_IS_LONG = "origin_offset_is_LONG"; - - public static final String ORIGIN_MESSAGE_HEADER = "origin_message_header"; - - public static final String ORIGIN_SOURCE_NAME = "origin_offset_name"; - - public static final String SHUFFLE_KEY = "SHUFFLE_KEY"; - - public static final String ORIGIN_MESSAGE_TRACE_ID = "origin_request_id"; - - protected static final String SHUFFLE_QUEUE_ID = "SHUFFLE_QUEUE_ID"; - protected static final String SHUFFLE_MESSAGES = "SHUFFLE_MESSAGES"; - /** - * 消息所属的window - */ - protected transient String MSG_OWNER = "MSG_OWNER"; - - private static final String SHUFFLE_TRACE_ID = "SHUFFLE_TRACE_ID"; - - private transient ShuffleMQCreator shuffleMQCreator; - - - - protected String shuffleOwnerName;//shuffle 拥有者到名子,如是窗口,则是windowname+groupname+updateflag - protected String windowName;//Provide objects generated by shuffle key, such as window objects - protected int splitCount; - - - protected transient IWindow window;//generator shuffle key - /** - * used for shuffle - */ - protected transient AbstractSupportShuffleSink producer;//sink for shuffle - - protected transient ShuffleMsgCache shuffleMsgCache = new ShuffleMsgCache(); - - - protected transient boolean isWindowTest = false;//used for test, can fired quickly - protected transient AtomicLong COUNT = new AtomicLong(0);//usded for test - - /** - * Layer 2 cache,Each piece is divided into multiple pieces - */ - protected class ShuffleMsgCache extends AbstractMultiSplitMessageCache> { - - public ShuffleMsgCache() { - super(messages -> { - if (messages == null || messages.size() == 0) { - return true; - } - ISplit split = messages.get(0).getLeft(); - JSONObject jsonObject = messages.get(0).getRight(); - JSONArray allMsgs = jsonObject.getJSONArray(SHUFFLE_MESSAGES); - for (int i = 1; i < messages.size(); i++) { - Pair pair = messages.get(i); - JSONObject msg = pair.getRight(); - JSONArray jsonArray = msg.getJSONArray(SHUFFLE_MESSAGES); - if (jsonArray != null) { - allMsgs.addAll(jsonArray); - } - } - JSONObject zipJsonObject = new JSONObject(); - zipJsonObject.put(COMPRESSION_MSG_DATA, CompressUtil.gZip(jsonObject.toJSONString())); - zipJsonObject.put(IS_COMPRESSION_MSG, true); - producer.batchAdd(new Message(zipJsonObject), split); - producer.flush(split.getQueueId()); - - return true; - }); - } - - @Override - protected String createSplitId(Pair msg) { - return msg.getLeft().getQueueId(); - } - } - - @Override protected boolean initConfigurable() { - this.sink=new AbstractSink() { - @Override protected boolean batchInsert(List messages) { - addMsg2ShuffleCache(messages); - return true; - } - }; - this.sink.init(); - - - isWindowTest = ComponentCreator.getPropertyBooleanValue("window.fire.isTest"); - return super.initConfigurable(); - } - protected transient AtomicBoolean hasNotify=new AtomicBoolean(false); - @Override protected IMessage doSink(IMessage message, AbstractContext context) { - if(hasNotify.compareAndSet(false,true)){ - notifyShuffleConsumerStart(); - } - return super.doSink(message, context); - } - - /** - * When the producer receives the first piece of data, it will notify the consumer to start consumption - */ - protected void notifyShuffleConsumerStart() { - AbstractStage consumerStage=(AbstractStage)((ChainPipeline)pipeline).getStageMap().get(nextStageLabels.get(0)); - IMessage message=new Message(new JSONObject()); - consumerStage.doMessage(message,new Context(message)); - } - - /** - * - * @param messageList - * @return - */ - protected boolean addMsg2ShuffleCache(List messageList) { - if(window!=null){ - //call back window domessage - for(IMessage message:messageList){ - window.doMessage(message,new Context(message)); - } - } - Map shuffleMap = keyByMsg(messageList); - if (shuffleMap != null && shuffleMap.size() > 0) { - Set splitIds = new HashSet<>(); - - for (Map.Entry entry : shuffleMap.entrySet()) { - ISplit split = this.shuffleMQCreator.getQueueList().get(entry.getKey()); - JSONObject msg = createMsg(entry.getValue(), split); - - shuffleMsgCache.addCache(new MutablePair<>(split, msg)); - splitIds.add(split.getQueueId()); - - } - - } - if (isWindowTest) { - long count = COUNT.addAndGet(messageList.size()); - System.out.println(shuffleOwnerName + " send shuffle msg count is " + count); - shuffleMsgCache.flush(); - } - return true; - } - - - - /** - * 对接收的消息按照不同shuffle key进行分组 - * - * @param messages - * @return - */ - protected Map keyByMsg(List messages) { - Map shuffleMap = new HashMap<>(); - for (IMessage msg : messages) { - if (msg.getHeader().isSystemMessage()) { - continue; - } - - String shuffleKey = generateShuffleKey(msg); - if (StringUtil.isEmpty(shuffleKey)) { - shuffleKey = ""; - LOG.debug("there is no group by value in message! " + msg.getMessageBody().toString()); - //continue; - } - Integer index = this.shuffleMQCreator.hash(shuffleKey); - JSONObject body = msg.getMessageBody(); - String offset = msg.getHeader().getOffset(); - String queueId = msg.getHeader().getQueueId(); - - body.put(ORIGIN_OFFSET, offset); - body.put(ORIGIN_QUEUE_ID, queueId); - body.put(ORIGIN_QUEUE_IS_LONG, msg.getHeader().getMessageOffset().isLongOfMainOffset()); - body.put(ORIGIN_MESSAGE_HEADER, JSONObject.toJSONString(msg.getHeader())); - body.put(ORIGIN_MESSAGE_TRACE_ID, msg.getHeader().getTraceId()); - body.put(SHUFFLE_KEY, shuffleKey); - - - JSONArray jsonArray = shuffleMap.get(index); - if (jsonArray == null) { - jsonArray = new JSONArray(); - shuffleMap.put(index, jsonArray); - } - jsonArray.add(body); - - } - return shuffleMap; - } - - /** - * Combine multiple messages into one to reduce the pressure on the shuffle queue and improve throughput - * @param messages - * @param split - * @return - */ - protected JSONObject createMsg(JSONArray messages, ISplit split) { - JSONObject msg = new JSONObject(); - //分片id - msg.put(SHUFFLE_QUEUE_ID, split.getQueueId()); - //合并的消息 - msg.put(SHUFFLE_MESSAGES, messages); - //消息owner - msg.put(MSG_OWNER, shuffleOwnerName); - // - try { - List traceList = new ArrayList<>(); - List groupByList = new ArrayList<>(); - for (int i = 0; i < messages.size(); i++) { - JSONObject object = messages.getJSONObject(i); - groupByList.add(object.getString("SHUFFLE_KEY")); - traceList.add(object.getString(ORIGIN_MESSAGE_TRACE_ID)); - } - String traceInfo = StringUtils.join(traceList); - String groupInfo = StringUtils.join(groupByList); - msg.put(SHUFFLE_TRACE_ID, StringUtils.join(traceList)); - TraceUtil.debug(traceInfo, "origin message out", split.getQueueId(), groupInfo, getConfigureName()); - } catch (Exception e) { - //do nothing - } - return msg; - } - - - - /** - * If it is a window shuffle, calculate the shuffle key through the window object; otherwise, give the shuffle key randomly according to the number of queues - * - * @param message - * @return - */ - protected String generateShuffleKey(IMessage message){ - if(window!=null){ - return window.generateShuffleKey(message); - }else { - return (Math.random()*this.shuffleMQCreator.getQueueList().size()+1)+""; - } - } - - protected AbstractSupportShuffleSink createProducer(String name, ISource source) { - this.shuffleMQCreator=ShuffleMQCreator.createShuffleCreator(((ChainPipeline)this.getPipeline()).getSource(),getPipeline().getNameSpace(),getPipeline().getConfigureName(),shuffleOwnerName,splitCount); - return shuffleMQCreator.getProducer(); - } - - - - @Override public boolean isAsyncNode() { - return true; - } - - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - window=configurableService.queryConfigurable(windowName,IWindow.TYPE); - if(this.producer==null){ - this.producer=createProducer(shuffleOwnerName,((ChainPipeline)this.getPipeline()).getSource()); - } - } - @Override - protected boolean openMockChannel() { - return false; - } - - public String getShuffleOwnerName() { - return shuffleOwnerName; - } - - public void setShuffleOwnerName(String shuffleOwnerName) { - this.shuffleOwnerName = shuffleOwnerName; - } - - public String getWindowName() { - return windowName; - } - - public void setWindowName(String windowName) { - this.windowName = windowName; - } - - public int getSplitCount() { - return splitCount; - } - - public void setSplitCount(int splitCount) { - this.splitCount = splitCount; - } -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/WindowChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/SynchronousWindowChainStage.java similarity index 62% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/WindowChainStage.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/SynchronousWindowChainStage.java index fff987f4..8dbf8a47 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/WindowChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/SynchronousWindowChainStage.java @@ -16,43 +16,34 @@ */ package org.apache.rocketmq.streams.common.topology.stages; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; import org.apache.rocketmq.streams.common.configurable.IConfigurable; +import org.apache.rocketmq.streams.common.configurable.annotation.ConfigurableReference; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.topology.model.IStageHandle; -import org.apache.rocketmq.streams.common.topology.model.IWindow; +import org.apache.rocketmq.streams.common.topology.ISynchronousWindow; +import org.apache.rocketmq.streams.common.topology.IWindow; -public class WindowChainStage extends AbstractWindowStage implements IAfterConfigurableRefreshListener { +public class SynchronousWindowChainStage extends AbstractStatelessChainStage { private static final long serialVersionUID = -6592591896560866562L; - protected transient IStageHandle handle = new IStageHandle() { + @ConfigurableReference protected IWindow window; - @Override - protected IMessage doProcess(IMessage message, AbstractContext context) { - window.doMessage(message, context); - if (!window.isSynchronous()) { - context.breakExecute(); - } - - return message; - } - - @Override - public String getName() { - return WindowChainStage.class.getName(); - } - - }; - - public WindowChainStage() { + public SynchronousWindowChainStage() { super.entityName = "window"; } @Override - protected IStageHandle selectHandle(T t, AbstractContext context) { - return handle; + protected IMessage handleMessage(IMessage message, AbstractContext context) { + if (window instanceof ISynchronousWindow) { + ISynchronousWindow notNeedShuffle = (ISynchronousWindow) window; + notNeedShuffle.accumulateDirectly(message, context); + } + if (!window.isSynchronous()) { + context.breakExecute(); + } + + return message; } public IWindow getWindow() { @@ -62,8 +53,7 @@ public IWindow getWindow() { public void setWindow(IWindow window) { this.window = window; if (IConfigurable.class.isInstance(window)) { - setWindowName(window.getConfigureName()); - setLabel(window.getConfigureName()); + setLabel(window.getName()); } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/UnionStartChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/UnionStartChainStage.java index ae934843..e880d76a 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/UnionStartChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/UnionStartChainStage.java @@ -16,15 +16,10 @@ */ package org.apache.rocketmq.streams.common.topology.stages; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; import java.util.Map; import java.util.Set; 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.topology.ChainPipeline; import org.apache.rocketmq.streams.common.utils.CollectionUtil; public class UnionStartChainStage extends EmptyChainStage { @@ -32,16 +27,15 @@ public class UnionStartChainStage extends EmptyChainStage { //每个pipline,对应一个消息来源,在消息头上会有消息来源的name,根据name转发数据 protected Map> msgSource2StageLables; - @Override - protected IMessage proccessMessage(IMessage message, AbstractContext context) { + protected IMessage handleMessage(IMessage message, AbstractContext context) { if (CollectionUtil.isEmpty(msgSource2StageLables)) { return message; } String msgSourceName = message.getHeader().getMsgRouteFromLable(); - Set lableNames=msgSource2StageLables.get(msgSourceName); - if(lableNames!=null){ - for(String lableName:lableNames){ + Set lableNames = msgSource2StageLables.get(msgSourceName); + if (lableNames != null) { + for (String lableName : lableNames) { message.getHeader().addRouteLabel(lableName); } } @@ -56,4 +50,5 @@ public Map> getMsgSource2StageLables() { public void setMsgSource2StageLables(Map> msgSource2StageLables) { this.msgSource2StageLables = msgSource2StageLables; } + } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ViewChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ViewChainStage.java deleted file mode 100644 index 553036ac..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ViewChainStage.java +++ /dev/null @@ -1,529 +0,0 @@ -/* - * 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.topology.stages; - -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.ServiceLoader; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; -import org.apache.rocketmq.streams.common.cache.compress.BitSetCache; -import org.apache.rocketmq.streams.common.channel.impl.view.ViewSink; -import org.apache.rocketmq.streams.common.channel.sink.ISink; -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.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -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.optimization.IHomologousOptimization; -import org.apache.rocketmq.streams.common.optimization.MessageGlobleTrace; -import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintCache; -import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintMetric; -import org.apache.rocketmq.streams.common.schedule.ScheduleManager; -import org.apache.rocketmq.streams.common.schedule.ScheduleTask; -import org.apache.rocketmq.streams.common.threadpool.ThreadPoolFactory; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; -import org.apache.rocketmq.streams.common.topology.model.IStageHandle; -import org.apache.rocketmq.streams.common.topology.model.Pipeline; -import org.apache.rocketmq.streams.common.topology.task.TaskAssigner; -import org.apache.rocketmq.streams.common.utils.CollectionUtil; -import org.apache.rocketmq.streams.common.utils.StringUtil; - -public class ViewChainStage extends OutputChainStage implements IAfterConfigurableRefreshListener { - private static final Log LOG = LogFactory.getLog(ViewChainStage.class); - - /** - * 动态加载的pipeline,pipeline的source type是view,且tablename=sink的view name - */ - protected transient List> pipelines = new ArrayList<>(); - /** - * 是否开启优化 - */ - protected transient Boolean isOpenOptimization = true; - - /** - * Used for fingerprint filtering - */ - @ENVDependence protected String logFingerprint; - @ENVDependence protected String logFingerprintSwitch; - /** - * Homologous expression result cache - */ - protected int homologousRulesCaseSize = 2000000; - protected int homologousExpressionCaseSize = 2000000; - - /** - * Pre fingerprint filtering - */ - protected int preFingerprintCaseSize = 2000000; - protected int parallelTasks = 4; - /** - * fingerprint cache - */ - protected transient FingerprintCache homologousRulesCache; - /** - * Automatically parses pipelines, generates pre-filter fingerprints and expression estimates - */ - protected transient IHomologousOptimization homologousOptimization; - - /** - * Supports mul-threaded task execution - */ - protected transient ExecutorService executorService; - /** - * 总处理数据数 - */ - private final transient AtomicLong COUNT = new AtomicLong(0); - /** - * 触发规则的个数 - */ - private final transient AtomicLong FIRE_RULE_COUNT = new AtomicLong(0); - /** - * 最早的处理时间 - */ - protected transient Long firstReceiveTime = null; - - - - /** - * 是否包含key by节点 - */ - protected transient boolean isContainsKeyBy = false; - - protected transient IStageHandle handle = new IStageHandle() { - @Override - protected IMessage doProcess(IMessage message, AbstractContext context) { - if (CollectionUtil.isEmpty(pipelines)) { - return null; - } - - if (homologousOptimization == null && isOpenOptimization) { - synchronized (this) { - if (homologousOptimization == null) { - String isOpenOptimizationStr = ComponentCreator.getProperties().getProperty("homologous.optimization.switch"); - boolean isOpenOptimization = true; - if (StringUtil.isNotEmpty(isOpenOptimizationStr)) { - isOpenOptimization = Boolean.parseBoolean(isOpenOptimizationStr); - } - isOpenOptimization = isOpenOptimization; - if (isOpenOptimization) { - Iterable iterable = ServiceLoader.load(IHomologousOptimization.class); - Iterator it = iterable.iterator(); - if (it.hasNext()) { - homologousOptimization = it.next(); - homologousOptimization.optimizate(pipelines, homologousExpressionCaseSize, preFingerprintCaseSize); - } - } - } - } - } - //Calculate QPS - double qps = calculateQPS(); - if (StringUtil.isEmpty(logFingerprint)) { - if (COUNT.get() % 10000 == 0) { - System.out.println(getConfigureName() + " qps is " + qps + "。the count is " + COUNT.get()); - } - } - - if (homologousOptimization != null) { - homologousOptimization.calculate(message, context); - } - - boolean onlyOne = pipelines.size() == 1; - //Judge whether to turn on fingerprint filtering. When it is turned on, filter through fingerprint first - String msgKey = getFilterKey(message); - BitSetCache.BitSet bitSet = getFilterValue(msgKey); - boolean isHitCache = true; - CountDownLatch countDownLatch = null; - if (bitSet == null && StringUtil.isNotEmpty(logFingerprint)&&StringUtil.isNotEmpty(logFingerprintSwitch)&&Boolean.valueOf(logFingerprintSwitch)) { - bitSet = new BitSetCache.BitSet(pipelines.size()); - isHitCache = false; - if (!isContainsKeyBy && pipelines.size() > 1 && parallelTasks > 1) { - countDownLatch = new CountDownLatch(pipelines.size()); - } - } - int index = 0; - for (ChainPipeline pipeline :pipelines) { - //If the fingerprint matches, filter it directly - if (isHitCache && bitSet != null && bitSet.get(index)) { - index++; - continue; - } - - IMessage copyMessage = message; - if (!onlyOne) { - copyMessage = message.deepCopy(); - } - Context newContext = new Context(copyMessage); - newContext.setHomologousResult(context.getHomologousResult()); - newContext.setQuickFilterResult(context.getQuickFilterResult()); - HomologousTask homologousTask = new HomologousTask(copyMessage, newContext, pipeline, bitSet, index, msgKey); - if (executorService != null && countDownLatch != null) { - homologousTask.setCountDownLatch(countDownLatch); - executorService.execute(homologousTask); - } else { - homologousTask.run(); - } - index++; - } - - if (countDownLatch != null) { - try { - countDownLatch.await(); - } catch (InterruptedException e) { - e.printStackTrace(); - } - } - if (!isHitCache) { - addNoFireMessage(msgKey, bitSet); - } - if (StringUtil.isNotEmpty(logFingerprint)) { - printQPSWithFingerprint(qps); - } - - return null; - } - - @Override - public String getName() { - return ViewChainStage.class.getName(); - } - }; - - @Override - public boolean isAsyncNode() { - for (Pipeline pipline : this.pipelines) { - if (pipline.isAsynNode() == true) { - return true; - } - } - return false; - } - - @Override - public void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage) { - sendSystem(message, context, this.pipelines); - } - - @Override - public void addNewSplit(IMessage message, AbstractContext context, NewSplitMessage newSplitMessage) { - sendSystem(message, context, this.pipelines); - } - - @Override - public void removeSplit(IMessage message, AbstractContext context, RemoveSplitMessage removeSplitMessage) { - sendSystem(message, context, this.pipelines); - } - - @Override - public void batchMessageFinish(IMessage message, AbstractContext context, BatchFinishMessage checkPointMessage) { - sendSystem(message, context, this.pipelines); - } - - /** - * 每隔一段时间会重新刷新数据,如果有新增的pipline会加载起来,如果有删除的会去除掉 - * - * @param configurableService - */ - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - List> newPipelines = loadSubPipelines(configurableService); - boolean isChanged = false; - if(newPipelines==null){ - return; - } - - - List> deletePipeline = new ArrayList<>(); - if (newPipelines.size() > 0) { - for (ChainPipeline pipeline : newPipelines) { - if (!this.pipelines.contains(pipeline)) { - isChanged = true; - ScheduleManager.getInstance().regist(new ScheduleTask(0,10,pipeline)); - } - } - for (ChainPipeline pipeline : this.pipelines) { - if (!newPipelines.contains(pipeline)) { - isChanged = true; - deletePipeline.add(pipeline); - } - } - } - - if (isChanged) { - this.pipelines = newPipelines; - this.homologousRulesCache = new FingerprintCache(homologousRulesCaseSize); - for (ChainPipeline pipeline : deletePipeline) { - pipeline.destroy(); - } - } - - if (this.parallelTasks > 0 && executorService == null) { - executorService = ThreadPoolFactory.createThreadPool(this.parallelTasks); - } - } - @Override - public void setSink(ISink channel) { - ViewSink viewSink=(ViewSink)channel; - this.sink = channel; - this.setNameSpace(channel.getNameSpace()); - this.setSinkName(channel.getConfigureName()); - this.setLabel(channel.getConfigureName()); - this.setConfigureName(viewSink.getViewTableName()); - } - /** - * 动态装配子pipeline - * - * @param configurableService configurableService - */ - protected List> loadSubPipelines(IConfigurableService configurableService) { - List taskAssigners = configurableService.queryConfigurableByType(TaskAssigner.TYPE); - if (taskAssigners == null) { - return null; - } - String taskName = getConfigureName(); - List> subPipelines=new ArrayList<>(); - for (TaskAssigner taskAssigner : taskAssigners) { - if (!taskName.equals(taskAssigner.getTaskName())) { - continue; - } - String pipelineName = taskAssigner.getPipelineName(); - if(pipelineName!=null){ - ChainPipeline pipeline = configurableService.queryConfigurable(Pipeline.TYPE, pipelineName); - if (pipeline != null) { - subPipelines.add(pipeline); - } - } - } - return subPipelines; - } - - - - - /** - * When fingerprint is enabled, print QPS, filter rate, cache condition and rule matching rate - * - * @param qps qps - */ - protected void printQPSWithFingerprint(double qps) { - FingerprintMetric fingerprintMetric = this.homologousRulesCache.getOrCreateMetric(getOrCreateFingerNameSpace()); - double rate = fingerprintMetric.getHitCacheRate(); - double fireRate = (double) FIRE_RULE_COUNT.get() / (double) COUNT.get(); - if (COUNT.get() % 1000 == 0) { - System.out.println("qps is " + qps + ",the count is " + COUNT.get() + " the cache hit rate " + rate + " the cache size is " + fingerprintMetric.getCacheSize() + "," + "the fire rule rate is " + fireRate); - } - } - - - /** - * 如果确定这个message,在某个pipeline不触发,则记录下来,下次直接跳过,不执行 - * - * @param msgKey msgKey - * @param bitSet bitSet - */ - protected void addNoFireMessage(String msgKey, BitSetCache.BitSet bitSet) { - - if (StringUtil.isEmpty(logFingerprint)) { - return; - } - if(StringUtil.isEmpty(logFingerprintSwitch)){ - return ; - } - - if(Boolean.valueOf(logFingerprintSwitch)==false){ - return ; - } - this.homologousRulesCache.addLogFingerprint(getOrCreateFingerNameSpace(), msgKey, bitSet); - } - - protected BitSetCache.BitSet getFilterValue(String msgKey) { - if (StringUtil.isEmpty(logFingerprint)) { - return null; - } - if(StringUtil.isEmpty(logFingerprintSwitch)){ - return null; - } - - if(Boolean.valueOf(logFingerprintSwitch)==false){ - return null; - } - return this.homologousRulesCache.getLogFingerprint(getOrCreateFingerNameSpace(), msgKey); - } - - protected String getFilterKey(IMessage message) { - - if (StringUtil.isEmpty(logFingerprint)) { - return null; - } - - if(StringUtil.isEmpty(logFingerprintSwitch)){ - return null; - } - - if(Boolean.valueOf(logFingerprintSwitch)==false){ - return null; - } - - return FingerprintCache.creatFingerpringKey(message, getOrCreateFingerNameSpace(), this.logFingerprint); - } - - - - protected String getOrCreateFingerNameSpace() { - return getConfigureName(); - } - - /** - * Print QPS in the scene without fingerprint on - */ - protected double calculateQPS() { - if (firstReceiveTime == null) { - synchronized (this) { - if (firstReceiveTime == null) { - firstReceiveTime = System.currentTimeMillis(); - } - } - } - long second = ((System.currentTimeMillis() - firstReceiveTime) / 1000); - if (second == 0) { - second = 1; - } - return (double) (COUNT.incrementAndGet() / second); - } - /** - * 把消息投递给pipline的channel,让子pipline完成任务 注意:子pipline对消息的任何修改,都不反映到当前的pipline - * - * @param t - * @param context - * @return - */ - @Override - protected IStageHandle selectHandle(T t, AbstractContext context) { - return handle; - } - - - - class HomologousTask implements Runnable { - protected IMessage message; - protected AbstractContext context; - protected ChainPipeline pipeline; - protected BitSetCache.BitSet bitSet; - protected int index; - protected String msgKey; - protected CountDownLatch countDownLatch; - - public HomologousTask(IMessage message, AbstractContext context, ChainPipeline pipeline, BitSetCache.BitSet bitSet, int index, String msgKey) { - this.message = message; - this.context = context; - this.pipeline = pipeline; - this.bitSet = bitSet; - this.index = index; - this.msgKey = msgKey; - } - - @Override public void run() { - try { - pipeline.doMessage(message, context); - Boolean isFinish = MessageGlobleTrace.existFinishBranch(message); - if (isFinish != null && !isFinish) { - if (bitSet != null) { - bitSet.set(index); - } - } - Boolean existFinishBranch = MessageGlobleTrace.existFinishBranch(message); - if (existFinishBranch != null && existFinishBranch) { - FIRE_RULE_COUNT.incrementAndGet(); - } - } catch (Exception e) { - e.printStackTrace(); - LOG.error("pipeline execute error " + pipeline.getConfigureName(), e); - } finally { - if (this.countDownLatch != null) { - this.countDownLatch.countDown(); - } - } - } - - public CountDownLatch getCountDownLatch() { - return countDownLatch; - } - - public void setCountDownLatch(CountDownLatch countDownLatch) { - this.countDownLatch = countDownLatch; - } - } - - public int getHomologousRulesCaseSize() { - return homologousRulesCaseSize; - } - - public void setHomologousRulesCaseSize(int homologousRulesCaseSize) { - this.homologousRulesCaseSize = homologousRulesCaseSize; - } - - public int getHomologousExpressionCaseSize() { - return homologousExpressionCaseSize; - } - - public void setHomologousExpressionCaseSize(int homologousExpressionCaseSize) { - this.homologousExpressionCaseSize = homologousExpressionCaseSize; - } - - public int getPreFingerprintCaseSize() { - return preFingerprintCaseSize; - } - - public void setPreFingerprintCaseSize(int preFingerprintCaseSize) { - this.preFingerprintCaseSize = preFingerprintCaseSize; - } - - public int getParallelTasks() { - return parallelTasks; - } - - public void setParallelTasks(int parallelTasks) { - this.parallelTasks = parallelTasks; - } - - public String getLogFingerprint() { - return logFingerprint; - } - - public void setLogFingerprint(String logFingerprint) { - this.logFingerprint = logFingerprint; - } - - public String getLogFingerprintSwitch() { - return logFingerprintSwitch; - } - - public void setLogFingerprintSwitch(String logFingerprintSwitch) { - this.logFingerprintSwitch = logFingerprintSwitch; - } -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/MapOperator.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/MapOperator.java index efa4954a..f083646f 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/MapOperator.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/MapOperator.java @@ -25,13 +25,11 @@ public abstract class MapOperator extends StageBuilder { @Override - protected IMessage operate(IMessage message, AbstractContext context) { - map(message,context); + protected IMessage handleMessage(IMessage message, AbstractContext context) { + map(message, context); return message; } - protected abstract void map(IMessage message, AbstractContext context) ; - - + protected abstract void map(IMessage message, AbstractContext context); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/StageBuilder.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/StageBuilder.java index f7ea3e36..f145f3dd 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/StageBuilder.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/StageBuilder.java @@ -17,20 +17,16 @@ package org.apache.rocketmq.streams.common.topology.stages.udf; import java.io.Serializable; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.topology.ChainStage; -import org.apache.rocketmq.streams.common.topology.builder.IStageBuilder; +import org.apache.rocketmq.streams.common.topology.IStageBuilder; import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; -import org.apache.rocketmq.streams.common.topology.model.IStageHandle; +import org.apache.rocketmq.streams.common.topology.model.AbstractChainStage; import org.apache.rocketmq.streams.common.topology.stages.AbstractStatelessChainStage; /** * 给用户提供自定义的抽象类 */ -public abstract class StageBuilder extends AbstractStatelessChainStage implements IStageBuilder, Serializable, IAfterConfigurableRefreshListener { +public abstract class StageBuilder extends AbstractStatelessChainStage implements IStageBuilder>, Serializable { private static final long serialVersionUID = 1L; @Override @@ -38,39 +34,13 @@ protected boolean initConfigurable() { return true; } - /** - * 子类实现,实现具体的处理逻辑 - * - * @param message message - * @param context context - * @param - * @return - */ - protected abstract T operate(IMessage message, AbstractContext context); - - @Override - protected IStageHandle selectHandle(IMessage message, AbstractContext context) { - return new IStageHandle() { - @Override - protected IMessage doProcess(IMessage message, AbstractContext context) { - operate(message, context); - return message; - } - - @Override - public String getName() { - return StageBuilder.class.getName(); - } - }; - } - @Override public boolean isAsyncNode() { return false; } @Override - public ChainStage createStageChain(PipelineBuilder pipelineBuilder) { + public AbstractChainStage createStageChain(PipelineBuilder pipelineBuilder) { return new UDFChainStage(this); } @@ -79,9 +49,4 @@ public void addConfigurables(PipelineBuilder pipelineBuilder) { } - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - - } - } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/UDFChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/UDFChainStage.java index 537baa19..63c6143b 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/UDFChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/UDFChainStage.java @@ -17,20 +17,15 @@ package org.apache.rocketmq.streams.common.topology.stages.udf; import org.apache.rocketmq.streams.common.checkpoint.CheckPointMessage; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.topology.model.IStageHandle; import org.apache.rocketmq.streams.common.topology.stages.AbstractStatelessChainStage; -import org.apache.rocketmq.streams.common.utils.Base64Utils; -import org.apache.rocketmq.streams.common.utils.InstantiationUtil; /** * 所有给用户自定义代码的通用类,会转化成这个stage */ -public class UDFChainStage extends AbstractStatelessChainStage implements IAfterConfigurableRefreshListener { - protected String udfOperatorClassSerializeValue;//用户自定义的operator的序列化字节数组,做了base64解码 +public class UDFChainStage extends AbstractStatelessChainStage { + // protected String udfOperatorClassSerializeValue;//用户自定义的operator的序列化字节数组,做了base64解码 protected transient StageBuilder selfChainStage; public UDFChainStage() { @@ -38,8 +33,8 @@ public UDFChainStage() { public UDFChainStage(StageBuilder selfOperator) { this.selfChainStage = selfOperator; - byte[] bytes = InstantiationUtil.serializeObject(selfOperator); - udfOperatorClassSerializeValue = Base64Utils.encode(bytes); +// byte[] bytes = InstantiationUtil.serializeObject(selfOperator); +// udfOperatorClassSerializeValue = Base64Utils.encode(bytes); } @Override public void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage) { @@ -50,10 +45,6 @@ public UDFChainStage(StageBuilder selfOperator) { return false; } - @Override protected IStageHandle selectHandle(IMessage message, AbstractContext context) { - return selfChainStage.selectHandle(message, context); - } - @Override public IMessage doMessage(IMessage message, AbstractContext context) { super.doMessage(message, context); if (!context.isContinue() && this.filterFieldNames != null && context.get("_logfinger") != null) { @@ -65,20 +56,12 @@ public UDFChainStage(StageBuilder selfOperator) { return message; } - public String getUdfOperatorClassSerializeValue() { - return udfOperatorClassSerializeValue; - } - - public void setUdfOperatorClassSerializeValue(String udfOperatorClassSerializeValue) { - this.udfOperatorClassSerializeValue = udfOperatorClassSerializeValue; + @Override protected IMessage handleMessage(IMessage message, AbstractContext context) { + return selfChainStage.doMessage(message, context); } - @Override public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - if (udfOperatorClassSerializeValue != null) { - byte[] bytes = Base64Utils.decode(udfOperatorClassSerializeValue); - selfChainStage = InstantiationUtil.deserializeObject(bytes); - } + @Override public void startJob() { + super.startJob(); preFingerprint = loadLogFinger(); } - } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/UDFUnionChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/UDFUnionChainStage.java index 66483cfa..ac4ea5ff 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/UDFUnionChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/UDFUnionChainStage.java @@ -16,20 +16,17 @@ */ package org.apache.rocketmq.streams.common.topology.stages.udf; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.configurable.annotation.ConfigurableReference; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; -import org.apache.rocketmq.streams.common.topology.model.IStageHandle; import org.apache.rocketmq.streams.common.topology.model.Union; import org.apache.rocketmq.streams.common.topology.stages.AbstractStatelessChainStage; -public class UDFUnionChainStage extends AbstractStatelessChainStage implements IAfterConfigurableRefreshListener { - protected String unionName;//union对象的名字 +public class UDFUnionChainStage extends AbstractStatelessChainStage { protected boolean isMainStream = false;//是主流,在union时,主流.union(支流) - protected transient Union union; + @ConfigurableReference protected Union union; @Override public boolean isAsyncNode() { @@ -37,47 +34,26 @@ public boolean isAsyncNode() { } @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - union = configurableService.queryConfigurable(Union.TYPE, unionName); - if (isMainStream) { - IStreamOperator> receiver = getReceiverAfterCurrentNode(); - union.setReceiver(receiver); + protected IMessage handleMessage(IMessage message, AbstractContext context) { + if (!isMainStream) { + union.doMessage(message, context); } - - } - - @Override - protected IStageHandle selectHandle(IMessage message, AbstractContext context) { - return new IStageHandle() { - @Override - protected IMessage doProcess(IMessage message, AbstractContext context) { - if (!isMainStream) { - union.doMessage(message, context); - } - return message; - } - - @Override - public String getName() { - return UDFUnionChainStage.class.getName(); - } - }; + return message; } public void setUnion(Union union) { this.union = union; if (union != null) { - setUnionName(union.getConfigureName()); - setLabel(union.getConfigureName()); + setLabel(union.getName()); } } - public String getUnionName() { - return unionName; - } - - public void setUnionName(String unionName) { - this.unionName = unionName; + @Override public void startJob() { + super.startJob(); + if (isMainStream) { + IStreamOperator> receiver = getReceiverAfterCurrentNode(); + union.setReceiver(receiver); + } } public boolean isMainStream() { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/task/StreamsTask.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/task/StreamsTask.java deleted file mode 100644 index 38a6e7aa..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/task/StreamsTask.java +++ /dev/null @@ -1,160 +0,0 @@ -/* - * 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.topology.task; - -import com.google.common.collect.Lists; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; -import org.apache.rocketmq.streams.common.topology.model.Pipeline; - -/** - * run one or multi pipeline's - */ -public class StreamsTask extends BasedConfigurable implements IAfterConfigurableRefreshListener { - private static final Log LOG = LogFactory.getLog(StreamsTask.class); - - public static final String TYPE = "stream_task"; - - - /** - * 任务的状态,目前有started,stopped俩种, 任务序列化保存在数据库 - */ - public static final String STATE_STARTED = "started"; - public static final String STATE_STOPPED = "stopped"; - - protected String state = "stopped"; - /** - * 在当前进程中任务的状态 - */ - protected transient AtomicBoolean isStarted = new AtomicBoolean(false); - /** - * The pipeline or subtask executed in this task - */ - protected transient List> pipelines = new ArrayList<>(); - protected List pipelineNames = new ArrayList<>(); - - public StreamsTask() { - setType(TYPE); - } - - public void start() { - if (this.isStarted.compareAndSet(false, true)) { - for (ChainPipeline pipeline : pipelines) { - startPipeline(pipeline); - } - } - } - - @Override public void destroy() { - for (ChainPipeline pipeline : pipelines) { - pipeline.destroy(); - } - } - - @Override public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - - List> newPipelines = new ArrayList<>(); - boolean isChanged = false; - - if (this.pipelineNames != null && !pipelineNames.isEmpty()) { - for (String pipelineName : this.pipelineNames) { - ChainPipeline pipeline = configurableService.queryConfigurable(Pipeline.TYPE, pipelineName); - if (pipeline != null) { - newPipelines.add(pipeline); - } - } - } - - List> deletePipeline = new ArrayList<>(); - if (newPipelines.size() > 0) { - for (ChainPipeline pipeline : newPipelines) { - if (!this.pipelines.contains(pipeline)) { - isChanged = true; - break; - } - } - for (ChainPipeline pipeline : this.pipelines) { - if (!newPipelines.contains(pipeline)) { - isChanged = true; - deletePipeline.add(pipeline); - } - } - } - - if (isChanged) { - this.pipelines = newPipelines; - for (ChainPipeline pipeline : deletePipeline) { - pipeline.destroy(); - } - } - - } - - /** - * start one pipeline - * - * @param pipeline pipeline - */ - protected void startPipeline(ChainPipeline pipeline) { - Thread thread = new Thread(pipeline::startChannel); - thread.start(); - } - - public List> getPipelines() { - return pipelines; - } - - public void setPipelines(List> pipelines) { - this.pipelines = pipelines; - List pipelineNames = Lists.newArrayList(); - for (ChainPipeline pipeline : this.pipelines) { - pipelineNames.add(pipeline.getConfigureName()); - } - this.pipelineNames = pipelineNames; - } - - public List getPipelineNames() { - return pipelineNames; - } - - public void setPipelineNames(List pipelineNames) { - this.pipelineNames = pipelineNames; - } - - public AtomicBoolean getIsStarted() { - return isStarted; - } - - public void setIsStarted(AtomicBoolean isStarted) { - this.isStarted = isStarted; - } - - public String getState() { - return state; - } - - public void setState(String state) { - this.state = state; - } -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/AESUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/AESUtil.java index 770a753c..f64dc7b4 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/AESUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/AESUtil.java @@ -27,7 +27,7 @@ */ public class AESUtil { - private static final String PRIVATE_KEY = "f835mnga013mb39c"; + private static final String PRIVATE_KEY = ""; private static final String CHARSET = "UTF-8"; @@ -36,8 +36,7 @@ private static byte[] encrypt(String content, String strKey) throws Exception { Cipher cipher = Cipher.getInstance("AES/CBC/PKCS5Padding"); IvParameterSpec iv = new IvParameterSpec(PRIVATE_KEY.getBytes(CHARSET)); cipher.init(Cipher.ENCRYPT_MODE, skeySpec, iv); - byte[] encrypted = cipher.doFinal(content.getBytes(CHARSET)); - return encrypted; + return cipher.doFinal(content.getBytes(CHARSET)); } private static String decrypt(byte[] content, String strKey) throws Exception { @@ -57,9 +56,7 @@ private static SecretKeySpec getKey(String strKey) throws Exception { arrB[i] = arrBTmp[i]; } - SecretKeySpec skeySpec = new SecretKeySpec(arrB, "AES"); - - return skeySpec; + return new SecretKeySpec(arrB, "AES"); } public static byte[] stringToMD5(String plainText) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ClassUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ClassUtil.java index f21fe16d..565475f3 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ClassUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ClassUtil.java @@ -26,11 +26,11 @@ import javax.tools.JavaFileObject; import javax.tools.StandardJavaFileManager; import javax.tools.ToolProvider; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ClassUtil { - private static final Log logger = LogFactory.getLog(ClassUtil.class); + private static final Logger logger = LoggerFactory.getLogger(ClassUtil.class); private static JavaCompiler compiler; static { @@ -63,10 +63,12 @@ private static void javac(List ops, String... files) { JavaCompiler.CompilationTask task = compiler.getTask(null, manager, null, ops, null, it); task.call(); if (logger.isDebugEnabled()) { - for (String file : files) { logger.debug("Compile Java File:" + file); } + for (String file : files) { + logger.debug("Compile Java File:" + file); + } } } catch (Exception e) { - logger.error(e); + logger.error("异常", e); } finally { if (manager != null) { try { @@ -92,7 +94,9 @@ private static void writeJavaFile(String file, String source) throws Exception { BufferedWriter bw = null; try { File dir = new File(getFilePath(file)); - if (!dir.exists()) { dir.mkdirs(); } + if (!dir.exists()) { + dir.mkdirs(); + } bw = new BufferedWriter(new FileWriter(file)); bw.write(source); bw.flush(); @@ -121,7 +125,7 @@ private static Class load(String name) { logger.debug("Load Class[" + name + "] by " + classLoader); } } catch (Exception e) { - logger.error(e); + logger.error("异常", e); } return cls; } @@ -141,7 +145,7 @@ public static Class loadClass(String filePath, String source, String clsName, javac(ops, filePath); return load(clsName); } catch (Exception e) { - logger.error(e); + logger.error("异常", e); } return null; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/CollectionUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/CollectionUtil.java index 96b1bc7e..3059664e 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/CollectionUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/CollectionUtil.java @@ -18,8 +18,10 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; @SuppressWarnings("rawtypes") public final class CollectionUtil { @@ -62,4 +64,22 @@ public static List asList(T... array) { return list; } + public static void put2Set(Map> map, String key, T value) { + Set set = map.get(key); + if (set == null) { + set = new HashSet(); + map.put(key, set); + } + set.add(value); + } + + public static void put2List(Map> map, String key, T value) { + List set = map.get(key); + if (set == null) { + set = new ArrayList<>(); + map.put(key, set); + } + set.add(value); + } + } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ConfigurableUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ConfigurableUtil.java index c6f5786f..f9ad45ee 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ConfigurableUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ConfigurableUtil.java @@ -24,8 +24,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configurable.AbstractConfigurable; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; import org.apache.rocketmq.streams.common.configurable.IConfigurable; import org.apache.rocketmq.streams.common.configurable.IFieldProcessor; @@ -35,45 +33,8 @@ import org.apache.rocketmq.streams.common.metadata.MetaDataField; public class ConfigurableUtil { - protected static class FieldProcessorGetter implements IFieldProcessor { - private Map fieldName2JsonValue = new HashMap<>(); - private Map changeableFieldName2JsonValue = new HashMap<>(); - private JSONObject message; - - public FieldProcessorGetter(String message) { - this.message = JSON.parseObject(message); - } - - @Override - public void doProcess(Object o, Field field) { - String fieldJsonStr = message.getString(field.getName()); - if (field.getAnnotation(Changeable.class) != null) {//对标记易变的字段不参与比较 - changeableFieldName2JsonValue.put(field.getName(), fieldJsonStr); - } - fieldName2JsonValue.put(field.getName(), fieldJsonStr); - } - } - - public static IConfigurable create(String className, String namespace, String name, JSONObject property, - JSONObject mock) { - IConfigurable configurable = ConfigurableUtil.create(namespace, name, property, className); - if (mock != null) { - addMockData(mock, property); - } - return configurable; - } - - public static void addMockData(JSONObject mock, JSONObject property) { - Iterator> it = mock.entrySet().iterator(); - while (it.hasNext()) { - Map.Entry entry = it.next(); - String fieldName = entry.getKey(); - String value = (String) property.get(fieldName); - if (StringUtil.isNotEmpty(value)) { - String actualValue = (String) entry.getValue(); - ComponentCreator.getProperties().put(value, actualValue); - } - } + public static IConfigurable create(String className, String namespace, String name, JSONObject property, JSONObject mock) { + return ConfigurableUtil.create(namespace, name, property, className); } public static boolean compare(IConfigurable configurable1, IConfigurable configurable2) { @@ -83,13 +44,13 @@ public static boolean compare(IConfigurable configurable1, IConfigurable configu if (!configurable1.getType().equals(configurable2.getType())) { return false; } - if (!configurable1.getConfigureName().equals(configurable2.getConfigureName())) { + if (!configurable1.getName().equals(configurable2.getName())) { return false; } - if (BasedConfigurable.class.isInstance(configurable1) && BasedConfigurable.class.isInstance(configurable2)) { - BasedConfigurable abstractConfigurable1 = (BasedConfigurable)configurable1; - BasedConfigurable abstractConfigurable2 = (BasedConfigurable)configurable2; - if (abstractConfigurable1.getUpdateFlag() == abstractConfigurable2.getUpdateFlag()) { + if (configurable1 instanceof BasedConfigurable && configurable2 instanceof BasedConfigurable) { + BasedConfigurable abstractConfigurable1 = (BasedConfigurable) configurable1; + BasedConfigurable abstractConfigurable2 = (BasedConfigurable) configurable2; + if (abstractConfigurable1 == abstractConfigurable2) { return true; } else { return false; @@ -135,7 +96,7 @@ public static void refreshMock(IConfigurable configurable) { public static T create(String namespace, String name, String className) { IConfigurable configurable = ReflectUtil.forInstance(className); configurable.setNameSpace(namespace); - configurable.setConfigureName(name); + configurable.setName(name); return (T) configurable; } @@ -181,7 +142,7 @@ public static void setProperty2Configurable(Object configurable, String filedNam public static MetaData createMetaData(String namespace, String name, String... fields) { MetaData metaData = new MetaData(); metaData.setNameSpace(namespace); - metaData.setConfigureName(name); + metaData.setName(name); if (fields == null || fields.length == 0) { metaData.toObject(metaData.toJson()); return metaData; @@ -212,4 +173,23 @@ public static MetaData createMetaData(String namespace, String name, String... f return metaData; } + protected static class FieldProcessorGetter implements IFieldProcessor { + private Map fieldName2JsonValue = new HashMap<>(); + private Map changeableFieldName2JsonValue = new HashMap<>(); + private JSONObject message; + + public FieldProcessorGetter(String message) { + this.message = JSON.parseObject(message); + } + + @Override + public void doProcess(Object o, Field field) { + String fieldJsonStr = message.getString(field.getName()); + if (field.getAnnotation(Changeable.class) != null) {//对标记易变的字段不参与比较 + changeableFieldName2JsonValue.put(field.getName(), fieldJsonStr); + } + fieldName2JsonValue.put(field.getName(), fieldJsonStr); + } + } + } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ContantsUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ContantsUtil.java index 261515d8..edc6c89c 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ContantsUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ContantsUtil.java @@ -21,13 +21,12 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import org.apache.rocketmq.streams.common.model.NameCreator; public class ContantsUtil { private static final List CONSTANTS_SIGNS = new ArrayList<>();//对于特殊字符优先处理,里面存储需要特殊处理的字符 private static final Map CONSTANTS_SIGNS_REPLACE = new HashMap<>();// 特殊字符和替换字符的映射 private static final Map CONSTANTS_REPLACE_SIGNS = new HashMap<>();//替换字符和特殊字符的映射 - private static final NameCreator EscapesNameCreator=new NameCreator(); + static { CONSTANTS_SIGNS.add("\\\\"); CONSTANTS_SIGNS.add("\\\""); @@ -43,7 +42,6 @@ public class ContantsUtil { CONSTANTS_SIGNS_REPLACE.put("\"", "&@--@&"); CONSTANTS_SIGNS_REPLACE.put("#######", "#######"); - CONSTANTS_REPLACE_SIGNS.put("%%%%%", "\\\\"); CONSTANTS_REPLACE_SIGNS.put("&@--@&", "\""); CONSTANTS_REPLACE_SIGNS.put("^^^^", "'"); @@ -52,23 +50,6 @@ public class ContantsUtil { CONSTANTS_REPLACE_SIGNS.put("#######", "'"); } - public static String replaceEscape(String str,Map flag2Escapes){ - int index=str.indexOf("\\"); - if(index==-1){ - return str; - } - String word=str.substring(index,index+1); - String flag= EscapesNameCreator.createName("escapes"); - str=str.substring(0,index)+flag+str.substring(index+1); - return replaceEscape(str,flag2Escapes); - } - public static void main(String[] args) { - String str = "34432\"fs"; - Map replaceEscape = new HashMap<>(16); - String value = replaceEscape(str, replaceEscape); - System.out.println(value); - } - /** * 替换特殊字符为替换字符串 * @@ -82,6 +63,22 @@ public static String replaceSpeciaSign(String str) { return str; } + public static String getConstant(String fieldName) { + if (StringUtil.isEmpty(fieldName)) { + return fieldName; + } + // fieldName = fieldName.trim(); + if (fieldName.startsWith("'") && fieldName.endsWith("'")) { + if (fieldName.equals("''")) { + return ""; + } else if (fieldName.equals("'")) { + return fieldName; + } + return fieldName.substring(1, fieldName.length() - 1); + } + return fieldName; + } + public static String restoreSpecialSign(String str) { for (String sign : CONSTANTS_REPLACE_SIGNS.keySet()) { str = str.replace(sign, CONSTANTS_REPLACE_SIGNS.get(sign)); @@ -181,7 +178,7 @@ protected static boolean isContantsStart(String expressionStr, String sign, int } public static String doConstantReplace(String expressionStr, Map flag2ExpressionStr, int flag, - List startFlags, List endFlags) { + List startFlags, List endFlags) { if (expressionStr.indexOf("'") == -1) { return expressionStr; } @@ -189,7 +186,7 @@ public static String doConstantReplace(String expressionStr, Map } public static String doConstantReplace(String expressionStr, Map flag2ExpressionStr, int flag, - List startFlags, List endFlags, boolean needReplaceSpecialSign) { + List startFlags, List endFlags, boolean needReplaceSpecialSign) { if (needReplaceSpecialSign) { expressionStr = replaceSpeciaSign(expressionStr); @@ -265,8 +262,12 @@ public static String doConstantReplace(String expressionStr, Map public static String createConsKey(int flag) { String tmp = flag + ""; if (flag < 10) { - tmp = "00" + tmp; + tmp = "0000" + tmp; } else if (flag < 100) { + tmp = "000" + tmp; + } else if (flag < 1000) { + tmp = "00" + tmp; + } else if (flag < 10000) { tmp = "0" + tmp; } return "constants_para_" + tmp; @@ -275,8 +276,12 @@ public static String createConsKey(int flag) { public static String createFlagKey(int flag) { String tmp = flag + ""; if (flag < 10) { - tmp = "00" + tmp; + tmp = "0000" + tmp; } else if (flag < 100) { + tmp = "000" + tmp; + } else if (flag < 1000) { + tmp = "00" + tmp; + } else if (flag < 10000) { tmp = "0" + tmp; } return "constants_" + tmp; @@ -346,7 +351,18 @@ protected static boolean isContantsEnd(String expressionStr, String sign, int i, return true; } - + public static void main(String[] args) { + String str = "splitarray('da''fsfds''ta','fdsdfs')"; + Map flag2ExpressionStr = new HashMap<>(16); + String value = doConstantReplace(str, flag2ExpressionStr, 1); + String[] values = value.split(","); + int i = 0; + for (String v : values) { + values[i] = restore(v, flag2ExpressionStr); + System.out.println(values[i]); + } + System.out.println(value); + } public static boolean containContant(String jsonValue) { if (isContant(jsonValue)) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DESUtils.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DESUtil.java similarity index 98% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DESUtils.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DESUtil.java index 7d202be7..89a45dfd 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DESUtils.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DESUtil.java @@ -29,13 +29,13 @@ import javax.crypto.SecretKeyFactory; import javax.crypto.spec.DESKeySpec; -public class DESUtils { +public class DESUtil { - public DESUtils() { + public DESUtil() { } public static String getSecretKey() throws Exception { - return getSecretKey((String)null); + return getSecretKey((String) null); } public static String getSecretKey(String seed) throws Exception { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DataTypeUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DataTypeUtil.java index bdad069b..cdb52936 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DataTypeUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DataTypeUtil.java @@ -30,8 +30,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.configurable.IConfigurable; import org.apache.rocketmq.streams.common.datatype.ArrayDataType; import org.apache.rocketmq.streams.common.datatype.BaseDataType; @@ -58,11 +56,13 @@ import org.apache.rocketmq.streams.common.datatype.SetDataType; import org.apache.rocketmq.streams.common.datatype.ShortDataType; import org.apache.rocketmq.streams.common.datatype.StringDataType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @SuppressWarnings("rawtypes") public class DataTypeUtil { - private static final Log LOG = LogFactory.getLog(DataTypeUtil.class); + private static final Logger LOG = LoggerFactory.getLogger(DataTypeUtil.class); private static final List dataTypes = new ArrayList<>(); /** @@ -75,7 +75,6 @@ public class DataTypeUtil { static { register(new HllDataType()); - register(new NumberDataType()); register(new StringDataType()); register(new IntDataType()); register(new LongDataType()); @@ -85,15 +84,16 @@ public class DataTypeUtil { register(new FloatDataType()); register(new ByteDataType()); register(new ShortDataType()); + register(new NumberDataType()); register(new ListDataType()); register(new SetDataType()); register(new ArrayDataType()); register(new MapDataType()); register(new ConfigurableDataType()); register(new JsonableDataType()); + register(new DateTimeDataType()); register(new JavaBeanDataType()); register(new SerializableDataType()); - register(new DateTimeDataType()); register(new NotSupportDataType()); } @@ -365,21 +365,22 @@ public static DataType createFieldDataType(Object object, String fieldName) { public static DataType createFieldDataType(Class clazz, String fieldName) { Method method = ReflectUtil.getGetMethod(clazz, fieldName); - Type type =null; + Type type = null; + String typeString = null; if (method == null) { try { Field field = clazz.getDeclaredField(fieldName); type = field.getType(); } catch (NoSuchFieldException e) { - throw new RuntimeException(clazz.getName() + "." + fieldName+" not exist get method, please create get/set method for the field"); + throw new RuntimeException(clazz.getName() + "." + fieldName + " not exist get method, please create get/set method for the field"); } - }else { + } else { type = method.getGenericReturnType(); - } - String typeString = type.toString(); - if (typeString.startsWith("class ")) { - typeString = null; + typeString = type.toString(); + if (typeString.startsWith("class ")) { + typeString = null; + } } return createDataType(ReflectUtil.getBeanFieldType(clazz, fieldName), typeString); @@ -558,15 +559,24 @@ public static boolean isString(DataType dataType) { /** * Gets the Datatype based on the fields of the class * - * @param o * @param field * @return */ - public static DataType createDataTypeByField(Object o, Field field) { + public static DataType createDataTypeByField(Object object, Field field) { + return createDataTypeByFieldFromClass(object.getClass(), field); + } + + /** + * Gets the Datatype based on the fields of the class + * + * @param field + * @return + */ + public static DataType createDataTypeByFieldFromClass(Class cal, Field field) { Class clazz = field.getType(); Method method = null; try { - method = ReflectUtil.getGetMethod(o.getClass(), field.getName()); + method = ReflectUtil.getGetMethod(cal, field.getName()); } catch (Exception e) { //todo nothing,if not has get/is method, use fieldDirectly } @@ -580,17 +590,13 @@ public static DataType createDataTypeByField(Object o, Field field) { genericTypeStr = field.getGenericType().toString(); } if (NotSupportDataType.class.isInstance(dataType)) { - Object object1 = ReflectUtil.getDeclaredField(o, field.getName()); - if (object1 != null) { - //如果是接口类,则通过值获取类型 - dataType = DataTypeUtil.getDataTypeFromClass(object1.getClass()); - if (genericTypeStr != null) { - int startIndex = genericTypeStr.indexOf("<"); - if (startIndex > -1) { - genericTypeStr = dataType.getDataClass().getName() + genericTypeStr.substring(startIndex); - } + Class fieldClass = ReflectUtil.getBeanFieldType(cal, field.getName()); + dataType = DataTypeUtil.getDataTypeFromClass(fieldClass); + if (genericTypeStr != null) { + int startIndex = genericTypeStr.indexOf("<"); + if (startIndex > -1) { + genericTypeStr = dataType.getDataClass().getName() + genericTypeStr.substring(startIndex); } - } } if (NotSupportDataType.class.isInstance(dataType)) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DateUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DateUtil.java index d716e0d3..abdd947d 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DateUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DateUtil.java @@ -27,8 +27,6 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.TimeUnit; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; public final class DateUtil { @@ -39,9 +37,12 @@ public final class DateUtil { public static final String DEFAULT_FORMAT = "yyyy-MM-dd HH:mm:ss"; public static final String SIMPLE_SECOND_FORMAT = "yyyyMMddHHmmss"; public static final String TIMESTAMP_FORMAT = "yyyyMMddHHmmssSSS"; + public static final long DAY_MILLISECONDS = 24 * 60 * 60 * 1000; private static final ThreadLocal> tl = new ThreadLocal>(); - - private static final Log LOG = LogFactory.getLog(DateUtil.class); + private static final int WORKTIME_DAY_OF_WEEK_BEGIN = 1; + private static final int WORKTIME_DAY_OF_WEEK_END = 5; + private static final int WORKTIME_HOUR_OF_DAY_BEGIN = 9; + private static final int WORKTIME_HOUR_OF_DAY_END = 18; /** * 给指定时间增加一个时间值 @@ -126,8 +127,6 @@ public static boolean equals(Date d1, Date d2) { return d1.getTime() == d2.getTime(); } - public static final long DAY_MILLISECONDS = 24 * 60 * 60 * 1000; - public static int dayDiff(Date d1, Date d2) { long timeDiff = d1.getTime() - d2.getTime(); if (timeDiff < 0) { @@ -279,11 +278,6 @@ public static int getWeekOfYear(Date date) { return calendar.get(Calendar.WEEK_OF_YEAR); } - private static final int WORKTIME_DAY_OF_WEEK_BEGIN = 1; - private static final int WORKTIME_DAY_OF_WEEK_END = 5; - private static final int WORKTIME_HOUR_OF_DAY_BEGIN = 9; - private static final int WORKTIME_HOUR_OF_DAY_END = 18; - /** * 判断是否是工作时间,工作时间为周一到周五(9:00-18:00) * @@ -453,6 +447,10 @@ public static List getWindowBeginTime(long eventTime, long slideInterval, } } + public static Date getWindowBeginTime(long eventTime, long sizeInterval) { + return getWindowBeginTime(eventTime, sizeInterval, sizeInterval).get(0); + } + /** * get the window start for a timestamp. * diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DipperThreadLocalUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DipperThreadLocalUtil.java index 895c8e19..8c246f53 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DipperThreadLocalUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DipperThreadLocalUtil.java @@ -25,7 +25,7 @@ public static void set(T t) { } public static T get() { - return (T)threadLocal.get(); + return (T) threadLocal.get(); } public static void remove() { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ENVUtile.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ENVUtil.java similarity index 90% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ENVUtile.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ENVUtil.java index 1b7e087e..2adeb8cd 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ENVUtile.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ENVUtil.java @@ -16,11 +16,10 @@ */ package org.apache.rocketmq.streams.common.utils; -import java.util.Iterator; import java.util.Map; import java.util.Properties; -public class ENVUtile { +public class ENVUtil { public static String getENVParameter(String key) { if (StringUtil.isEmpty(key)) { @@ -46,8 +45,7 @@ public static String getENVParameter(String key, Properties properties, boolean public static void main(String[] args) { Map map = System.getenv(); - for (Iterator itr = map.keySet().iterator(); itr.hasNext(); ) { - String key = itr.next(); + for (String key : map.keySet()) { System.out.println(key + "=" + map.get(key)); } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/FileUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/FileUtil.java index 0c5b3cbe..fbb75a19 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/FileUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/FileUtil.java @@ -26,9 +26,9 @@ import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; -import java.io.OutputStream; import java.net.JarURLConnection; import java.net.MalformedURLException; +import java.net.URISyntaxException; import java.net.URL; import java.net.URLConnection; import java.util.ArrayList; @@ -37,25 +37,16 @@ import java.util.jar.JarEntry; import java.util.jar.JarFile; import org.apache.commons.io.FileUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -//import org.apache.http.client.config.RequestConfig; -//import org.apache.http.client.methods.CloseableHttpResponse; -//import org.apache.http.client.methods.HttpGet; -//import org.apache.http.impl.client.CloseableHttpClient; -//import org.apache.http.impl.client.HttpClients; -//import org.apache.log4j.lf5.util.StreamUtils; import org.apache.rocketmq.streams.common.interfaces.ILineMessageProcessor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class FileUtil { - private static final Log LOG = LogFactory.getLog(FileUtil.class); - public static final String LINE_SIGN = System.getProperty("line.separator"); - public static final String LOCAL_FILE_HEADER = "file:"; - public static final String CLASS_PATH_FILE_HEADER = "classpath://"; + private static final Logger LOG = LoggerFactory.getLogger(FileUtil.class); /** * 查找指定的文件 @@ -88,7 +79,7 @@ public static File createFileSupportResourceFile(String fileUrl) { public static String getJarPath() { String path = FileUtil.class.getProtectionDomain().getCodeSource().getLocation().getPath(); if (path.endsWith(".jar")) { - int startInde = path.lastIndexOf(File.separator); + int startInde = path.lastIndexOf("."); path = path.substring(0, startInde); return path; } @@ -106,7 +97,10 @@ private static URL getJarFileURL(String fileUrl) { } if (fileUrl.startsWith(CLASS_PATH_FILE_HEADER)) { fileUrl = fileUrl.replaceFirst(CLASS_PATH_FILE_HEADER, ""); - URL url = PropertiesUtils.class.getClassLoader().getResource(fileUrl); + URL url = PropertiesUtil.class.getClassLoader().getResource(fileUrl); + if (url == null) { + return null; + } String path = url.toString(); if (path.indexOf("jar:file") != -1) { return url; @@ -318,13 +312,18 @@ public static File findFile(File targetFile, String fileName, boolean supportReg * @return */ public static File getResourceFile(String propertiesPath) { - URL url = PropertiesUtils.class.getClassLoader().getResource(propertiesPath); + URL url = PropertiesUtil.class.getClassLoader().getResource(propertiesPath); if (url == null) { LOG.error("can not load component's properties file " + propertiesPath); return null; } - return new File(url.getPath()); + try { + return new File(url.toURI()); + } catch (URISyntaxException e) { + e.printStackTrace(); + throw new RuntimeException(e); + } } protected static boolean isMatch(String oriFileName, String matchedFileName, boolean supportRegex) { @@ -554,6 +553,12 @@ public static boolean write(String fileName, List rows, boolean isAppend return write(file, rows, isAppend); } + public static boolean write(String filePath, String row, boolean isAppend) { + List rows = new ArrayList<>(); + rows.add(row); + return write(filePath, rows, isAppend); + } + public static boolean write(File file, List rows, boolean isAppend) { BufferedWriter bw = null; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/IOUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/IOUtil.java index 660c9deb..bc23a0d9 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/IOUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/IOUtil.java @@ -38,7 +38,7 @@ public static String in2Str(InputStream in, String encoding) { int tmp = -1; char temp; while ((tmp = reader.read()) != -1) { - temp = (char)tmp; + temp = (char) tmp; sb.append(temp); } } catch (Exception e) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/IPUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/IPUtil.java index c029c19b..9b346357 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/IPUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/IPUtil.java @@ -21,18 +21,16 @@ import java.util.Enumeration; import java.util.regex.Matcher; import java.util.regex.Pattern; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class IPUtil { - private static final Log LOG = LogFactory.getLog(IPUtil.class); - - private static volatile InetAddress LOCAL_ADDRESS = null; - private static final Pattern IP_PATTERN = Pattern.compile("\\d{1,3}(\\.\\d{1,3}){3,5}$"); - public static final String LOCALHOST = "127.0.0.1"; public static final String ANYHOST = "0.0.0.0"; + private static final Logger LOG = LoggerFactory.getLogger(IPUtil.class); + private static final Pattern IP_PATTERN = Pattern.compile("\\d{1,3}(\\.\\d{1,3}){3,5}$"); + private static volatile InetAddress LOCAL_ADDRESS = null; public static String getLocalIdentification() { return getLocalIP(); @@ -172,13 +170,19 @@ public static boolean isInRange(String ip, String cidr) { * @return 存在返回true,否则返回false */ public static boolean ipInSection(String ip, String ipSection) { - if (ipSection == null) { throw new NullPointerException("IP段不能为空!"); } - if (ip == null) { throw new NullPointerException("IP不能为空!"); } + if (ipSection == null) { + throw new NullPointerException("IP段不能为空!"); + } + if (ip == null) { + throw new NullPointerException("IP不能为空!"); + } ipSection = ipSection.trim(); ip = ip.trim(); final String REGX_IP = "((25[0-5]|2[0-4]\\d|1\\d{2}|[1-9]\\d|\\d)\\.){3}(25[0-5]|2[0-4]\\d|1\\d{2}|[1-9]\\d|\\d)"; final String REGX_IPB = REGX_IP + "\\-" + REGX_IP; - if (!ipSection.matches(REGX_IPB) || !ip.matches(REGX_IP)) { return false; } + if (!ipSection.matches(REGX_IPB) || !ip.matches(REGX_IP)) { + return false; + } int idx = ipSection.indexOf('-'); String[] sips = ipSection.substring(0, idx).split("\\."); String[] sipe = ipSection.substring(idx + 1).split("\\."); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/IdUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/IdUtil.java new file mode 100644 index 00000000..8625c019 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/IdUtil.java @@ -0,0 +1,53 @@ +/* + * 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.utils; + +import java.net.InetAddress; + +public class IdUtil { + + public static String instanceId() { + String hostName = System.getProperty("HOSTNAME"); + if (hostName == null || hostName.isEmpty()) { + return RuntimeUtil.getDipperInstanceId(); + } + return hostName; + } + + public static String workerId() { + String hostName = System.getProperty("HOSTNAME"); + if (hostName == null || hostName.isEmpty()) { + return RuntimeUtil.getDipperInstanceIdWithoutPid(); + } + return hostName; + } + + public static int objectId(Object obj) { + return System.identityHashCode(obj); + } + + public static void main(String[] args) { + try { + InetAddress addr = InetAddress.getLocalHost(); + System.out.println(addr.getHostName().replaceAll("[^%|a-zA-Z0-9_-]+", "")); + System.out.println(addr.getHostName()); + } catch (Exception e) { + e.printStackTrace(); + } + } + +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/InstantiationUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/InstantiationUtil.java index 18382304..df76fa4f 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/InstantiationUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/InstantiationUtil.java @@ -16,12 +16,9 @@ */ package org.apache.rocketmq.streams.common.utils; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; import java.io.ObjectStreamClass; import java.util.HashMap; import org.nustaq.serialization.FSTConfiguration; @@ -40,7 +37,7 @@ public class InstantiationUtil { * @return */ public static byte[] serializeObject(Object o) { - return conf.asByteArray(o); + return conf.asByteArray(o); // try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); // ObjectOutputStream oos = new ObjectOutputStream(baos)) { // oos.writeObject(o); @@ -61,7 +58,7 @@ public static byte[] serializeObject(Object o) { * @throws ClassNotFoundException */ public static T deserializeObject(byte[] bytes) { - return (T)conf.asObject(bytes); + return (T) conf.asObject(bytes); // // final ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); // try { @@ -79,9 +76,22 @@ public static T deserializeObject(byte[] bytes) { } public static class ClassLoaderObjectInputStream extends ObjectInputStream { - protected final ClassLoader classLoader; private static final HashMap> primitiveClasses = new HashMap(9); + static { + primitiveClasses.put("boolean", Boolean.TYPE); + primitiveClasses.put("byte", Byte.TYPE); + primitiveClasses.put("char", Character.TYPE); + primitiveClasses.put("short", Short.TYPE); + primitiveClasses.put("int", Integer.TYPE); + primitiveClasses.put("long", Long.TYPE); + primitiveClasses.put("float", Float.TYPE); + primitiveClasses.put("double", Double.TYPE); + primitiveClasses.put("void", Void.TYPE); + } + + protected final ClassLoader classLoader; + public ClassLoaderObjectInputStream(InputStream in, ClassLoader classLoader) throws IOException { super(in); this.classLoader = classLoader; @@ -95,7 +105,7 @@ protected Class resolveClass(ObjectStreamClass desc) throws IOException, Clas try { return Class.forName(name, false, this.classLoader); } catch (ClassNotFoundException var5) { - Class cl = (Class)primitiveClasses.get(name); + Class cl = (Class) primitiveClasses.get(name); if (cl != null) { return cl; } else { @@ -106,17 +116,5 @@ protected Class resolveClass(ObjectStreamClass desc) throws IOException, Clas return super.resolveClass(desc); } } - - static { - primitiveClasses.put("boolean", Boolean.TYPE); - primitiveClasses.put("byte", Byte.TYPE); - primitiveClasses.put("char", Character.TYPE); - primitiveClasses.put("short", Short.TYPE); - primitiveClasses.put("int", Integer.TYPE); - primitiveClasses.put("long", Long.TYPE); - primitiveClasses.put("float", Float.TYPE); - primitiveClasses.put("double", Double.TYPE); - primitiveClasses.put("void", Void.TYPE); - } } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/Ip2LongUtils.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/Ip2LongUtil.java similarity index 95% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/Ip2LongUtils.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/Ip2LongUtil.java index 74e38bb9..d4387e22 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/Ip2LongUtils.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/Ip2LongUtil.java @@ -16,29 +16,10 @@ */ package org.apache.rocketmq.streams.common.utils; -public class Ip2LongUtils { - - public Long ipDotDec2Long(String ip, Boolean ignoreError) { - if (ip == null || ip.isEmpty()) { - return null; - } - - if (!ipCheck(ip)) { - if (ignoreError == true) { - return null; - } else { - if ("000.000.000.000".equals(ip)) { - return 0L; - } - throw new IllegalArgumentException("illegal ip : " + ip); - } - } - - return ipDotDec2Long(ip); - } +public class Ip2LongUtil { public static String ipInt2DotDec(int ipInt) throws IllegalArgumentException { - return ipLong2DotDec(((long)ipInt) & 0xFFFFFFFFL); + return ipLong2DotDec(((long) ipInt) & 0xFFFFFFFFL); } public static String ipLong2DotDec(long ipLong) throws IllegalArgumentException { @@ -49,13 +30,13 @@ public static String ipLong2DotDec(long ipLong) throws IllegalArgumentException } StringBuffer sb = new StringBuffer(15); - sb.append((int)(ipLong >>> 24)); + sb.append((int) (ipLong >>> 24)); sb.append('.'); - sb.append((int)((ipLong & 0x00FFFFFF) >>> 16)); + sb.append((int) ((ipLong & 0x00FFFFFF) >>> 16)); sb.append('.'); - sb.append((int)((ipLong & 0x0000FFFF) >>> 8)); + sb.append((int) ((ipLong & 0x0000FFFF) >>> 8)); sb.append('.'); - sb.append((int)((ipLong & 0x000000FF))); + sb.append((int) ((ipLong & 0x000000FF))); return sb.toString(); } @@ -163,7 +144,7 @@ public static int ipDotDec2Int(String ip) { public static long ipDotDec2Long(String ipDotDec) throws IllegalArgumentException { int intValue = ipDotDec2Int(ipDotDec); - return ((long)intValue) & 0xFFFFFFFFL; + return ((long) intValue) & 0xFFFFFFFFL; } public static String ipHex2DotDec(String ipHex) throws IllegalArgumentException { @@ -293,4 +274,23 @@ static int validate2(String text, int fromIndex, int endIndex) { return value; } + public Long ipDotDec2Long(String ip, Boolean ignoreError) { + if (ip == null || ip.isEmpty()) { + return null; + } + + if (!ipCheck(ip)) { + if (ignoreError == true) { + return null; + } else { + if ("000.000.000.000".equals(ip)) { + return 0L; + } + throw new IllegalArgumentException("illegal ip : " + ip); + } + } + + return ipDotDec2Long(ip); + } + } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/JarUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/JarUtil.java index c7d5ad23..0a7b39c1 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/JarUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/JarUtil.java @@ -109,7 +109,7 @@ public static void change(JarFile jarFile, Map jarFilePath2Conte } private static void writeFile(List lists, - JarFile jarFile, Map jarFilePath2Content) throws IOException { + JarFile jarFile, Map jarFilePath2Content) throws IOException { FileOutputStream fos = new FileOutputStream(jarFile.getName(), true); JarOutputStream jos = new JarOutputStream(fos); try { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/JsonableUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/JsonableUtil.java index 90856981..27d8ac53 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/JsonableUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/JsonableUtil.java @@ -33,6 +33,7 @@ public class JsonableUtil { private static final DataType stringDataType = new StringDataType(); private static final DataType listDataType = new ListDataType(List.class, stringDataType); private static final DataType mapDataType = new MapDataType(Map.class, stringDataType, stringDataType); + private static Gson gson = new GsonBuilder().disableHtmlEscaping().setPrettyPrinting().create(); public static String toJson(List value) { return listDataType.toDataJson(value); @@ -52,12 +53,11 @@ public static Map getMapData(String value) { return mapDataType.getData(value); } - private static Gson gson = new GsonBuilder().disableHtmlEscaping().setPrettyPrinting().create(); - public static String formatJson(JSONObject jsonObject) { String value = gson.toJson(jsonObject); return value; } + public static String formatJson(JSONArray jsonObject) { String value = gson.toJson(jsonObject); return value; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/KryoUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/KryoUtil.java index 538cb027..cbb644c3 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/KryoUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/KryoUtil.java @@ -16,14 +16,16 @@ */ package org.apache.rocketmq.streams.common.utils; - import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.UnsupportedEncodingException; +import java.lang.reflect.Constructor; +import java.util.concurrent.ConcurrentHashMap; import org.apache.commons.codec.binary.Base64; -import org.objenesis.strategy.StdInstantiatorStrategy; - -import java.io.*; +import sun.reflect.ReflectionFactory; /** * Kryo Utils @@ -37,7 +39,7 @@ public class KryoUtil { private static final ThreadLocal kryoLocal = new ThreadLocal() { @Override protected Kryo initialValue() { - Kryo kryo = new Kryo(); + Kryo kryo = new Kryox(); /** * 不要轻易改变这里的配置!更改之后,序列化的格式就会发生变化, @@ -211,4 +213,57 @@ public static T readObjectFromString(String str, Class clazz) { throw new IllegalStateException(e); } } + + public static class Kryox extends Kryo { + + private final ReflectionFactory REFLECTION_FACTORY = ReflectionFactory + .getReflectionFactory(); + + private final ConcurrentHashMap, Constructor> _constructors = new ConcurrentHashMap, Constructor>(); + + @Override + public T newInstance(Class type) { + try { + return super.newInstance(type); + } catch (Exception e) { + return (T) newInstanceFromReflectionFactory(type); + } + } + + private Object newInstanceFrom(Constructor constructor) { + try { + return constructor.newInstance(); + } catch (final Exception e) { + throw new RuntimeException(e); + } + } + + @SuppressWarnings("unchecked") + public T newInstanceFromReflectionFactory(Class type) { + Constructor constructor = _constructors.get(type); + if (constructor == null) { + constructor = newConstructorForSerialization(type); + Constructor saved = _constructors.putIfAbsent(type, constructor); + if (saved != null) { + constructor = saved; + } + + } + return (T) newInstanceFrom(constructor); + } + + private Constructor newConstructorForSerialization( + Class type) { + try { + Constructor constructor = REFLECTION_FACTORY + .newConstructorForSerialization(type, + Object.class.getDeclaredConstructor()); + constructor.setAccessible(true); + return constructor; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + } } \ No newline at end of file diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/LogParserUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/LogParserUtil.java index db9fdaa3..4817b62c 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/LogParserUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/LogParserUtil.java @@ -63,11 +63,6 @@ public static String parse(String log, String... spiltSigns) { return log; } - private static class PreCal { - boolean needContansts = false; - boolean needBrackets = false; - } - private static PreCal preCal(String log) { PreCal preCal = new PreCal(); for (int i = 0; i < log.length() - 1; i++) { @@ -86,10 +81,6 @@ public static String parseContantsBySign(String log, String sign, String prefix) return parseContantsBySign(log, sign, prefix, new IntegerValue(), new HashMap<>()); } - // public static String parseExpression(String log){ - // - // } - public static String parseContantsBySign(String log, String sign, String prefix, IntegerValue index, Map flags) { String tmp = log; @@ -117,6 +108,10 @@ public static String parseContantsBySign(String log, String sign, String prefix, return parseContantsBySign(log, sign, prefix, index, flags); } + // public static String parseExpression(String log){ + // + // } + public static String parseContants(String log) { return parseContants(log, new HashMap<>()); } @@ -226,6 +221,11 @@ public static Map parseExpression(String log, String sign) { return result; } + private static class PreCal { + boolean needContansts = false; + boolean needBrackets = false; + } + protected static class IntegerValue { private int i = 1; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/MapKeyUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/MapKeyUtil.java index 94a9e69c..56d2f7ce 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/MapKeyUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/MapKeyUtil.java @@ -17,15 +17,19 @@ package org.apache.rocketmq.streams.common.utils; import com.alibaba.fastjson.JSONObject; - -import java.util.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; public class MapKeyUtil { /** * 默认分隔符 */ - private static final String SIGN = ";"; + public static final String SIGN = ";"; public static String createKey(List keyItems) { return createKey(SIGN, keyItems); @@ -90,18 +94,18 @@ public static String[] splitKey(String key) { return key.split(SIGN); } - public static String getFirst(String key){ + public static String getFirst(String key) { String[] keys = splitKey(key); return keys[0]; } - public static String getLast(String key){ + public static String getLast(String key) { String[] keys = splitKey(key); assert keys.length >= 1 : "keys length must less than 1"; return keys[keys.length - 1]; } - public static String getByIndex(String key, int index){ + public static String getByIndex(String key, int index) { String[] keys = splitKey(key); assert keys.length >= index : "index must less than length"; return keys[index]; @@ -169,4 +173,17 @@ public static Map parseMap(String... mapStrs) { return result; } + public static String createKeyFromObject(Object[] ps) { + List values = new ArrayList<>(); + for (Object object : ps) { + if (object == null) { + values.add(""); + } else if (object instanceof String) { + values.add(object.toString()); + } else { + values.add(object.toString()); + } + } + return createKey(values); + } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/MessageUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/MessageUtil.java index 18d886b9..c2c98417 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/MessageUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/MessageUtil.java @@ -20,18 +20,18 @@ import com.alibaba.fastjson.JSONArray; import com.alibaba.fastjson.JSONObject; import java.util.List; -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.IMessage; import org.apache.rocketmq.streams.common.interfaces.IJsonobjectProcessor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class MessageUtil { - private static final Log LOG = LogFactory.getLog(MessageUtil.class); + private static final Logger LOG = LoggerFactory.getLogger(MessageUtil.class); public static JSONObject reprocessingSingleMessage(String data, boolean isJsonData, boolean isArrayData, - IJsonobjectProcessor messageProcessor) { + IJsonobjectProcessor messageProcessor) { JSONArray jsonArray = reprocessingMessage(data, isJsonData, isArrayData, messageProcessor); if (jsonArray == null || jsonArray.size() == 0) { return null; @@ -43,7 +43,7 @@ public static JSONObject reprocessingSingleMessage(String data, boolean isJsonDa } public static JSONArray reprocessingMessage(String data, boolean isJsonData, boolean isArrayData, - IJsonobjectProcessor messageProcessor) { + IJsonobjectProcessor messageProcessor) { try { return reprocessingMessage0(data, isJsonData, isArrayData, messageProcessor); } catch (Exception e) { @@ -56,7 +56,7 @@ public static JSONArray reprocessingMessage(String data, boolean isJsonData, boo @SuppressWarnings("unchecked") public static JSONArray reprocessingMessage0(String data, boolean isJsonData, boolean isArrayData, - IJsonobjectProcessor messageProcessor) { + IJsonobjectProcessor messageProcessor) { JSONObject jsonObject = null; JSONArray jsonArray = new JSONArray(); if (isArrayData) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PipelineHTMLUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PipelineHTMLUtil.java index 8ae00828..02d82866 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PipelineHTMLUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PipelineHTMLUtil.java @@ -19,18 +19,22 @@ import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.JSONObject; import java.util.List; -import java.util.logging.Filter; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; -import org.apache.rocketmq.streams.common.topology.metric.NotFireReason; +import org.apache.rocketmq.streams.common.channel.source.AbstractSource; +import org.apache.rocketmq.streams.common.channel.source.ISource; import org.apache.rocketmq.streams.common.topology.metric.StageGroup; -import org.apache.rocketmq.streams.common.topology.model.AbstractRule; +import org.apache.rocketmq.streams.common.topology.model.AbstractChainStage; import org.apache.rocketmq.streams.common.topology.model.AbstractStage; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; -import org.apache.rocketmq.streams.common.topology.stages.ScriptChainStage; public class PipelineHTMLUtil { - public static String createHTML(ChainPipeline pipeline){ - StringBuilder html=new StringBuilder("\n" + + public static String createHTML(ChainPipeline pipeline) { + try { + + } catch (Exception e) { + e.printStackTrace(); + } + StringBuilder html = new StringBuilder("\n" + "\t\n" + "\t\t\n" + "\t\tTest\n" + @@ -52,247 +56,305 @@ public static String createHTML(ChainPipeline pipeline){ "\t\n" + "\t"); - - List stageGroups= pipeline.getRootStageGroups(); - int i=100; - StringBuilder hideHtmlBuilder=new StringBuilder(); - String baseTab="\t\t"; - html.append(baseTab+"

    \n"); - html.append(baseTab+"\tCREATE TABLE "+pipeline.getMsgSourceName()+"\n"); - - html.append(baseTab+"\t\n"); - hideHtmlBuilder.append(baseTab+"\t
  1. \n"); - hideHtmlBuilder.append(baseTab+"\t\t"+pipeline.getCreateTableSQL()+"\n"); - hideHtmlBuilder.append(baseTab+"\t
  2. \n"); + List stageGroups = pipeline.getRootStageGroups(); + int i = 100; + StringBuilder hideHtmlBuilder = new StringBuilder(); + String baseTab = "\t\t"; + html.append(baseTab + "
      \n"); + html.append(baseTab + "\tCREATE TABLE " + pipeline.getMsgSourceName() + "\n"); + + html.append(baseTab + "\t\n"); + hideHtmlBuilder.append(baseTab + "\t
    1. \n"); + hideHtmlBuilder.append(baseTab + "\t\t" + pipeline.getCreateTableSQL() + "\n"); + hideHtmlBuilder.append(baseTab + "\t
    2. \n"); i++; - html.append(baseTab+"\t\n"); - hideHtmlBuilder.append(baseTab+"\t
    3. \n"); - JSONObject jsonObject=JSON.parseObject(pipeline.getSource().toJson()); + html.append(baseTab + "\t\n"); + hideHtmlBuilder.append(baseTab + "\t
    4. \n"); + if (pipeline == null || pipeline.getSource() == null) { + return ""; + } + JSONObject jsonObject = JSON.parseObject(pipeline.getSource().toJson()); jsonObject.remove("metaData"); - hideHtmlBuilder.append(baseTab+"\t\t"+JsonableUtil.formatJson(jsonObject)+"\n"); - hideHtmlBuilder.append(baseTab+"\t
    5. \n"); + jsonObject.remove("context"); + hideHtmlBuilder.append(baseTab + "\t\t" + JsonableUtil.formatJson(jsonObject) + "\n"); + hideHtmlBuilder.append(baseTab + "\t\n"); + + if (pipeline.getSource() instanceof AbstractSource) { + AbstractSource abstractSource = (AbstractSource) pipeline.getSource(); + i++; + + html.append(baseTab + "\t\n"); + hideHtmlBuilder.append(baseTab + "\t
    6. \n"); + if (pipeline == null || pipeline.getSource() == null) { + return ""; + } - html.append(hideHtmlBuilder.toString()); - html.append(baseTab+"\n
    \n"); - i=1000; - for(StageGroup stageGroup:stageGroups){ - String stageGroupHtml=createStageGroupHTML(stageGroup,i,2); - html.append(stageGroupHtml); - i=i+1000; - } - html.append("\t\n"); - html.append("\n"); - return html.toString(); - } + hideHtmlBuilder.append(baseTab + "\t\t启动时间:" + abstractSource.getSourceMetric().startTime() + "\n"); + hideHtmlBuilder.append(baseTab + "\t\n"); - private static String createStageGroupHTML(StageGroup group,int index,int tabSize) { - StringBuilder stringBuilder=new StringBuilder(); - StringBuilder hideHtmlBuilder=new StringBuilder(); - String baseTab=""; - for(int i=0;i\n"); - stringBuilder.append(baseTab+"\t"+group.getViewName()+"\n"); + i++; + html.append(baseTab + "\t\n"); + hideHtmlBuilder.append(baseTab + "\t
  3. \n"); + if (pipeline == null || pipeline.getSource() == null) { + return ""; + } - stringBuilder.append(baseTab+"\t\n"); - hideHtmlBuilder.append(baseTab+"\t
  4. \n"); - hideHtmlBuilder.append(baseTab+"\t\t"+group.getSql()+"\n"); - hideHtmlBuilder.append(baseTab+"\t
  5. \n"); - index++; + hideHtmlBuilder.append(baseTab + "\t\t第一条消息时间:" + abstractSource.getSourceMetric().firstMsgReceive() + "\n"); + hideHtmlBuilder.append(baseTab + "\t\n"); + i++; - if(group.getChildren().size()>0){ - stringBuilder.append(baseTab+"\t\n"); + html.append(baseTab + "\t\n"); + hideHtmlBuilder.append(baseTab + "\t
  6. \n"); + if (pipeline == null || pipeline.getSource() == null) { + return ""; + } - hideHtmlBuilder.append(baseTab+"\t
  7. \n"); - int i=0; - for(StageGroup subStageGroup:group.getChildren()){ - subStageGroup.calculateMetric(); - int subIndex=index*1000+(i*100); - hideHtmlBuilder.append(createStageGroupHTML(subStageGroup,subIndex,tabSize+2)); - i++; + hideHtmlBuilder.append(baseTab + "\t\t最新消息时间:" + abstractSource.getSourceMetric().lastMsgReceive() + "\n"); + hideHtmlBuilder.append(baseTab + "\t
  8. \n"); + + i++; + + html.append(baseTab + "\t\n"); + hideHtmlBuilder.append(baseTab + "\t
  9. \n"); + if (pipeline == null || pipeline.getSource() == null) { + return ""; } - hideHtmlBuilder.append(baseTab+"\t
  10. \n"); - index++; - } + hideHtmlBuilder.append(baseTab + "\t\t" + abstractSource.getSourceMetric().toProgress() + "\n"); + hideHtmlBuilder.append(baseTab + "\t\n"); + i++; + html.append(baseTab + "\t\n"); + hideHtmlBuilder.append(baseTab + "\t
  11. \n"); + if (pipeline == null || pipeline.getSource() == null) { + return ""; + } - if(group.getAllStages().size()>0){ - stringBuilder.append(baseTab+"\t\n"); + hideHtmlBuilder.append(baseTab + "\t\t" + abstractSource.getSourceMetric().toCost() + "\n"); + hideHtmlBuilder.append(baseTab + "\t
  12. \n"); + } - hideHtmlBuilder.append(baseTab+"\t
  13. \n"); - int i=0; - for(AbstractStage stage:group.getAllStages()){ - int subIndex=index*1000+(i*100); - hideHtmlBuilder.append(createStageHTML(stage,subIndex,tabSize+2)); + html.append(hideHtmlBuilder.toString()); + html.append(baseTab + "\n
\n"); + i = 1000; + for (StageGroup stageGroup : stageGroups) { + String stageGroupHtml = createStageGroupHTML(stageGroup, i, 2); + html.append(stageGroupHtml); + i = i + 1000; + } + html.append("\t\n"); + html.append("\n"); + return html.toString(); + } + + private static String createStageGroupHTML(StageGroup group, int index, int tabSize) { + StringBuilder stringBuilder = new StringBuilder(); + StringBuilder hideHtmlBuilder = new StringBuilder(); + String baseTab = ""; + for (int i = 0; i < tabSize; i++) { + baseTab += "\t"; + } + group.calculateMetric(); + stringBuilder.append(baseTab + "
    \n"); + stringBuilder.append(baseTab + "\t" + group.getViewName() + "\n"); + +// if(isSourceStageGroup){ +// stringBuilder.append(baseTab + "\t\n"); +// hideHtmlBuilder.append(baseTab + "\t
  1. \n"); +// hideHtmlBuilder.append(baseTab + "\t\t" + group.getSql() + "\n"); +// hideHtmlBuilder.append(baseTab + "\t
  2. \n"); +// index++; +// } + + if (group.getChildren().size() > 0) { + stringBuilder.append(baseTab + "\t\n"); + + hideHtmlBuilder.append(baseTab + "\t
  3. \n"); + int i = 0; + for (StageGroup subStageGroup : group.getChildren()) { + subStageGroup.calculateMetric(); + int subIndex = index * 1000 + (i * 100); + hideHtmlBuilder.append(createStageGroupHTML(subStageGroup, subIndex, tabSize + 2)); i++; } - hideHtmlBuilder.append(baseTab+"\t
  4. \n"); + hideHtmlBuilder.append(baseTab + "\t\n"); index++; - } + if (group.getAllStages().size() > 0) { + stringBuilder.append(baseTab + "\t\n"); - stringBuilder.append(baseTab+"\t\n"); - index++; + hideHtmlBuilder.append(baseTab + "\t
  5. \n"); + int i = 0; + for (AbstractStage stage : group.getAllStages()) { + int subIndex = index * 1000 + (i * 100); + hideHtmlBuilder.append(createStageHTML(stage, subIndex, tabSize + 2)); + i++; + } + hideHtmlBuilder.append(baseTab + "\t
  6. \n"); + index++; + } - stringBuilder.append(baseTab+"\t\n"); + stringBuilder.append(baseTab + "\t\n"); index++; - - stringBuilder.append(baseTab+"\t\n"); + stringBuilder.append(baseTab + "\t\n"); index++; + stringBuilder.append(baseTab + "\t\n"); index++; - - + index++; stringBuilder.append(hideHtmlBuilder.toString()); - stringBuilder.append(baseTab+"\n
\n"); + stringBuilder.append(baseTab + "\n\n"); return stringBuilder.toString(); } protected static String createStageHTML(AbstractStage stage, int index, int tabSize) { - StringBuilder stringBuilder=new StringBuilder(); - StringBuilder hideHtmlBuilder=new StringBuilder(); - String baseTab=""; - for(int i=0;i\n"); + +// stringBuilder.append(baseTab + "\t\n"); +// hideHtmlBuilder.append(baseTab + "\t
  • \n"); +// hideHtmlBuilder.append(baseTab + "\t\t" + stage.getSql() + "\n\n"); +// hideHtmlBuilder.append(baseTab + "\t
  • \n"); +// index++; + if ("org.apache.rocketmq.streams.stage.ShuffleSourceChainStage".equals(stage.getClass().getName())) { + + ISource source = ReflectUtil.getDeclaredField(stage, "shuffleSource"); + // stringBuilder.append(baseTab + "\tShufflerSource:" + stage.getLabel() + "\n"); + stringBuilder.append(baseTab + "\t\n"); + hideHtmlBuilder.append(baseTab + "\t
  • \n"); + JSONObject jsonObject = JSON.parseObject(source.toJson()); + jsonObject.remove("metaData"); + hideHtmlBuilder.append(baseTab + "\t\t" + JsonableUtil.formatJson(jsonObject) + "\n"); + hideHtmlBuilder.append(baseTab + "\t
  • \n"); + index++; + } else { +// if(OutputChainStage.class.isInstance(stage)){ +// stringBuilder.append(baseTab + "\tInsert:" + stage.getLabel() + "\n"); +// }else { +// stringBuilder.append(baseTab + "\t" + stage.getClass().getSimpleName().replace("ChainStage", "") + ":" + stage.getLabel() + "\n"); +// } + + stringBuilder.append(baseTab + "\t\n"); - stringBuilder.append(baseTab+"
      \n"); - stringBuilder.append(baseTab+"\t"+ stage.getClass().getSimpleName().replace("ChainStage","")+":"+stage.getLabel()+"\n"); - + hideHtmlBuilder.append(baseTab + "\t\t
    1. \n"); + hideHtmlBuilder.append(baseTab + "\t\t\t" + stage.getSql() + "\n\n"); + hideHtmlBuilder.append(baseTab + "\t\t
    2. \n"); + index++; - stringBuilder.append(baseTab+"\t\n"); - hideHtmlBuilder.append(baseTab+"\t
    3. \n"); - hideHtmlBuilder.append(baseTab+"\t\t"+stage.getSql()+"\n\n"); - hideHtmlBuilder.append(baseTab+"\t
    4. \n"); - index++; + } - if(ScriptChainStage.class.isInstance(stage)){ - ScriptChainStage scriptChainStage=(ScriptChainStage)stage; - stringBuilder.append(baseTab+"\t\n"); - hideHtmlBuilder.append(baseTab+"\t
    5. \n"); - hideHtmlBuilder.append(baseTab+"\t\t"+scriptChainStage.getScript().getValue()+"\n\n"); - hideHtmlBuilder.append(baseTab+"\t
    6. \n"); + if ("org.apache.rocketmq.streams.stage.ShuffleSourceChainStage".equals(stage.getClass().getName())) { + stringBuilder.append(baseTab + "\t\n"); + index++; + } else { + stringBuilder.append(baseTab + "\t\n"); index++; } - - if(FilterChainStage.class.isInstance(stage)){ - FilterChainStage filterChainStage=(FilterChainStage)stage; - if(filterChainStage.getRules()!=null&&filterChainStage.getRules().size()>0){ - AbstractRule rule=(AbstractRule) filterChainStage.getRules().get(0); - stringBuilder.append(baseTab+"\t\n"); - - hideHtmlBuilder.append(baseTab+"\t\t
    7. \n"); - hideHtmlBuilder.append(baseTab+"\t\t\t"+rule.toString()+"\n\n"); - hideHtmlBuilder.append(baseTab+"\t\t
    8. \n"); - index++; - - if(stage.getStageMetric().getInCount()>stage.getStageMetric().getOutCount()){ - stringBuilder.append(baseTab+"\t\n"); - - - hideHtmlBuilder.append(baseTab+"\t
    9. \n"); - hideHtmlBuilder.append(baseTab+"\t\t"+filterChainStage.getStageMetric().createNotFireReason()+"\n\n"); - hideHtmlBuilder.append(baseTab+"\t
    10. \n"); - - - index++; - } - - } - + if (!"org.apache.rocketmq.streams.stage.ShuffleOutputChainStage".equals(stage.getClass().getName()) && !"org.apache.rocketmq.streams.stage.ShuffleSourceChainStage".equals(stage.getClass().getName())) { + stringBuilder.append(baseTab + "\t\n"); + index++; } + if (!"org.apache.rocketmq.streams.stage.ShuffleSourceChainStage".equals(stage.getClass().getName())) { - stringBuilder.append(baseTab+"\t\n"); - index++; - - - - - - - - - stringBuilder.append(baseTab+"\t\n"); - index++; - - + stringBuilder.append(baseTab + "\t\n"); + index++; - stringBuilder.append(baseTab+"\t\n"); - index++; + stringBuilder.append(baseTab + "\t\n"); + index++; - stringBuilder.append(baseTab+"\t\n"); - index++; + stringBuilder.append(baseTab + "\t\n"); + index++; + } - stringBuilder.append(baseTab+"\t\n"); - index++; + if (FilterChainStage.class.isInstance(stage) && (stage.getStageMetric().getInCount() > stage.getStageMetric().getOutCount())) { + FilterChainStage filterChainStage = (FilterChainStage) stage; + stringBuilder.append(baseTab + "\t\n"); + hideHtmlBuilder.append(baseTab + "\t
    11. \n"); + hideHtmlBuilder.append(baseTab + "\t\t" + filterChainStage.getStageMetric().createNotFireReason() + "\n\n"); + hideHtmlBuilder.append(baseTab + "\t
    12. \n"); - stringBuilder.append(baseTab+"\t\n"); - index++; + index++; + } + if (CollectionUtil.isNotEmpty(stage.getNextStageLabels())) { + stringBuilder.append(baseTab + "\t\n"); + index++; + } stringBuilder.append(hideHtmlBuilder.toString()); - stringBuilder.append(baseTab+"\n
    \n"); + stringBuilder.append(baseTab + "\n\n"); return stringBuilder.toString(); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PrintUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PrintUtil.java index 7fa4ebd3..78df46dc 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PrintUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PrintUtil.java @@ -28,19 +28,60 @@ import java.util.Map; import java.util.Properties; import java.util.Set; -import org.apache.rocketmq.streams.common.component.ComponentCreator; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; import org.apache.rocketmq.streams.common.configurable.IConfigurable; import org.apache.rocketmq.streams.common.configurable.IFieldProcessor; import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; import org.apache.rocketmq.streams.common.datatype.DataType; import org.apache.rocketmq.streams.common.datatype.GenericParameterDataType; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; import org.apache.rocketmq.streams.common.topology.model.AbstractStage; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; public class PrintUtil { public static String LINE = System.getProperty("line.separator"); + private static boolean isSimpleModel = true; + private static Set excludeFieldNames = new HashSet<>(); + private static Map excludeDefalutValueMap = new HashMap<>(); + + static { + excludeFieldNames.add("dbChannelName"); + excludeFieldNames.add("entityName"); + excludeFieldNames.add("channelName"); + excludeFieldNames.add("indexs"); + excludeFieldNames.add("ruleName"); + excludeFieldNames.add("msgMetaDataName"); + excludeFieldNames.add("actionNames"); + excludeFieldNames.add("scriptNames"); + excludeFieldNames.add("varNames"); + excludeFieldNames.add("expressionName"); + excludeFieldNames.add("expressionStr"); + + excludeFieldNames.add("configureName"); + excludeFieldNames.add("nameSpace"); + excludeFieldNames.add("type"); + excludeFieldNames.add("version"); + excludeFieldNames.add("extendField"); + excludeDefalutValueMap.put("isJsonData", "true"); + excludeDefalutValueMap.put("msgIsJsonArray", "false"); + excludeDefalutValueMap.put("maxThread", "10"); + excludeDefalutValueMap.put("offset", "0"); + excludeDefalutValueMap.put("syncCount", "1000"); + excludeDefalutValueMap.put("syncTimeout", "60000"); + excludeDefalutValueMap.put("outputThreadCount", "-1"); + excludeDefalutValueMap.put("isAutoFlush", "false"); + excludeDefalutValueMap.put("batchSize", "6000"); + excludeDefalutValueMap.put("timeout", "60000"); + excludeDefalutValueMap.put("activtyTimeOut", "3000"); + excludeDefalutValueMap.put("startNow", "true"); + excludeDefalutValueMap.put("topic", ""); + excludeDefalutValueMap.put("pollingTime", "86400"); + excludeDefalutValueMap.put("closeSplitMode", "false"); + excludeDefalutValueMap.put("result", "while"); + excludeDefalutValueMap.put("ruleStatus", "3"); + excludeDefalutValueMap.put("isBatchMessage", "true"); + excludeDefalutValueMap.put("cancelAfterConfigurableRefreshListerner", "false"); + } public static void print(List list) { for (String str : list) { @@ -97,49 +138,6 @@ public static void print(JSONArray jsonArray) { } } - private static boolean isSimpleModel = true; - private static Set excludeFieldNames = new HashSet<>(); - private static Map excludeDefalutValueMap = new HashMap<>(); - - static { - excludeFieldNames.add("dbChannelName"); - excludeFieldNames.add("entityName"); - excludeFieldNames.add("channelName"); - excludeFieldNames.add("indexs"); - excludeFieldNames.add("ruleName"); - excludeFieldNames.add("msgMetaDataName"); - excludeFieldNames.add("actionNames"); - excludeFieldNames.add("scriptNames"); - excludeFieldNames.add("varNames"); - excludeFieldNames.add("expressionName"); - excludeFieldNames.add("expressionStr"); - - excludeFieldNames.add("configureName"); - excludeFieldNames.add("nameSpace"); - excludeFieldNames.add("type"); - excludeFieldNames.add("version"); - excludeFieldNames.add("extendField"); - excludeDefalutValueMap.put("isJsonData", "true"); - excludeDefalutValueMap.put("msgIsJsonArray", "false"); - excludeDefalutValueMap.put("maxThread", "10"); - excludeDefalutValueMap.put("offset", "0"); - excludeDefalutValueMap.put("syncCount", "1000"); - excludeDefalutValueMap.put("syncTimeout", "60000"); - excludeDefalutValueMap.put("outputThreadCount", "-1"); - excludeDefalutValueMap.put("isAutoFlush", "false"); - excludeDefalutValueMap.put("batchSize", "6000"); - excludeDefalutValueMap.put("timeout", "60000"); - excludeDefalutValueMap.put("activtyTimeOut", "3000"); - excludeDefalutValueMap.put("startNow", "true"); - excludeDefalutValueMap.put("topic", ""); - excludeDefalutValueMap.put("pollingTime", "86400"); - excludeDefalutValueMap.put("closeSplitMode", "false"); - excludeDefalutValueMap.put("result", "while"); - excludeDefalutValueMap.put("ruleStatus", "3"); - excludeDefalutValueMap.put("isBatchMessage", "true"); - excludeDefalutValueMap.put("cancelAfterConfigurableRefreshListerner", "false"); - } - public static String print(IConfigurable configurable, String... paras) { StringBuilder sb = new StringBuilder(); if (paras != null && paras.length > 0 && StringUtil.isNotEmpty(paras[0])) { @@ -168,12 +166,10 @@ public void doProcess(Object o, Field field) { if (isSimpleModel && excludeDefaultValue != null && excludeDefaultValue.equals(valueStr)) { return; } - if (field.getAnnotation(ENVDependence.class) != null && BasedConfigurable.class.isInstance( - configurable)) { - BasedConfigurable basedConfigurable = (BasedConfigurable)configurable; - String oriValue = basedConfigurable.getOriFieldValue(field, valueStr); + if (field.getAnnotation(ENVDependence.class) != null && configurable instanceof BasedConfigurable) { + String oriValue = valueStr; sb.append(field.getName() + "=" + oriValue + LINE); - String actualValue = ComponentCreator.getProperties().getProperty(oriValue); + String actualValue = (oriValue); if (StringUtil.isNotEmpty(actualValue)) { sb.append(field.getName() + "_mock_value=" + actualValue + LINE); } @@ -189,21 +185,20 @@ public void doProcess(Object o, Field field) { public static String getDataTypeStr(DataType dataType, Object value) { if (GenericParameterDataType.class.isInstance(dataType)) { - return ((GenericParameterDataType)dataType).toDataStr(value); + return ((GenericParameterDataType) dataType).toDataStr(value); } return dataType.toDataJson(value); } public static String print(ChainPipeline pipline) { JSONObject jsonObject = new JSONObject(); - jsonObject.put("namespace.name", pipline.getNameSpace() + "." + pipline.getConfigureName()); - jsonObject.put("sucessInit", pipline.isInitSuccess()); + jsonObject.put("namespace.name", pipline.getNameSpace() + "." + pipline.getName()); List stages = pipline.getStages(); StringBuilder stringBuilder = new StringBuilder(); stringBuilder.append(pipline.getChannelName()); for (AbstractStage stage : stages) { stringBuilder.append("--->"); - stringBuilder.append(stage.getConfigureName()); + stringBuilder.append(stage.getName()); } jsonObject.put("stages", stringBuilder.toString()); return JsonableUtil.formatJson(jsonObject); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PropertiesUtils.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PropertiesUtil.java similarity index 74% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PropertiesUtils.java rename to rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PropertiesUtil.java index 50707b59..29c177dc 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PropertiesUtils.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PropertiesUtil.java @@ -23,16 +23,15 @@ import java.io.IOException; import java.io.InputStreamReader; import java.net.URL; -import java.util.Iterator; import java.util.Map; import java.util.Properties; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class PropertiesUtils { - private static final Log LOG = LogFactory.getLog(PropertiesUtils.class); +public class PropertiesUtil { + private static final Logger LOG = LoggerFactory.getLogger(PropertiesUtil.class); - public static Properties getDefaultComponentProperties(Class clazz) { + public static Properties getDefaultComponentProperties(Class clazz) { return getResourceProperties(getComponentPropertyPath(clazz)); } @@ -52,20 +51,18 @@ public static void putProperty(String line, String sign, Map properties) { int startIndex = line.indexOf(sign); String key = line.substring(0, startIndex); String value = line.substring(startIndex + sign.length()); - properties.put(key, value); } /** * 加载指定位置的属性 * - * @param propertiesPath - * @return + * @param propertiesPath 资源路径 + * @return 资源对象 */ - public static Properties getResourceProperties(Class clazz, String propertiesPath) { + public static Properties getResourceProperties(Class clazz, String propertiesPath) { URL url = clazz.getClassLoader().getResource(propertiesPath); if (url == null) { - // LOG.error("can not load component's properties file " + propertiesPath); return null; } BufferedReader br = null; @@ -79,7 +76,9 @@ public static Properties getResourceProperties(Class clazz, String propertiesPat return null; } finally { try { - br.close(); + if (br != null) { + br.close(); + } } catch (IOException e) { e.printStackTrace(); } @@ -90,11 +89,11 @@ public static Properties getResourceProperties(Class clazz, String propertiesPat /** * 加载指定位置的属性 * - * @param propertiesPath - * @return + * @param propertiesPath 资源路径 + * @return 资源对象 */ public static Properties getResourceProperties(String propertiesPath) { - return getResourceProperties(PropertiesUtils.class, propertiesPath); + return getResourceProperties(PropertiesUtil.class, propertiesPath); } public static String getComponentPropertyPath(Class clazz) { @@ -102,27 +101,21 @@ public static String getComponentPropertyPath(Class clazz) { } public static void print(Properties properties) { - Iterator> it = properties.entrySet().iterator(); - while (it.hasNext()) { - Map.Entry entry = it.next(); + for (Map.Entry entry : properties.entrySet()) { System.out.println(entry.getKey() + ":" + entry.getValue()); } } - public static String getComponentSpringXmlPath(Class clazz) { - return getComponentPath(clazz, "xml"); - } - /** * 获取组件属性文件默认的相对路径及名称 * - * @param - * @return + * @param clazz 类 + * @param rear 类 + * @return 路径 */ - private static String getComponentPath(Class clazz, String rear) { + private static String getComponentPath(Class clazz, String rear) { String filePath = "component/" + clazz.getSimpleName(); - String propertyFilePath = filePath + "." + rear; - return propertyFilePath; + return filePath + "." + rear; } public static void flush(Properties properties, String outputFilePath) { @@ -131,17 +124,15 @@ public static void flush(Properties properties, String outputFilePath) { bw = new BufferedWriter(new FileWriter(outputFilePath)); properties.store(bw, null); bw.flush(); - ; } catch (IOException e) { e.printStackTrace(); } finally { - if (bw != null) { - try { + try { + if (bw != null) { bw.close(); - } catch (IOException e) { - e.printStackTrace(); } - ; + } catch (IOException e) { + e.printStackTrace(); } } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/RandomStrUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/RandomStrUtil.java index ab2ea36f..f2ca9cb9 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/RandomStrUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/RandomStrUtil.java @@ -20,12 +20,12 @@ public class RandomStrUtil { + private static final SecureRandom random = new SecureRandom(); private static char[] digits = - {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', + { + '0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o', 'p', 'q', 'r', 's', 't', 'u', 'v', 'w', 'x', 'y', 'z'}; - private static final SecureRandom random = new SecureRandom(); - public static String getRandomStr(int length) { StringBuffer sb = new StringBuffer(); for (int i = 0; i < length; i++) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ReflectUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ReflectUtil.java index 4de47d0e..7fcedb8e 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ReflectUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ReflectUtil.java @@ -24,15 +24,13 @@ import java.lang.reflect.Method; import java.lang.reflect.Modifier; import java.lang.reflect.ParameterizedType; +import java.lang.reflect.Type; import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; -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.BasedConfigurable; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.configurable.IConfigurable; import org.apache.rocketmq.streams.common.configurable.IFieldProcessor; import org.apache.rocketmq.streams.common.configurable.annotation.NoSerialized; import org.apache.rocketmq.streams.common.datatype.ArrayDataType; @@ -41,16 +39,19 @@ import org.apache.rocketmq.streams.common.datatype.MapDataType; import org.apache.rocketmq.streams.common.datatype.NotSupportDataType; import org.apache.rocketmq.streams.common.datatype.SetDataType; +import org.apache.rocketmq.streams.common.model.Entity; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * 类ReflectUtil的实现描述 */ @SuppressWarnings({"rawtypes", "unchecked"}) public class ReflectUtil { + public static final String SPLIT_SIGN = "\\."; // 及联调用的分隔符。如a.b.c private static final String AUTOWIRED_CLASS_NAME = "org.springframework.beans.factory.annotation.Autowired"; + private static final Logger LOG = LoggerFactory.getLogger(ReflectUtil.class); private static Class AUTOWIRED_CLASS = null; - private static final Log LOG = LogFactory.getLog(ReflectUtil.class); - public static final String SPLIT_SIGN = "\\."; // 及联调用的分隔符。如a.b.c static { try { @@ -61,19 +62,27 @@ public class ReflectUtil { } public static T getDeclaredField(Class clazz, Object object, String fieldName) { + String msg = fieldName; + if (object != null) { + msg = object.getClass().getName() + ":" + fieldName; + } + try { Method method = getGetMethod(clazz, fieldName); if (method != null) { return (T) method.invoke(object); } + Field field = clazz.getDeclaredField(fieldName); field.setAccessible(true); return (T) field.get(object); - } catch (Exception e) { - String msg = fieldName; - if (object != null) { - msg = object.getClass().getName() + ":" + fieldName; + } catch (NoSuchFieldException ee) { + Class parent = clazz.getSuperclass(); + if (parent != null) { + return getDeclaredField(parent, object, fieldName); } + throw new RuntimeException("get field value error " + msg, ee); + } catch (Exception e) { throw new RuntimeException("get field value error " + msg, e); } } @@ -139,8 +148,7 @@ public static Class forClass(String className) { } else if ("boolean".equals(className)) { return boolean.class; } - Class clazz = Class.forName(className); - return clazz; + return Class.forName(className); } catch (Exception e) { throw new RuntimeException("create class error " + className, e); } @@ -161,20 +169,20 @@ public static T forInstance(String className) { } public static Object deserializeObject(JSONObject jsonObject) { - String className = jsonObject.getString(IConfigurableService.CLASS_NAME); + String className = jsonObject.getString(IConfigurable.CLASS_NAME); if (StringUtil.isEmpty(className)) { throw new RuntimeException("can not Deserialize object ,the class name is empty"); } Object object = ReflectUtil.forInstance(className); scanFields(object, (o, field) -> { String fileName = field.getName(); - Class fieldClass=ReflectUtil.getBeanFieldType(o.getClass(),fileName); + Class fieldClass = ReflectUtil.getBeanFieldType(o.getClass(), fileName); DataType dataType = DataTypeUtil.getDataTypeFromClass(fieldClass); String genericTypeStr = null; if (field.getGenericType() != null) { genericTypeStr = field.getGenericType().toString(); } - if (NotSupportDataType.class.isInstance(dataType)) { + if (dataType instanceof NotSupportDataType) { Object object1 = ReflectUtil.getDeclaredField(o, field.getName()); if (object1 != null) { //如果是接口类,则通过值获取类型 @@ -188,7 +196,7 @@ public static Object deserializeObject(JSONObject jsonObject) { } } - setDataTypeParadigmType(dataType, genericTypeStr, ParameterizedType.class.isInstance(field.getGenericType())); + setDataTypeParadigmType(dataType, genericTypeStr, field.getGenericType() instanceof ParameterizedType); String fieldJson = jsonObject.getString(fileName); if (fieldJson == null) { return; @@ -196,9 +204,7 @@ public static Object deserializeObject(JSONObject jsonObject) { Object value = dataType.getData(fieldJson); try { if (value != null) { - ReflectUtil.setBeanFieldValue(object,field.getName(),value); -// field.setAccessible(true); -// field.set(object, value); + ReflectUtil.setBeanFieldValue(object, field.getName(), value); } } catch (Exception e) { throw new RuntimeException("Deserialize error ,the field " + fileName + " Deserialize error "); @@ -219,17 +225,17 @@ public static JSONObject serializeObject(Object o) { } JSONObject objectJson = new JSONObject(); - objectJson.put(IConfigurableService.CLASS_NAME, o.getClass().getName()); + objectJson.put(IConfigurable.CLASS_NAME, o.getClass().getName()); scanFields(o, (o1, field) -> { field.setAccessible(true); String fileName = field.getName(); - Class fieldClass=ReflectUtil.getBeanFieldType(o1.getClass(),fileName); + Class fieldClass = ReflectUtil.getBeanFieldType(o1.getClass(), fileName); DataType dataType = DataTypeUtil.getDataTypeFromClass(fieldClass); String genericTypeStr = null; if (field.getGenericType() != null) { genericTypeStr = field.getGenericType().toString(); } - if (NotSupportDataType.class.isInstance(dataType)) { + if (dataType instanceof NotSupportDataType) { Object object1 = ReflectUtil.getDeclaredField(o, field.getName()); if (object1 != null) { //如果是接口类,则通过值获取类型 @@ -243,9 +249,9 @@ public static JSONObject serializeObject(Object o) { } } - setDataTypeParadigmType(dataType, genericTypeStr, ParameterizedType.class.isInstance(field.getGenericType())); + setDataTypeParadigmType(dataType, genericTypeStr, field.getGenericType() instanceof ParameterizedType); try { - Object value = ReflectUtil.getDeclaredField(o1,fileName); + Object value = ReflectUtil.getDeclaredField(o1, fileName); if (value == null) { return; } @@ -271,7 +277,7 @@ public static void scanConfiguableFields(Object o, IFieldProcessor fieldProcesso } public static void scanConfiguableFields(Object o, Class clazz, IFieldProcessor fieldProcessor) { - scanFields(o, clazz, fieldProcessor, AbstractConfigurable.class); + scanFields(o, clazz, fieldProcessor, Entity.class); } public static void scanFields(Object o, IFieldProcessor fieldProcessor) { @@ -290,13 +296,13 @@ public static void deserializeObject(Object object, byte[] bytes) { @Override public void doProcess(Object o, Field field) { field.setAccessible(true); - Class fieldClass=ReflectUtil.getBeanFieldType(o.getClass(),field.getName()); + Class fieldClass = ReflectUtil.getBeanFieldType(o.getClass(), field.getName()); DataType dataType = DataTypeUtil.getDataTypeFromClass(fieldClass); String genericTypeStr = null; if (field.getGenericType() != null) { genericTypeStr = field.getGenericType().toString(); } - if (NotSupportDataType.class.isInstance(dataType)) { + if (dataType instanceof NotSupportDataType) { Object object1 = ReflectUtil.getDeclaredField(o, field.getName()); if (object1 != null) { //如果是接口类,则通过值获取类型 @@ -310,11 +316,11 @@ public void doProcess(Object o, Field field) { } } - setDataTypeParadigmType(dataType, genericTypeStr, ParameterizedType.class.isInstance(field.getGenericType())); + setDataTypeParadigmType(dataType, genericTypeStr, field.getGenericType() instanceof ParameterizedType); Object value = dataType.byteToValue(bytes, offset.get()); offset.addAndGet(dataType.toBytes(value, false).length); try { - ReflectUtil.setBeanFieldValue(object,field.getName(),value); + ReflectUtil.setBeanFieldValue(object, field.getName(), value); } catch (Exception e) { throw new RuntimeException("can not set field value " + field.getName(), e); } @@ -343,7 +349,7 @@ public void doProcess(Object o, Field field) { if (field.getGenericType() != null) { genericTypeStr = field.getGenericType().toString(); } - if (NotSupportDataType.class.isInstance(dataType)) { + if (dataType instanceof NotSupportDataType) { Object object1 = ReflectUtil.getDeclaredField(o, field.getName()); if (object1 != null) { //如果是接口类,则通过值获取类型 @@ -357,7 +363,7 @@ public void doProcess(Object o, Field field) { } } - setDataTypeParadigmType(dataType, genericTypeStr, ParameterizedType.class.isInstance(field.getGenericType())); + setDataTypeParadigmType(dataType, genericTypeStr, field.getGenericType() instanceof ParameterizedType); byte[] bytes = new byte[0]; try { @@ -394,25 +400,25 @@ public void doProcess(Object o, Field field) { * @param genericType */ public static void setDataTypeParadigmType(DataType dataType, String genericType, boolean isGenericType) { - if (ListDataType.class.isInstance(dataType)) { + if (dataType instanceof ListDataType) { String genericTypeStr = "java.util.List"; if (genericType != null && ParameterizedType.class.isInstance(genericType)) { genericTypeStr = DataTypeUtil.convertGenericParameterString(genericType); } ((ListDataType) dataType).parseGenericParameter(genericTypeStr); - } else if (SetDataType.class.isInstance(dataType)) { + } else if (dataType instanceof SetDataType) { String genericTypeStr = "java.util.Set"; if (genericType != null && isGenericType) { genericTypeStr = DataTypeUtil.convertGenericParameterString(genericType); } ((SetDataType) dataType).parseGenericParameter(genericTypeStr); - } else if (MapDataType.class.isInstance(dataType)) { + } else if (dataType instanceof MapDataType) { String genericTypeStr = "java.util.Map"; if (genericType != null && isGenericType) { genericTypeStr = DataTypeUtil.convertGenericParameterString(genericType); } ((MapDataType) dataType).parseGenericParameter(genericTypeStr); - } else if (ArrayDataType.class.isInstance(dataType)) { + } else if (dataType instanceof ArrayDataType) { String genericTypeStr = "java.lang.String[]"; if (genericType != null && isGenericType) { genericTypeStr = DataTypeUtil.convertGenericParameterString(genericType); @@ -427,6 +433,9 @@ public static void setDataTypeParadigmType(DataType dataType, String genericType * @param fieldProcessor */ public static void scanFields(Object o, Class clazz, IFieldProcessor fieldProcessor, Class basedClass) { + if (clazz == null) { + return; + } if (basedClass.getName().equals(clazz.getName())) { return; } @@ -448,8 +457,8 @@ public static void scanFields(Object o, Class clazz, IFieldProcessor fieldProces continue; } fieldProcessor.doProcess(o, field); - }catch (Exception e){ - throw new RuntimeException("Error serializing object, class is "+clazz.getName()+", field is "+field.getName(),e); + } catch (Exception e) { + throw new RuntimeException("Error serializing object, class is " + clazz.getName() + ", field is " + field.getName(), e); } } @@ -543,6 +552,12 @@ public static Object invoke(Object object, String methodName, Class[] classes, O * @return 返回结果 */ public static T getBeanFieldOrJsonValue(Object bean, String fieldName) { + if (Map.class.isInstance(bean)) { + Object object = ((Map) bean).get(fieldName); + if (object != null) { + return (T) object; + } + } String[] fieldNames = fieldName.split(SPLIT_SIGN); Object result = null; Object modelBean = bean;// ChannelMessage @@ -550,21 +565,21 @@ public static T getBeanFieldOrJsonValue(Object bean, String fieldName) { if (modelBean == null) { return null; } - if (JSONObject.class.isInstance(modelBean)) { + if (modelBean instanceof JSONObject) { result = getJsonItemValue(modelBean, name); - } else if (JSONArray.class.isInstance(modelBean)) { + } else if (modelBean instanceof JSONArray) { result = getJsonItemValue(modelBean, name); - } else if (Map.class.isInstance(modelBean)) { + } else if (modelBean instanceof Map) { result = ((Map) modelBean).get(name); - } else if(String.class.isInstance(modelBean)){ - String value=(String)modelBean; + } else if (modelBean instanceof String) { + String value = (String) modelBean; if (value.startsWith("[") && value.endsWith("]")) { modelBean = JSON.parseArray(value); } else { modelBean = JSON.parseObject(value); } - result=getJsonItemValue(modelBean,name); - }else { + result = getJsonItemValue(modelBean, name); + } else { result = getDeclaredField(modelBean, name);// ChannelMessage.messageBody } modelBean = result; @@ -573,20 +588,20 @@ public static T getBeanFieldOrJsonValue(Object bean, String fieldName) { } private static T getJsonItemValue(Object bean, String fieldName) { - if (fieldName.indexOf("[") == -1 && JSONObject.class.isInstance(bean)) { + if (!fieldName.contains("[") && bean instanceof JSONObject) { JSONObject jsonObject = (JSONObject) bean; Object value = jsonObject.get(fieldName); return (T) value; } - if (fieldName.indexOf("[") != -1) { - if (JSONObject.class.isInstance(bean)) { + if (fieldName.contains("[")) { + if (bean instanceof JSONObject) { JSONObject jsonObject = (JSONObject) bean; int startIndex = fieldName.indexOf("["); String word = fieldName.substring(0, startIndex); JSONArray jsonArray = jsonObject.getJSONArray(word); String lastStr = fieldName.substring(startIndex); return getArrayIndexValue(jsonArray, lastStr); - } else if (JSONArray.class.isInstance(bean)) { + } else if (bean instanceof JSONArray) { JSONArray jsonArray = (JSONArray) bean; int startIndex = fieldName.indexOf("["); String lastStr = fieldName.substring(startIndex); @@ -605,7 +620,7 @@ private static T getArrayIndexValue(JSONArray jsonArray, String indexDescrip if (indexStr.equals("*")) { return (T) jsonArray.toJSONString(); } - int index = Integer.valueOf(indexStr); + int index = Integer.parseInt(indexStr); indexDescription = indexDescription.substring(endIndex + 1); startIndex = indexDescription.indexOf("["); if (startIndex == -1) { @@ -705,7 +720,6 @@ public static void setBeanFieldValue(Object object, String modelFieldName, Objec } } catch (Exception e) { - e.printStackTrace(); throw new RuntimeException("setBeanFieldValue error, the modelFieldName is " + modelFieldName + "; the object is " + object + "; the fieldValue is " + fieldValue + ", the field value class is " + fieldValue.getClass().getName(), e); } } @@ -721,15 +735,22 @@ public static Class getBeanFieldType(Class clazz, String modelFieldName) { String[] fieldNames = modelFieldName.split(SPLIT_SIGN); Class aClass = clazz; Class result = null; - for (String fieldName : fieldNames) { - Method method = getGetMethod(aClass, fieldName); - if (method == null) { - return null; + try { + for (String fieldName : fieldNames) { + Method method = getGetMethod(aClass, fieldName); + if (method == null) { + Field field = clazz.getDeclaredField(fieldName); + Type type = field.getType(); + return Class.forName(((Class) type).getTypeName()); + } + result = getFieldType(aClass, fieldName); + aClass = result; } - result = getFieldType(aClass, fieldName); - aClass = result; + return result; + } catch (Exception e) { + throw new RuntimeException("get field Type error " + clazz.getName() + "." + modelFieldName, e); } - return result; + } public static boolean isJavaBean(Class clazz) { @@ -762,10 +783,10 @@ public static boolean isJavaBean(Class clazz) { * @return */ private static T getFieldValue(Object bean, String fieldName) { - if(bean==null){ + if (bean == null) { return null; } - return (T)getDeclaredField(bean, fieldName); + return (T) getDeclaredField(bean, fieldName); } private static Class getFieldType(Class clazz, String modelFieldName) { @@ -783,9 +804,7 @@ public static Method getSetMethod(Class aClass, String modelFieldName, Class< } try { aClass1 = DataTypeUtil.getDataTypeFromClass(aClass1).getDataClass(); - Method method = findMethodFromClass(aClass, setMethodName, new Class[] {aClass1}); - - return method; + return findMethodFromClass(aClass, setMethodName, new Class[] {aClass1}); } catch (Exception e) { LOG.error(aClass.getSimpleName() + "\r\n" + e.getMessage(), e); throw new RuntimeException(aClass.getSimpleName() + "\r\n" + e.getMessage() + " ,modelFieldName: " + modelFieldName, e); @@ -862,10 +881,7 @@ private static String getIsMethodGetName(Class clazz, String methodName, String */ public static Boolean existMethodName(Class clazz, String methodName) { Method method = getMethodByName(clazz, methodName); - if (method != null) { - return true; - } - return false; + return method != null; } /** @@ -892,8 +908,7 @@ public static Method getMethodByName(Class clazz, String methodName) { * @return */ private static String getGetMethodName(String fieldName) { - String methodName = "get" + fieldName.substring(0, 1).toUpperCase() + fieldName.substring(1); - return methodName; + return "get" + fieldName.substring(0, 1).toUpperCase() + fieldName.substring(1); } /** @@ -906,13 +921,11 @@ private static String getIsMethodName(String fieldName) { if (fieldName.startsWith("is")) { return fieldName; } - String methodName = "is" + fieldName.substring(0, 1).toUpperCase() + fieldName.substring(1); - return methodName; + return "is" + fieldName.substring(0, 1).toUpperCase() + fieldName.substring(1); } private static String getSetMethodName(String fieldName) { - String setMethodName = "set" + fieldName.substring(0, 1).toUpperCase() + fieldName.substring(1); - return setMethodName; + return "set" + fieldName.substring(0, 1).toUpperCase() + fieldName.substring(1); } /** @@ -925,8 +938,7 @@ private static String getIsSetMethodName(String fieldName) { if (fieldName.startsWith("is")) { fieldName = fieldName.substring(2); } - String methodName = "set" + fieldName.substring(0, 1).toUpperCase() + fieldName.substring(1); - return methodName; + return "set" + fieldName.substring(0, 1).toUpperCase() + fieldName.substring(1); } public static Object forInstance(Class clazz, Class[] classes, Object[] objects) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/RocketmqUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/RocketmqUtil.java new file mode 100644 index 00000000..4e434e95 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/RocketmqUtil.java @@ -0,0 +1,352 @@ +/* + * 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.utils; + +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.rocketmq.acl.common.AclClientRPCHook; +import org.apache.rocketmq.acl.common.SessionCredentials; +import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer; +import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; +import org.apache.rocketmq.client.consumer.MessageQueueListener; +import org.apache.rocketmq.client.consumer.listener.MessageListenerOrderly; +import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.client.producer.DefaultMQProducer; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.common.consumer.ConsumeFromWhere; +import org.apache.rocketmq.common.message.MessageQueue; +import org.apache.rocketmq.common.protocol.body.KVTable; +import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig; +import org.apache.rocketmq.remoting.RPCHook; +import org.apache.rocketmq.remoting.protocol.LanguageCode; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.command.CommandUtil; + +public class RocketmqUtil { + + public static DefaultMQProducer initDefaultMQProducer(String namesrvAddr, String instanceName, String produceGroup) { + return initDefaultMQProducer(false, null, null, namesrvAddr, instanceName, produceGroup, null, null); + } + + public static DefaultMQProducer initDefaultMQProducer(boolean aclEnable, String accessKey, String secretKey, String namesrvAddr, String instanceName, String produceGroup, Integer msgTimeout, Integer maxMessageSize) { + RPCHook rpcHook = null; + if (aclEnable) { + rpcHook = new AclClientRPCHook(new SessionCredentials(accessKey, secretKey)); + } + DefaultMQProducer producer = new DefaultMQProducer(rpcHook); + producer.setNamesrvAddr(namesrvAddr); + producer.setInstanceName(instanceName); + if (produceGroup != null) { + producer.setProducerGroup(produceGroup); + } + if (msgTimeout != null) { + producer.setSendMsgTimeout(msgTimeout); + } + if (maxMessageSize != null) { + producer.setMaxMessageSize(maxMessageSize); + } + producer.setLanguage(LanguageCode.JAVA); + producer.setEnableBackpressureForAsyncMode(true); + return producer; + } + + public static DefaultLitePullConsumer initDefaultMQPullConsumer(String namesrvAddr, String instanceName, String consumerGroup, String topic) { + return initDefaultMQPullConsumer(namesrvAddr, instanceName, consumerGroup, topic, null); + } + + public static DefaultLitePullConsumer initDefaultMQPullConsumer(String namesrvAddr, String instanceName, String consumerGroup, String topic, ConsumeFromWhere consumeFromWhere) { + return initDefaultMQPullConsumer(false, null, null, namesrvAddr, instanceName, consumerGroup, topic, consumeFromWhere, null); + } + + public static DefaultLitePullConsumer initDefaultMQPullConsumer(boolean aclEnable, String accessKey, String secretKey, String namesrvAddr, String instanceName, String consumerGroup, String topic, ConsumeFromWhere consumeFromWhere, + Long messageConsumeTimeout) { + RPCHook rpcHook = null; + if (aclEnable) { + rpcHook = new AclClientRPCHook(new SessionCredentials(accessKey, secretKey)); + } + try { + DefaultLitePullConsumer consumer = new DefaultLitePullConsumer(consumerGroup, rpcHook); + consumer.setNamesrvAddr(namesrvAddr); + consumer.setInstanceName(instanceName); + if (consumeFromWhere != null) { + consumer.setConsumeFromWhere(consumeFromWhere); + } + if (messageConsumeTimeout != null) { + consumer.setConsumerPullTimeoutMillis(messageConsumeTimeout); + } + consumer.subscribe(topic, "*"); + consumer.setLanguage(LanguageCode.JAVA); + MessageQueueListener origin = consumer.getMessageQueueListener(); + MessageListenerDelegator delegator = new MessageListenerDelegator(origin); + consumer.setMessageQueueListener(delegator); + return consumer; + } catch (Exception e) { + throw new RuntimeException("create pull consumer: " + instanceName + " failed", e); + } + } + + public static DefaultMQPushConsumer initDefaultMQPushConsumer(String namesrvAddr, String instanceName, String consumerGroup, String topic) { + return initDefaultMQPushConsumer(false, null, null, namesrvAddr, instanceName, consumerGroup, null, null, null, ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET, topic); + } + + public static DefaultMQPushConsumer initDefaultMQPushConsumer(boolean aclEnable, String accessKey, String secretKey, String namesrvAddr, String instanceName, String consumerGroup, Integer maxReconsumeTimes, Long messageConsumeTimeout, + Integer consumeThreadMin, ConsumeFromWhere where, String topic) { + RPCHook rpcHook = null; + if (aclEnable) { + rpcHook = new AclClientRPCHook(new SessionCredentials(accessKey, secretKey)); + } + DefaultMQPushConsumer consumer = new DefaultMQPushConsumer(rpcHook); + try { + consumer.setNamesrvAddr(namesrvAddr); + consumer.setInstanceName(instanceName); + consumer.setConsumerGroup(consumerGroup); + if (maxReconsumeTimes != null) { + consumer.setMaxReconsumeTimes(maxReconsumeTimes); + } + if (messageConsumeTimeout != null) { + consumer.setConsumeTimeout(messageConsumeTimeout); + } + if (consumeThreadMin != null) { + consumer.setConsumeThreadMin(consumeThreadMin); + } + if (where != null) { + consumer.setConsumeFromWhere(where); + if (where.equals(ConsumeFromWhere.CONSUME_FROM_TIMESTAMP)) { + consumer.setConsumeTimestamp(UtilAll.timeMillisToHumanString3(System.currentTimeMillis())); + } + } + consumer.subscribe(topic, "*"); + consumer.setLanguage(LanguageCode.JAVA); + consumer.registerMessageListener((MessageListenerOrderly) (list, context) -> null); + } catch (MQClientException e) { + throw new RuntimeException("create push consumer: " + instanceName + " failed", e); + } + + return consumer; + } + + public static String createSubGroup(boolean aclEnable, String accessKey, String secretKey, String namesrvAddr, String adminExtGroup, String instanceName, String subGroup, String clusterName) { + DefaultMQAdminExt defaultMQAdminExt = null; + try { + defaultMQAdminExt = startMQAdminTool(aclEnable, accessKey, secretKey, namesrvAddr, adminExtGroup, instanceName); + SubscriptionGroupConfig initConfig = new SubscriptionGroupConfig(); + initConfig.setGroupName(subGroup); + + Set masterSet = CommandUtil.fetchMasterAddrByClusterName(defaultMQAdminExt, clusterName); + for (String addr : masterSet) { + defaultMQAdminExt.createAndUpdateSubscriptionGroupConfig(addr, initConfig); + } + } catch (Exception e) { + throw new IllegalArgumentException("create subGroup: " + subGroup + " failed", e); + } finally { + if (defaultMQAdminExt != null) { + defaultMQAdminExt.shutdown(); + } + } + return subGroup; + } + + public static String createTopic(String namesrvAddr, String topic, Integer splitNum, String cluster, Boolean isOrder) { + return createTopic(false, null, null, namesrvAddr, null, null, topic, splitNum, cluster, isOrder); + } + + public static String createTopic(Boolean aclEnable, String accessKey, String secretKey, String namesrvAddr, String adminExtGroup, String instanceName, String topic, Integer splitNum, String clusterName, Boolean isOrder) { + DefaultMQAdminExt defaultMQAdminExt = null; + try { + defaultMQAdminExt = startMQAdminTool(aclEnable, accessKey, secretKey, namesrvAddr, adminExtGroup, instanceName); + TopicConfig topicConfig = new TopicConfig(); + topicConfig.setReadQueueNums(splitNum); + topicConfig.setWriteQueueNums(splitNum); + topicConfig.setTopicName(topic.trim()); + topicConfig.setOrder(isOrder); + + Set masterSet = CommandUtil.fetchMasterAddrByClusterName(defaultMQAdminExt, clusterName); + for (String addr : masterSet) { + defaultMQAdminExt.createAndUpdateTopicConfig(addr, topicConfig); + } + } catch (Exception e) { + throw new IllegalArgumentException("create subGroup: " + topic + " failed", e); + } finally { + if (defaultMQAdminExt != null) { + defaultMQAdminExt.shutdown(); + } + } + return topic; + } + + public static List getConsumerList(String namesrvAddr, String topic, String consumerGroup) { + DefaultMQAdminExt defaultMQAdminExt = null; + try { + defaultMQAdminExt = startMQAdminTool(false, null, null, namesrvAddr, null, null); + return defaultMQAdminExt.getDefaultMQAdminExtImpl().getMqClientInstance().findConsumerIdList(topic, consumerGroup); + } catch (Exception e) { + e.printStackTrace(); + } finally { + if (defaultMQAdminExt != null) { + defaultMQAdminExt.shutdown(); + } + } + return null; + } + + public static void putKeyConfig(String namespace, String key, String value, String namesrvAddr) { + DefaultMQAdminExt defaultMQAdminExt = null; + try { + defaultMQAdminExt = startMQAdminTool(false, null, null, namesrvAddr, null, null); + defaultMQAdminExt.createAndUpdateKvConfig(namespace, key, value); + } catch (Exception e) { + throw new RuntimeException("put key config error", e); + } finally { + if (defaultMQAdminExt != null) { + defaultMQAdminExt.shutdown(); + } + } + } + + public static String getKeyConfig(String namespace, String key, String namesrvAddr) { + DefaultMQAdminExt defaultMQAdminExt = null; + try { + defaultMQAdminExt = startMQAdminTool(false, null, null, namesrvAddr, null, null); + return defaultMQAdminExt.getKVConfig(namespace, key); + } catch (Exception e) { + return null; + } finally { + if (defaultMQAdminExt != null) { + defaultMQAdminExt.shutdown(); + } + } + } + + public static void deleteKeyConfig(String namespace, String key, String namesrvAddr) { + DefaultMQAdminExt defaultMQAdminExt = null; + try { + defaultMQAdminExt = startMQAdminTool(false, null, null, namesrvAddr, null, null); + defaultMQAdminExt.deleteKvConfig(namespace, key); + } catch (Exception e) { + e.printStackTrace(); + } finally { + if (defaultMQAdminExt != null) { + defaultMQAdminExt.shutdown(); + } + } + } + + public static KVTable getKVListByNameSpace(String namespace, String namesrvAddr) { + DefaultMQAdminExt defaultMQAdminExt = null; + try { + defaultMQAdminExt = startMQAdminTool(false, null, null, namesrvAddr, null, null); + return defaultMQAdminExt.getKVListByNamespace(namespace); + } catch (Exception e) { + return null; + } finally { + if (defaultMQAdminExt != null) { + defaultMQAdminExt.shutdown(); + } + } + } + + public static void resetOffsetNew(String namesrvAddr, String consumerGroup, String topic, long timestamp) { + DefaultMQAdminExt defaultMQAdminExt = null; + try { + defaultMQAdminExt = startMQAdminTool(false, null, null, namesrvAddr, null, null); + defaultMQAdminExt.resetOffsetByTimestamp(topic, consumerGroup, timestamp, true); + } catch (Exception e) { + System.err.printf("[%s] not exists, reset error%n", consumerGroup); + } finally { + if (defaultMQAdminExt != null) { + defaultMQAdminExt.shutdown(); + } + } + } + + public static DefaultMQAdminExt startMQAdminTool(boolean aclEnable, String accessKey, String secretKey, String namesrvAddr, String adminExtGroup, String instanceName) throws MQClientException { + RPCHook rpcHook = null; + if (aclEnable) { + rpcHook = new AclClientRPCHook(new SessionCredentials(accessKey, secretKey)); + } + DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt(rpcHook); + defaultMQAdminExt.setNamesrvAddr(namesrvAddr); + if (adminExtGroup != null) { + defaultMQAdminExt.setAdminExtGroup(adminExtGroup); + } + if (instanceName != null) { + defaultMQAdminExt.setInstanceName(instanceName); + } + + defaultMQAdminExt.setMqClientApiTimeout(30000); + defaultMQAdminExt.start(); + return defaultMQAdminExt; + } + +} + +class MessageListenerDelegator implements MessageQueueListener { + private final MessageQueueListener delegator; + private final Set lastDivided = new HashSet<>(); + private final Set removingQueue = new HashSet<>(); + private final AtomicBoolean needSync = new AtomicBoolean(false); + private final Object mutex = new Object(); + + public MessageListenerDelegator(MessageQueueListener delegator) { + this.delegator = delegator; + } + + @Override + public void messageQueueChanged(String topic, Set mqAll, Set mqDivided) { + //上一次分配有,但是这一次没有,需要对这些mq进行状态移除 + for (MessageQueue last : lastDivided) { + if (!mqDivided.contains(last)) { + removingQueue.add(last); + } + } + + this.lastDivided.clear(); + this.lastDivided.addAll(mqDivided); + + needSync.set(true); + delegator.messageQueueChanged(topic, mqAll, mqDivided); + + synchronized (this.mutex) { + this.mutex.notifyAll(); + } + } + + public Set getLastDivided() { + return Collections.unmodifiableSet(this.lastDivided); + } + + public Set getRemovingQueue() { + return Collections.unmodifiableSet(this.removingQueue); + } + + public boolean needSync() { + return this.needSync.get(); + } + + public void hasSynchronized() { + this.needSync.set(false); + } + + public Object getMutex() { + return mutex; + } +} \ No newline at end of file diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/RuntimeUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/RuntimeUtil.java index 22d85aa5..c125b630 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/RuntimeUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/RuntimeUtil.java @@ -51,9 +51,11 @@ public static String getUserDir() { * @return */ public static String getDipperInstanceId() { - return MapKeyUtil.createKeyBySign("_", IPUtil.getLocalIdentification(), getPid()).replaceAll("\\.", "_"); + } + public static String getDipperInstanceIdWithoutPid() { + return MapKeyUtil.createKeyBySign("_", IPUtil.getLocalIdentification()).replaceAll("\\.", "_"); } public static void main(String[] args) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SQLFormatterUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SQLFormatterUtil.java index b0f4515d..8ff05468 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SQLFormatterUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SQLFormatterUtil.java @@ -23,13 +23,15 @@ public class SQLFormatterUtil { + public static final String WHITESPACE = " \n\r\f\t"; + static final String indentString = " "; + static final String initial = "\n "; private static final Set BEGIN_CLAUSES = new HashSet(); private static final Set END_CLAUSES = new HashSet(); private static final Set LOGICAL = new HashSet(); private static final Set QUANTIFIERS = new HashSet(); private static final Set DML = new HashSet(); private static final Set MISC = new HashSet(); - public static final String WHITESPACE = " \n\r\f\t"; static { BEGIN_CLAUSES.add("left"); @@ -70,9 +72,6 @@ public class SQLFormatterUtil { MISC.add("on"); } - static final String indentString = " "; - static final String initial = "\n "; - public String format(String source) { return new FormatProcess(source).perform(); } @@ -87,16 +86,14 @@ private static class FormatProcess { boolean afterInsert = false; int inFunction = 0; int parensSinceSelect = 0; - private LinkedList parenCounts = new LinkedList(); - private LinkedList afterByOrFromOrSelects = new LinkedList(); - int indent = 0; - StringBuilder result = new StringBuilder(); StringTokenizer tokens; String lastToken; String token; String lcToken; + private LinkedList parenCounts = new LinkedList(); + private LinkedList afterByOrFromOrSelects = new LinkedList(); public FormatProcess(String sql) { tokens = new StringTokenizer( @@ -106,6 +103,21 @@ public FormatProcess(String sql) { ); } + private static boolean isFunctionName(String tok) { + final char begin = tok.charAt(0); + final boolean isIdentifier = Character.isJavaIdentifierStart(begin) || '"' == begin; + return isIdentifier && + !LOGICAL.contains(tok) && + !END_CLAUSES.contains(tok) && + !QUANTIFIERS.contains(tok) && + !DML.contains(tok) && + !MISC.contains(tok); + } + + private static boolean isWhitespace(String token) { + return WHITESPACE.indexOf(token) >= 0; + } + public String perform() { //result.append( initial ); @@ -330,21 +342,6 @@ private void openParen() { parensSinceSelect++; } - private static boolean isFunctionName(String tok) { - final char begin = tok.charAt(0); - final boolean isIdentifier = Character.isJavaIdentifierStart(begin) || '"' == begin; - return isIdentifier && - !LOGICAL.contains(tok) && - !END_CLAUSES.contains(tok) && - !QUANTIFIERS.contains(tok) && - !DML.contains(tok) && - !MISC.contains(tok); - } - - private static boolean isWhitespace(String token) { - return WHITESPACE.indexOf(token) >= 0; - } - private void newline() { result.append("\n"); for (int i = 0; i < indent; i++) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SQLUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SQLUtil.java index 7bce3cb5..7116cf1a 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SQLUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SQLUtil.java @@ -40,15 +40,14 @@ public class SQLUtil { /** * 创建 insert into duplicate 格式的语句 + * * @param metaData * @param rows - * @return - * eg : - * insert into table_20210710000000(ds, `value`, data_time) values('1', '2', '2021-09-05 00:00:01') on duplicate key update - * ds = values(ds), `value` = values(`value`), data_time = values(data_time); - * + * @return eg : + * insert into table_20210710000000(ds, `value`, data_time) values('1', '2', '2021-09-05 00:00:01') on duplicate key update + * ds = values(ds), `value` = values(`value`), data_time = values(data_time); */ - public static String createInsertWithDuplicateKeyUpdateSql(MetaData metaData, List> rows){ + public static String createInsertWithDuplicateKeyUpdateSql(MetaData metaData, List> rows) { StringBuilder sb = new StringBuilder(); sb.append(createInsertSegment(metaData, false)); @@ -66,30 +65,29 @@ public static String createInsertWithDuplicateKeyUpdateSql(MetaData metaData, Li * * @param metaData * @param isDuplicateKey - * @return - * eg: `field1`, `field2`, `field3` - * or - * `field1` = values(`field1`), `field2` = values(`field2`), `field3` = values(`field3`) + * @return eg: `field1`, `field2`, `field3` + * or + * `field1` = values(`field1`), `field2` = values(`field2`), `field3` = values(`field3`) */ - private static String createMetaDataFieldSegment(MetaData metaData, boolean containsIdField, boolean isDuplicateKey){ + private static String createMetaDataFieldSegment(MetaData metaData, boolean containsIdField, boolean isDuplicateKey) { List fields = metaData.getMetaDataFields(); StringBuilder sb = new StringBuilder(); boolean isFirst = true; String idName = metaData.getIdFieldName(); - for(MetaDataField field : fields){ - if(!containsIdField && field.getFieldName().equals(idName)){ + for (MetaDataField field : fields) { + if (!containsIdField && field.getFieldName().equals(idName)) { continue; } - if(isFirst){ + if (isFirst) { isFirst = false; - }else{ + } else { sb.append(","); } sb.append("`"); sb.append(field.getFieldName()); sb.append("`"); - if(isDuplicateKey){ + if (isDuplicateKey) { sb.append("="); sb.append("values"); sb.append("("); @@ -104,28 +102,27 @@ private static String createMetaDataFieldSegment(MetaData metaData, boolean cont /** * 创建 " insert into table() " 片段, 不包括后面的values; + * * @param metaData * @param isContainsId 是否包含id字段 - * @return - * insert into table_20210710000000(ds, `value`, data_time) + * @return insert into table_20210710000000(ds, `value`, data_time) */ - public static String createInsertSegment(MetaData metaData, boolean isContainsId){ + public static String createInsertSegment(MetaData metaData, boolean isContainsId) { return createPrefixSegment(INSERT, metaData, isContainsId); } /** * 创建 " insert into table() " 片段, 不包括后面的values; + * * @param metaData * @param isContainsId 是否包含id字段 - * @return - * insert into table_20210710000000(ds, `value`, data_time) + * @return insert into table_20210710000000(ds, `value`, data_time) */ - public static String createInsertIgnoreSegment(MetaData metaData, boolean isContainsId){ + public static String createInsertIgnoreSegment(MetaData metaData, boolean isContainsId) { return createPrefixSegment(INSERT_IGNORE, metaData, isContainsId); } - - private static String createPrefixSegment(String prefix, MetaData metaData, boolean isContainsId){ + private static String createPrefixSegment(String prefix, MetaData metaData, boolean isContainsId) { StringBuilder sb = new StringBuilder(); sb.append(prefix); sb.append(" "); @@ -138,28 +135,29 @@ private static String createPrefixSegment(String prefix, MetaData metaData, bool /** * 返回values('a', b, c) + * * @param metaData * @param rows * @param containsIdField 是否包含id字段 * @return */ - public static String createValuesSegment(MetaData metaData, List> rows, boolean containsIdField){ + public static String createValuesSegment(MetaData metaData, List> rows, boolean containsIdField) { - if(rows == null || rows.size() == 0){ + if (rows == null || rows.size() == 0) { return null; } MetaData tmpMetaData = metaData; //如果不包含主键id, 则过滤id字段 - if(!containsIdField){ + if (!containsIdField) { tmpMetaData = MetaDataUtils.getMetaDataWithOutId(metaData); } StringBuilder sb = new StringBuilder(); sb.append("values"); boolean isFirstValue = true; for (Map row : rows) { - if(isFirstValue){ + if (isFirstValue) { isFirstValue = false; - }else{ + } else { sb.append(","); } String value = createValuesSegment(tmpMetaData, row); @@ -168,7 +166,7 @@ public static String createValuesSegment(MetaData metaData, List fieldName2Value){ + private static String createValuesSegment(MetaData metaData, Map fieldName2Value) { StringBuilder valueSql = new StringBuilder(); boolean isFirst = true; valueSql.append("("); @@ -191,14 +189,12 @@ private static String createValuesSegment(MetaData metaData, Map } /** - * * @param metaData * @param containsIdField - * @return - * eg : - * on duplicate key update ds = values(ds), `value` = values(`value`), data_time = values(data_time) + * @return eg : + * on duplicate key update ds = values(ds), `value` = values(`value`), data_time = values(data_time) */ - public static String createDuplicateKeyUpdateSegment(MetaData metaData, boolean containsIdField){ + public static String createDuplicateKeyUpdateSegment(MetaData metaData, boolean containsIdField) { StringBuilder sb = new StringBuilder(); sb.append(DUPLICATE_KEY); sb.append(" "); @@ -566,6 +562,7 @@ public static String createInSql(Collection collection) { /** * create multi like sentences + * * @param keywordList * @return */ diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ScheduleUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ScheduleUtil.java index 4014944d..ab72ecdd 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ScheduleUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ScheduleUtil.java @@ -26,7 +26,7 @@ public static IScheduleExecutor convert(IStreamOperator receiver) { if (!(receiver instanceof IConfigurable)) { throw new RuntimeException("can not convert to IScheduleExecutor, need function Iconfiguable interface"); } - IConfigurable configurable = (IConfigurable)receiver; + IConfigurable configurable = (IConfigurable) receiver; return new IScheduleExecutor() { @Override public void doExecute() throws InterruptedException { @@ -34,8 +34,8 @@ public void doExecute() throws InterruptedException { } @Override - public String getConfigureName() { - return configurable.getConfigureName(); + public String getName() { + return configurable.getName(); } @Override diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SerializeUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SerializeUtil.java index e6240d80..557064ef 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SerializeUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SerializeUtil.java @@ -25,27 +25,27 @@ import java.lang.reflect.Array; import java.util.Optional; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.datatype.ArrayDataType; import org.apache.rocketmq.streams.common.datatype.DataType; import org.apache.rocketmq.streams.common.datatype.StringDataType; import org.apache.rocketmq.streams.common.interfaces.ISerialize; -import org.nustaq.serialization.FSTConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class SerializeUtil { - private static final Log LOG = LogFactory.getLog(SerializeUtil.class); - /** + private static final Logger LOG = LoggerFactory.getLogger(SerializeUtil.class); + + /** * 把一个对象序列化成字节,对象中的字段是datatype支持的 * * @param object * @return */ public static byte[] serialize(Object object) { - if(ISerialize.class.isInstance(object)){ + if (ISerialize.class.isInstance(object)) { // byte[] bytes = conf.asByteArray(object); // return bytes; - return KryoUtil.writeObjectToByteArray(object); + return KryoUtil.writeObjectToByteArray(object); } DataType dataType = DataTypeUtil.getDataTypeFromClass(object.getClass()); if (ArrayDataType.class.isInstance(dataType)) { @@ -83,18 +83,20 @@ public static T deserialize(byte[] bytes, AtomicInteger offset) { DataType dataType = DataTypeUtil.getDataType(dataTypeName); return (T) dataType.byteToValue(bytes, offset); } + /** * 把一个对象的字段,通过字节填充,字段不能有null值 * * @param bytes */ - public static T deserialize(byte[] bytes,Class clazz) { - if(ISerialize.class.isAssignableFrom(clazz)){ - return (T)KryoUtil.readObjectFromByteArray(bytes,clazz); + public static T deserialize(byte[] bytes, Class clazz) { + if (ISerialize.class.isAssignableFrom(clazz)) { + return (T) KryoUtil.readObjectFromByteArray(bytes, clazz); //return (T)conf.asObject(bytes); } return deserialize(bytes, new AtomicInteger(0)); } + /** * 把一个对象的字段,通过字节填充,字段不能有null值 * diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ServiceLoadUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ServiceLoadUtil.java index 77696f86..7d5b57a4 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ServiceLoadUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ServiceLoadUtil.java @@ -25,25 +25,26 @@ public class ServiceLoadUtil { - private static ICache serviceCache=new SoftReferenceCache<>(); - public static T loadService(Class interfaceClass,String serviceName){ - List allService = (List)serviceCache.get(interfaceClass.getName()); - if(allService==null){ - allService=new ArrayList<>(); + private static ICache serviceCache = new SoftReferenceCache<>(); + + public static T loadService(Class interfaceClass, String serviceName) { + List allService = (List) serviceCache.get(interfaceClass.getName()); + if (allService == null) { + allService = new ArrayList<>(); Iterable iterable = ServiceLoader.load(interfaceClass); for (T t : iterable) { allService.add(t); } serviceCache.put(interfaceClass.getName(), allService); } - if(CollectionUtil.isEmpty(allService)){ + if (CollectionUtil.isEmpty(allService)) { return null; } - if(StringUtil.isEmpty(serviceName)){ + if (StringUtil.isEmpty(serviceName)) { return allService.get(0); } - for(T t:allService){ - ServiceName annotation = (ServiceName)t.getClass().getAnnotation(ServiceName.class); + for (T t : allService) { + ServiceName annotation = (ServiceName) t.getClass().getAnnotation(ServiceName.class); if (annotation == null) { return null; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ShellUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ShellUtil.java index 3dab207e..9e55e5f8 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ShellUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ShellUtil.java @@ -22,14 +22,15 @@ import java.io.InputStreamReader; import java.util.HashMap; import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.configuration.SystemContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ShellUtil { private static final String SEPERATOR_LINE = "\r\n"; - private static final Log LOG = LogFactory.getLog(ShellUtil.class); + private static final Logger LOG = LoggerFactory.getLogger(ShellUtil.class); private static final String ENV_CMD = "env"; public static Map exeENV() { @@ -119,7 +120,7 @@ public static String runShell(String shStr, boolean needReturnValue) { String osType = System.getProperty("os.name"); if ("Windows 10".equalsIgnoreCase(osType)) { process = Runtime.getRuntime().exec(shStr.split(" "), null, - new File(ComponentCreator.getProperties().getProperty("siem.python.workdir"))); + new File(SystemContext.getStringParameter(ConfigurationKey.PYTHON_WORK_DIR))); } else { process = Runtime.getRuntime().exec(new String[] {"/bin/sh", "-c", shStr}); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/StringUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/StringUtil.java index 37cb1d20..d31b6edd 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/StringUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/StringUtil.java @@ -25,7 +25,6 @@ import java.util.Random; import java.util.regex.Matcher; import java.util.regex.Pattern; -import org.apache.rocketmq.streams.common.component.ComponentCreator; import org.apache.rocketmq.streams.common.optimization.OptimizationRegex; public final class StringUtil { @@ -34,14 +33,18 @@ public final class StringUtil { * 是否开启 用于快速优化,抽取正则里面的词,能够先匹配,目的是快速失效 的开关 默认为false */ private static boolean regex_fast_switch = false; - - static { - // dipper.properties 配置文件中 获取 - String propertyFlag = ComponentCreator.getProperties().getProperty("regex.fast.switch"); - if ("true".equals(propertyFlag)) { - regex_fast_switch = true; - } - } + /** + * 根据要匹配的字符串,匹配模式字符串,是否忽略大小写和超时时间创建matcher + * + * @param content 待匹配的字符串 + * @param patternStr 匹配模式 + * @param caseInsensitive 是否忽略大小写 + * @param timeout 超时时间 + * @return + */ + private static Map pattern2MatcherForCaseInsensitive = new HashMap(); + private static Map pattern2Matcher = new HashMap(); + private static String hexString = "0123456789abcdef"; public static boolean isEmpty(String string) { return string == null || "".equals(string.trim()); @@ -265,18 +268,6 @@ public static boolean isSameString(String s1, String s2) { return false; } - /** - * 根据要匹配的字符串,匹配模式字符串,是否忽略大小写和超时时间创建matcher - * - * @param content 待匹配的字符串 - * @param patternStr 匹配模式 - * @param caseInsensitive 是否忽略大小写 - * @param timeout 超时时间 - * @return - */ - private static Map pattern2MatcherForCaseInsensitive = new HashMap(); - private static Map pattern2Matcher = new HashMap(); - private static Matcher createMatcher(String content, String patternStr, boolean caseInsensitive, long timeout) { Map map = null; if (caseInsensitive) { @@ -352,8 +343,6 @@ public static String defaultString(String propertyValue, String defalut) { return propertyValue; } - private static String hexString = "0123456789abcdef"; - /* * 将16进制数字解码成字符串,适用于所有字符(包括中文) */ diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/TraceUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/TraceUtil.java index ac7b2024..a0afcc37 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/TraceUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/TraceUtil.java @@ -19,20 +19,15 @@ import java.util.ArrayList; import java.util.List; import java.util.UUID; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TraceUtil { - protected static final Log LOG = LogFactory.getLog(TraceUtil.class); - public static final String TRACE_ID_FLAG = "traceId"; - - private static final String RUNNING_MODE = "true"; - public static final String IGNORE_TRACE_ID = "-1"; - + protected static final Logger LOG = LoggerFactory.getLogger(TraceUtil.class); + private static final String RUNNING_MODE = "true"; private static final String TRACE_SPLIT_FLAG = "@@"; /** diff --git a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/MessasgeCacheTest.java b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/MessasgeCacheTest.java index df2a34e4..04204f92 100644 --- a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/MessasgeCacheTest.java +++ b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/MessasgeCacheTest.java @@ -18,9 +18,9 @@ import com.alibaba.fastjson.JSONObject; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import org.apache.rocketmq.streams.common.channel.impl.OutputPrintChannel; import org.apache.rocketmq.streams.common.context.Message; +import org.apache.rocketmq.streams.common.threadpool.ThreadPoolFactory; import org.junit.Test; public class MessasgeCacheTest { @@ -30,7 +30,7 @@ public void testSink() throws InterruptedException { OutputPrintChannel second = new OutputPrintChannel(); first.init(); second.init(); - ExecutorService executorService = Executors.newFixedThreadPool(100); + ExecutorService executorService = ThreadPoolFactory.createFixedThreadPool(100, MessasgeCacheTest.class.getName() + "-sink_test"); for (int i = 0; i < 10; i++) { JSONObject msg = new JSONObject(); msg.put("name", "chris" + i); diff --git a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/cache/compress/FingerprintCacheTest.java b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/cache/compress/FingerprintCacheTest.java new file mode 100644 index 00000000..c03f9686 --- /dev/null +++ b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/cache/compress/FingerprintCacheTest.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.cache.compress; + +import org.apache.rocketmq.streams.common.cache.compress.impl.LightCache; +import org.junit.Test; + +public class FingerprintCacheTest { + + @Test + public void testFingerprintCache() { + int cacheSize = 10000000; + LightCache localCache = new LightCache(cacheSize); + for (int i = 0; i < cacheSize; i++) { + localCache.put("1.1.1." + i, i); + } + System.out.println(cacheSize + " cost memory is " + localCache.calMemory() + "M"); + } +} diff --git a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/cache/compress/HybridByteBufferTableTest.java b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/cache/compress/HybridByteBufferTableTest.java index fe72ebc6..b4ce6b82 100644 --- a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/cache/compress/HybridByteBufferTableTest.java +++ b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/cache/compress/HybridByteBufferTableTest.java @@ -22,12 +22,7 @@ import org.apache.rocketmq.streams.common.cache.HybridByteBufferTable; import org.junit.Test; -/** - * @author zengyu.cw - * @program rocketmq-streams-apache - * @create 2021-12-20 16:52:53 - * @description - */ + public class HybridByteBufferTableTest { @Test diff --git a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/cache/compress/NumberUtilsTest.java b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/cache/compress/NumberUtilsTest.java index 6e2bcff7..644432eb 100644 --- a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/cache/compress/NumberUtilsTest.java +++ b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/cache/compress/NumberUtilsTest.java @@ -22,12 +22,7 @@ import org.apache.rocketmq.streams.common.utils.NumberUtils; import org.junit.Test; -/** - * @author zengyu.cw - * @program rocketmq-streams-apache - * @create 2021-11-21 22:41:15 - * @description - */ + public class NumberUtilsTest { private void testFloat(float f) { diff --git a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/channel/SourceTest.java b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/channel/SourceTest.java index 4380859a..40a885e7 100644 --- a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/channel/SourceTest.java +++ b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/channel/SourceTest.java @@ -52,7 +52,7 @@ public void testSourceNotJson() throws InterruptedException { return null; } // 处理消息,消息存储在messageBody中, 如果消息不是json,在channel设置JsonData=false,系统会把原始消息放到data字段中 - Object oriMsg = ((UserDefinedMessage)message.getMessageBody()).getMessageValue(); + Object oriMsg = ((UserDefinedMessage) message.getMessageBody()).getMessageValue(); System.out.println(oriMsg); return null; }); diff --git a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/datatype/DataTypeTest.java b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/datatype/DataTypeTest.java index e4abfdcb..b81f9dd4 100644 --- a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/datatype/DataTypeTest.java +++ b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/datatype/DataTypeTest.java @@ -28,11 +28,11 @@ public void testDataType() { DataType datatype = DataTypeUtil.getDataTypeFromClass(Integer.class); assertTrue(datatype.matchClass(Integer.class) == true); String jsonStr = datatype.toDataJson(num);//把int序列化成字符串 - Integer value = (Integer)datatype.getData(jsonStr);//反序列化成int + Integer value = (Integer) datatype.getData(jsonStr);//反序列化成int assertTrue(value == num); byte[] bytes = datatype.toBytes(num, true);//对数字压缩,序列化成字节数组 assertTrue(bytes.length == 3); - value = (Integer)datatype.byteToValue(bytes);//还原数据 + value = (Integer) datatype.byteToValue(bytes);//还原数据 assertTrue(value == num); } } diff --git a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/regex/RegexTest.java b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/regex/RegexTest.java index 81e29432..a39c3064 100644 --- a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/regex/RegexTest.java +++ b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/regex/RegexTest.java @@ -18,23 +18,15 @@ import com.google.re2j.Pattern; import java.util.Set; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; import org.apache.rocketmq.streams.common.optimization.RegexEngine; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; public class RegexTest { - @Before - public void init() { - ComponentCreator.getProperties().setProperty(ConfigureFileKey.DIPPER_REGEX_ENGINE, "re2j"); - } - @Test public void testRe2Basic() { - RegexEngine regexEngine = new RegexEngine<>(); + RegexEngine regexEngine = new RegexEngine<>(true); regexEngine.addRegex("python", 0); regexEngine.addRegex("\\.dll", 1); String message = "python test.py"; diff --git a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/serializa/CountAccum.java b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/serializa/CountAccum.java index 9714b455..41f20cb3 100644 --- a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/serializa/CountAccum.java +++ b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/serializa/CountAccum.java @@ -31,16 +31,16 @@ public void setCount(int count) { } @Override public String toJson() { - JSONObject msg=new JSONObject(); - msg.put("count",count); + JSONObject msg = new JSONObject(); + msg.put("count", count); return msg.toJSONString(); } @Override public void toObject(String jsonString) { - if(jsonString==null){ + if (jsonString == null) { return; } - JSONObject msg=JSONObject.parseObject(jsonString); - count=msg.getInteger("count"); + JSONObject msg = JSONObject.parseObject(jsonString); + count = msg.getInteger("count"); } } diff --git a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/serializa/Person.java b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/serializa/Person.java index 11e7462a..2221cdf4 100644 --- a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/serializa/Person.java +++ b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/serializa/Person.java @@ -28,10 +28,10 @@ public class Person extends BasedConfigurable { protected String name; protected int age; + protected Map accums; private Boolean isMale; private List addresses; private Map childName2Age; - protected Map accums; public static Person createPerson(String namespace) { Person person = new Person(); @@ -52,7 +52,6 @@ public static Person createPerson(String namespace) { person.accums = new HashMap<>(); CountAccum count = new CountAccum(); count.count = 1; - person.setId(1222L); person.accums.put("count", count); return person; } diff --git a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/stages/SelfChainStageTest.java b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/stages/SelfChainStageTest.java index 8b29160f..c4224084 100644 --- a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/stages/SelfChainStageTest.java +++ b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/stages/SelfChainStageTest.java @@ -1,36 +1,35 @@ -/* - * 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.stages; - -import org.apache.rocketmq.streams.common.classloader.ByteClassLoader; -import org.apache.rocketmq.streams.common.topology.stages.OpenAPIChainStage; -import org.junit.Test; -import sun.misc.ProxyGenerator; - -public class SelfChainStageTest { - @Test - public void testClassByte() { - OpenAPIChainStage mapUDFOperator = new OpenAPIChainStage(); - - byte[] classBytes = ProxyGenerator.generateProxyClass(mapUDFOperator.getClass().getName(), new Class[] {OpenAPIChainStage.class}); - //String byteBase64= Base64Utils.encode(classBytes); - //byte[] bytes=Base64Utils.decode(byteBase64); - ByteClassLoader byteClassLoader = new ByteClassLoader(this.getClass().getClassLoader()); - Class clazz = byteClassLoader.defineClass(null, classBytes); - System.out.println(clazz.getName()); - } -} +///* +// * 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.stages; +// +//import org.apache.rocketmq.streams.common.classloader.ByteClassLoader; +//import org.apache.rocketmq.streams.common.topology.stages.OpenAPIChainStage; +//import org.junit.Test; +//import java.lang.reflect.ProxyGenerator; +//public class SelfChainStageTest { +// @Test +// public void testClassByte() { +// OpenAPIChainStage mapUDFOperator = new OpenAPIChainStage(); +// +// byte[] classBytes = ProxyGenerator.generateProxyClass(mapUDFOperator.getClass().getName(), new Class[] {OpenAPIChainStage.class}); +// //String byteBase64= Base64Utils.encode(classBytes); +// //byte[] bytes=Base64Utils.decode(byteBase64); +// ByteClassLoader byteClassLoader = new ByteClassLoader(this.getClass().getClassLoader()); +// Class clazz = byteClassLoader.defineClass(null, classBytes); +// System.out.println(clazz.getName()); +// } +//} diff --git a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/ConfigurableComponent.java b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/ConfigurableComponent.java deleted file mode 100644 index c37571bf..00000000 --- a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/ConfigurableComponent.java +++ /dev/null @@ -1,184 +0,0 @@ -/* - * 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.configurable; - -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.component.AbstractComponent; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.component.ConfigureDescriptor; -import org.apache.rocketmq.streams.common.configurable.IConfigurable; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.utils.ConfigurableUtil; -import org.apache.rocketmq.streams.configurable.service.AbstractConfigurableService; -import org.apache.rocketmq.streams.configurable.service.ConfigurableServcieType; -import org.apache.rocketmq.streams.configurable.service.ConfigurableServiceFactory; - -/** - * 对Configurable对象,做统一的管理,统一查询,插入和更新。 insert/update 把configuabel对象写入存储, - * 支持文件存储(file),内存存储(memory)和db存储(DB)。可以在配置通过这个ConfigureFileKey.CONNECT_TYPE key 配置 - * query 是基于内存的查询,对象定时load到内存,可以在属性文件通过这个ConfigureFileKey.POLLING_TIME key配置加载周期, - * 单位是秒 新对象加载后生效,已经存在的对象只有updateFlag发生变化才会被替换 - */ -public class ConfigurableComponent extends AbstractComponent implements IConfigurableService { - - private static final Log LOG = LogFactory.getLog(ConfigurableComponent.class); - - protected volatile IConfigurableService configureService = null; - - protected transient String namespace; - - public ConfigurableComponent() { - initConfigurableServiceDescriptor(); - addConfigureDescriptor(new ConfigureDescriptor(CONNECT_TYPE, false, ConfigurableServcieType.DEFAULT_SERVICE_NAME)); - } - - public static ConfigurableComponent getInstance(String namespace) { - return ComponentCreator.getComponent(namespace, ConfigurableComponent.class); - } - - @Override - protected boolean initProperties(Properties properties) { - try { - if (configureService != null) { - return true; - } - this.configureService = ConfigurableServiceFactory.createConfigurableService(properties); - return true; - } catch (Exception e) { - LOG.error("ConfigurableComponent create error,properties= " + properties, e); - return false; - } - - } - - @Override - public boolean startComponent(String namespace) { - try { - this.namespace = namespace; - configureService.initConfigurables(namespace); - return true; - } catch (Exception e) { - LOG.error("ConfigurableComponent init error, namespace is " + namespace, e); - return false; - } - - } - - /** - * 启动测试模式,用内存数据库存储和加载configurable数据 - */ - public static void beginTestMode() { - System.setProperty(ConfigurableComponent.CONNECT_TYPE, ConfigurableServcieType.MEMORY_SERVICE_NAME); - } - - /** - * 关闭测试模式,用配置文件中配置的属性加载configurable数据 - */ - public static void endTestMode() { - System.clearProperty(ConfigurableComponent.CONNECT_TYPE); - } - - @Override - public boolean stop() { - return true; - } - - @Override - public IConfigurableService getService() { - return configureService; - } - - @Override - public void initConfigurables(String namespace) { - configureService.initConfigurables(namespace); - } - - @Override - public boolean refreshConfigurable(String namespace) { - return configureService.refreshConfigurable(namespace); - } - - public void mockConfigurable(String namespace) { - refreshConfigurable(namespace); - - } - - @Override - public List queryConfigurable(String type) { - return configureService.queryConfigurable(type); - } - - @Override - public List queryConfigurableByType(String type) { - return configureService.queryConfigurableByType(type); - } - - @Override - public IConfigurable queryConfigurableByIdent(String type, String name) { - return configureService.queryConfigurableByIdent(type, name); - } - - @Override - public IConfigurable queryConfigurableByIdent(String identification) { - return configureService.queryConfigurableByIdent(identification); - } - - @Override - public void insert(IConfigurable configurable) { - configureService.insert(configurable); - ConfigurableUtil.refreshMock(configurable); - } - - @Override - public void update(IConfigurable configurable) { - configureService.update(configurable); - } - - @Override - public Map queryConfigurableMapByType(String type) { - return configureService.queryConfigurableMapByType(type); - } - - @SuppressWarnings("unchecked") - @Override - public T queryConfigurable(String configurableType, String name) { - return (T) queryConfigurableByIdent(configurableType, name); - } - - @Override - public String getNamespace() { - if (configureService instanceof AbstractConfigurableService) { - return configureService.getNamespace(); - } - return namespace; - } - - @Override - public Collection findAll() { - return configureService.findAll(); - } - - @Override - public List loadConfigurableFromStorage(String type) { - return configureService.loadConfigurableFromStorage(type); - } -} diff --git a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/model/Configure.java b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/model/Configure.java deleted file mode 100644 index 4697a270..00000000 --- a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/model/Configure.java +++ /dev/null @@ -1,126 +0,0 @@ -/* - * 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.configurable.model; - -import org.apache.rocketmq.streams.common.model.Entity; - -/** - * configuable如果存储在db,这个是db表的映射对象 - */ -public class Configure extends Entity { - - private static final long serialVersionUID = 5668017348345235669L; - - private String nameSpace; - private String type; - private String name; - // private String identification; - private String jsonValue; - private String modifyTime; - private String remark; - private int openRange; - - public static String createTableSQL(String tableName) { - return "/******************************************/\n" - + "/******************************************/\n" - + "CREATE TABLE IF NOT EXISTS `" + tableName + "` (\n" - + " `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT COMMENT '主键',\n" - + " `gmt_create` datetime NOT NULL COMMENT '创建时间',\n" - + " `gmt_modified` datetime NOT NULL COMMENT '修改时间',\n" - + " `namespace` varchar(32) NOT NULL COMMENT '项目标识',\n" - + " `type` varchar(32) NOT NULL COMMENT '配置类型',\n" - + " `name` varchar(128) NOT NULL COMMENT '配置名称',\n" - + " `json_value` text NOT NULL COMMENT '配置内容',\n" - + " `status` tinyint(3) unsigned NOT NULL DEFAULT '0' COMMENT '1:正在使用 0:已失效',\n" - + " PRIMARY KEY (`id`),\n" - + " UNIQUE KEY `uk_namespace_type_name` (`namespace`,`type`,`name`),\n" - + " KEY `idx_namespace` (`namespace`)\n" - + ") ENGINE=InnoDB AUTO_INCREMENT=1814834 DEFAULT CHARSET=utf8 COMMENT='统一接入配置项'\n" - + ";"; - } - - public String getNameSpace() { - return nameSpace; - } - - public void setNameSpace(String nameSpace) { - this.nameSpace = nameSpace; - } - - public String getType() { - return type; - } - - public void setType(String type) { - this.type = type; - } - - public String getName() { - return name; - } - - public void setName(String name) { - this.name = name; - } - - // public String getIdentification() { - // return identification; - // } - - // public void createIdentification() { - // this.identification = MapKeyUtil.createKey(nameSpace, type, name); - // } - - public String getJsonValue() { - return jsonValue; - } - - public void setJsonValue(String jsonValue) { - this.jsonValue = jsonValue; - } - - public String getModifyTime() { - return modifyTime; - } - - public void setModifyTime(String modifyTime) { - this.modifyTime = modifyTime; - } - - public String getRemark() { - return remark; - } - - public void setRemark(String remark) { - this.remark = remark; - } - - public int getOpenRange() { - return openRange; - } - - public void setOpenRange(int openRange) { - this.openRange = openRange; - } - - @Override - public String toString() { - return "Configure{" + "nameSpace='" + nameSpace + '\'' + ", type='" + type + '\'' + ", name='" + name + '\'' - + ", jsonValue='" + jsonValue + '\'' + ", modifyTime='" + modifyTime + '\'' + ", remark='" + remark + '\'' - + ", openRange=" + openRange + '}'; - } -} diff --git a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/AbstractConfigurableService.java b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/AbstractConfigurableService.java deleted file mode 100644 index 37eb2786..00000000 --- a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/AbstractConfigurableService.java +++ /dev/null @@ -1,510 +0,0 @@ -/* - * 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.configurable.service; - -import com.alibaba.fastjson.JSONObject; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.component.AbstractComponent; -import org.apache.rocketmq.streams.common.configurable.AbstractConfigurable; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurable; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.datatype.DataType; -import org.apache.rocketmq.streams.common.model.Entity; -import org.apache.rocketmq.streams.common.utils.ConfigurableUtil; -import org.apache.rocketmq.streams.common.utils.DataTypeUtil; -import org.apache.rocketmq.streams.common.utils.MapKeyUtil; -import org.apache.rocketmq.streams.common.utils.ReflectUtil; -import org.apache.rocketmq.streams.common.utils.StringUtil; -import org.apache.rocketmq.streams.configurable.model.Configure; - -public abstract class AbstractConfigurableService implements IConfigurableService { - - private static final Log LOG = LogFactory.getLog(AbstractConfigurableService.class); - - private static final String CLASS_NAME = IConfigurableService.CLASS_NAME; - - protected Map> type2ConfigurableMap = new HashMap<>(); - - protected Map name2ConfigurableMap = new HashMap<>(); - - protected Map configurableMap = new HashMap<>(); - - protected Properties properties; - - protected transient String namespace; - - public AbstractConfigurableService(Properties properties) { - this.properties = properties; - } - - public AbstractConfigurableService() { - } - - @Override public IConfigurable queryConfigurableByIdent(String identification) { - return name2ConfigurableMap.get(identification); - } - - protected String getConfigureKey(String nameSpace, String type, String name) { - return MapKeyUtil.createKey(nameSpace, type, name); - } - - protected void updateConfiguresCache(IConfigurable configurable) { - if (configurable == null) { - return; - } - configurable.toJson(); - String key = getConfigureKey(configurable.getNameSpace(), configurable.getType(), configurable.getConfigureName()); - configurableMap.put(key, configurable); - } - - protected void updateConfiguresCache(List configureList) { - for (IConfigurable iConfigurable : configureList) { - updateConfiguresCache(iConfigurable); - } - } - - protected boolean equals(String key, List newConfigureList) { - for (Object o : newConfigureList) { - IConfigurable configure = (IConfigurable) o; - String tempKey = getConfigureKey(configure.getNameSpace(), configure.getType(), configure.getConfigureName()); - if (key.equals(tempKey)) { - IConfigurable oldConfigure = configurableMap.get(key); - if (oldConfigure == null) { - continue; - } - return ConfigurableUtil.compare(oldConfigure, configure); - } - } - return false; - } - - @Override public List queryConfigurableByType(String type) { - List list = queryConfigurable(type); - if (list == null) { - return new ArrayList(); - } - List result = new ArrayList<>(); - for (IConfigurable configurable : list) { - result.add((T) configurable); - } - return result; - } - - @Override public boolean refreshConfigurable(String namespace) { - this.namespace = namespace; - Map> tempType2ConfigurableMap = new HashMap<>(); - Map tempName2ConfigurableMap = new HashMap<>(); - GetConfigureResult configures = loadConfigurable(namespace); - if (configures != null && configures.isQuerySuccess() && configures.getConfigurables() != null) { - List configurables = configures.getConfigurables(); - if (configurables != null && !configurables.isEmpty()) { - checkAndUpdateConfigurables(configurables, tempType2ConfigurableMap, tempName2ConfigurableMap); - for (IConfigurable configurable : this.name2ConfigurableMap.values()) { - if (configurable instanceof IAfterConfigurableRefreshListener) { - ((IAfterConfigurableRefreshListener) configurable).doProcessAfterRefreshConfigurable(this); - } - } - } - - return true; - } - return false; - } - - @Override public T queryConfigurable(String configurableType, String name) { - return (T) queryConfigurableByIdent(configurableType, name); - } - - protected List checkAndUpdateConfigurables(List configurables, Map> tempType2ConfigurableMap, Map tempName2ConfigurableMap) { - List configurableList = new ArrayList<>(); - for (IConfigurable configurable : configurables) { - try { - boolean isUpdate = update(configurable, tempName2ConfigurableMap, tempType2ConfigurableMap); - if (isUpdate) { - configurableList.add(configurable); - } - } catch (Exception e) { - LOG.error("组件初始化异常:" + e.getMessage() + ",name=" + configurable.getConfigureName(), e); - } - } - destroyOldConfigurables(tempName2ConfigurableMap); - this.name2ConfigurableMap = tempName2ConfigurableMap; - this.type2ConfigurableMap = tempType2ConfigurableMap; - return configurableList; - } - - private void destroyOldConfigurables(Map tempName2ConfigurableMap) { - for (Map.Entry entry : this.name2ConfigurableMap.entrySet()) { - String key = entry.getKey(); - IConfigurable value = entry.getValue(); - if (!tempName2ConfigurableMap.containsKey(key)) { - destroyOldConfigurable(value); - } - } - - } - - private void destroyOldConfigurable(IConfigurable oldConfigurable) { - if (oldConfigurable instanceof AbstractConfigurable) { - oldConfigurable.destroy(); - } - String key = getConfigureKey(oldConfigurable.getNameSpace(), oldConfigurable.getType(), oldConfigurable.getConfigureName()); - configurableMap.remove(key); - } - - protected void initConfigurable(IConfigurable configurable) { - if (configurable instanceof AbstractConfigurable) { - AbstractConfigurable abstractConfigurable = (AbstractConfigurable) configurable; - abstractConfigurable.setConfigurableService(this); - } - configurable.init(); - - } - - private ScheduledExecutorService scheduledExecutorService; - - @Override public void initConfigurables(final String namespace) { - refreshConfigurable(namespace); - long polingTime = -1; - if (this.properties != null) { - String pollingTimeStr = this.properties.getProperty(AbstractComponent.POLLING_TIME); - if (StringUtil.isNotEmpty(pollingTimeStr)) { - polingTime = Long.parseLong(pollingTimeStr); - } - } - if (polingTime > 0) { - scheduledExecutorService = new ScheduledThreadPoolExecutor(3); - scheduledExecutorService.scheduleWithFixedDelay(new Runnable() { - - @Override public void run() { - try { - refreshConfigurable(namespace); - } catch (Exception e) { - LOG.error("Load configurables error:" + e.getMessage(), e); - } - } - }, polingTime, polingTime, TimeUnit.SECONDS); - } - } - - @Override public List queryConfigurable(String type) { - String key = MapKeyUtil.createKey(type); - return type2ConfigurableMap.get(key); - } - - @Override public IConfigurable queryConfigurableByIdent(String type, String name) { - String key = MapKeyUtil.createKey(type, name); - return name2ConfigurableMap.get(key); - } - - /** - * 根据namespace加载配置信息 - * - * @param namespace - * @return - * @throws Exception - */ - protected abstract GetConfigureResult loadConfigurable(String namespace); - - @Override public void update(IConfigurable configurable) { - updateConfigurable(configurable); - } - - protected abstract void updateConfigurable(IConfigurable configurable); - - protected abstract void insertConfigurable(IConfigurable configurable); - - protected boolean update(IConfigurable configurable, Map name2ConfigurableMap, Map> type2ConfigurableMap) { - if (configurable == null) { - return false; - } - - boolean isUpdate = false; - List configurableList = new ArrayList<>(); - configurableList.add(configurable); - - String nameKey = MapKeyUtil.createKey(configurable.getType(), configurable.getConfigureName()); - if (this.name2ConfigurableMap.containsKey(nameKey)) { - String configureKey = getConfigureKey(namespace, configurable.getType(), configurable.getConfigureName()); - IConfigurable oldConfigurable = this.name2ConfigurableMap.get(nameKey); - if (equals(configureKey, configurableList)) { - configurable = oldConfigurable; - } else { - destroyOldConfigurable(oldConfigurable); - initConfigurable(configurable); - isUpdate = true; - } - } else { - initConfigurable(configurable); - isUpdate = true; - } - updateConfiguresCache(configurable); - name2ConfigurableMap.put(nameKey, configurable); - String typeKey = MapKeyUtil.createKey(configurable.getType()); - put2Map(type2ConfigurableMap, typeKey, configurable); - return isUpdate; - } - - @Override public void insert(IConfigurable configurable) { - insertConfigurable(configurable); - } - - /** - * 给一个扣,可以跨命名空间查询数据 - * - * @param namespaces - * @return - */ - public List queryConfiguableByNamespace(String... namespaces) { - List configurables = new ArrayList<>(); - if (namespaces == null || namespaces.length == 0) { - return configurables; - } - for (String namespace : namespaces) { - GetConfigureResult result = loadConfigurable(namespace); - if (result.querySuccess) { - if (result.configurables != null && result.configurables.size() > 0) { - configurables.addAll(result.configurables); - } - } else { - throw new RuntimeException("Load configurable error, the namespace is " + namespace); - } - } - return configurables; - - } - - /** - * 往一个value是list的map中添加数据,如果list是空创建,否则直接插入 - * - * @param map - * @param key - * @param configurable - */ - protected void put2Map(Map> map, String key, IConfigurable configurable) { - List list = map.computeIfAbsent(key, k -> new ArrayList<>()); - list.add(configurable); - } - - @Override public Collection findAll() { - return name2ConfigurableMap.values(); - } - - /** - * 把configurable转换成configure - * - * @param configurable - * @return - */ - protected Configure createConfigure(IConfigurable configurable) { - Configure configure = new Configure(); - configure.setType(configurable.getType()); - configure.setName(configurable.getConfigureName()); - configure.setNameSpace(configurable.getNameSpace()); - String jsonString = configurable.toJson(); - if (!StringUtil.isEmpty(jsonString)) { - JSONObject jsonObject = JSONObject.parseObject(jsonString); - jsonObject.put(CLASS_NAME, configurable.getClass().getName()); - configure.setJsonValue(jsonObject.toJSONString()); - } - return configure; - } - - @Override public Map queryConfigurableMapByType(String type) { - List configurables = queryConfigurable(type); - if (configurables == null) { - return new HashMap(); - } - Map result = new HashMap(); - for (IConfigurable configurable : configurables) { - result.put(configurable.getConfigureName(), (T) configurable); - } - return result; - } - - /** - * 把configure转换成configurable - * - * @param configures - * @return - */ - protected List convert(List configures) { - if (configures == null) { - return new ArrayList(); - } - List configurables = new ArrayList(); - for (Configure configure : configures) { - IConfigurable configurable = convert(configure); - if (configurable != null) { - configurables.add(configurable); - } - - } - return configurables; - } - - protected IConfigurable createConfigurableFromJson(String namespace, String type, String name, String jsonValue) { - if (StringUtil.isEmpty(jsonValue)) { - return null; - } - JSONObject jsonObject = JSONObject.parseObject(jsonValue); - String className = jsonObject.getString(CLASS_NAME); - IConfigurable configurable = createConfigurable(className); - if (configurable == null) { - return null; - } - configurable.setConfigureName(name); - configurable.setNameSpace(namespace); - configurable.setType(type); - if (AbstractConfigurable.class.isInstance(configurable)) { - AbstractConfigurable abstractConfigurable = (AbstractConfigurable) configurable; - abstractConfigurable.setConfigurableService(this); - } - configurable.toObject(jsonValue); - return configurable; - } - - /** - * 提供一个入口,可以让外部用户改变configure对应的configurable的值 - * - * @param configure - * @return - */ - protected IConfigurable convert(Configure configure) { - - return convertConfigurable(configure); - } - - protected IConfigurable convertConfigurable(Configure configure) { - String className = null; - try { - String jsonString = configure.getJsonValue(); - IConfigurable configurable = createConfigurableFromJson(configure.getNameSpace(), configure.getType(), configure.getName(), jsonString); - if (configurable instanceof Entity) { - // add by wangtl 20171110 Configurable接口第三方包也在用,故不能Configurable里加接口,只能加到抽象类里,这里强转下 - Entity abs = (Entity) configurable; - abs.setId(configure.getId()); - abs.setGmtCreate(configure.getGmtCreate()); - abs.setGmtModified(configure.getGmtModified()); - /* - * abs.setTempKey((configurable.getNameSpace() + configurable.getType() + - * configurable.getConfigureName() + jsonString).hashCode()); - */ - } - convertPost(configurable); - return configurable; - } catch (Exception e) { - LOG.error("转换异常:" + configure.toString(), e); - return null; - } - } - - /** - * 如果需要改变configurable的属性,可以再这里设置 - * - * @param configurable - */ - @SuppressWarnings("rawtypes") protected void convertPost(IConfigurable configurable) { - if (this.properties == null) { - return; - } - String identification = MapKeyUtil.createKey(configurable.getNameSpace(), configurable.getType(), configurable.getConfigureName()); - String propertyValue = this.properties.getProperty(identification); - if (StringUtil.isEmpty(propertyValue)) { - return; - } - String[] fieldName2Values = propertyValue.split(","); - if (fieldName2Values.length == 0) { - return; - } - for (String fieldName2Value : fieldName2Values) { - try { - String[] fieldName2ValueArray = fieldName2Value.split(":"); - if (fieldName2ValueArray.length != 2) { - continue; - } - String fieldName = fieldName2ValueArray[0]; - String value = fieldName2ValueArray[1]; - Class clazz = ReflectUtil.getBeanFieldType(configurable.getClass(), fieldName); - DataType dataType = DataTypeUtil.createDataType(clazz, null); - if (dataType == null) { - continue; - } - Object fieldValue = dataType.getData(value); - ReflectUtil.setBeanFieldValue(configurable, fieldName, fieldValue); - - } catch (Exception e) { - LOG.error("convert post error " + fieldName2Value, e); - } - } - } - - /** - * 创建configurable对象 - * - * @param className class name - * @return - */ - @SuppressWarnings("rawtypes") protected IConfigurable createConfigurable(String className) { - return ReflectUtil.forInstance(className); - } - - public class GetConfigureResult { - - private boolean querySuccess; - private List configurables; - - public boolean isQuerySuccess() { - return querySuccess; - } - - public void setQuerySuccess(boolean querySuccess) { - this.querySuccess = querySuccess; - } - - public List getConfigurables() { - return configurables; - } - - public void setConfigurables(List configurables) { - this.configurables = configurables; - } - } - - @Override public String getNamespace() { - return namespace; - } - - public Properties getProperties() { - return properties; - } - - public void setProperties(Properties properties) { - this.properties = properties; - } -} diff --git a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/AbstractSupportParentConfigureService.java b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/AbstractSupportParentConfigureService.java deleted file mode 100644 index d5ae3db1..00000000 --- a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/AbstractSupportParentConfigureService.java +++ /dev/null @@ -1,198 +0,0 @@ -/* - * 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.configurable.service; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Properties; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.configurable.IConfigurable; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.utils.StringUtil; - -/** - * namespace 分层,支持顶级命名空间,顶级命名空间的对象,所有命名空间都可见。顶级命名空间是固定值IConfigurableService.PARENT_CHANNEL_NAME_SPACE - */ -public abstract class AbstractSupportParentConfigureService extends AbstractConfigurableService - implements IConfigurableService { - - private static final Log LOG = LogFactory.getLog(AbstractSupportParentConfigureService.class); - protected IConfigurableService configureService = null; - protected IConfigurableService parentConfigureService = null; - protected Properties properties; - - public AbstractSupportParentConfigureService() { - super(null); - } - - public void initMethod(Properties property) { - this.properties = property; - initBeforeInitConfigurable(property); - } - - protected abstract void initBeforeInitConfigurable(Properties property); - - @Override - public void initConfigurables(String namespace) { - - if (!IConfigurableService.PARENT_CHANNEL_NAME_SPACE.equals(namespace)) { - parentConfigureService.initConfigurables(IConfigurableService.PARENT_CHANNEL_NAME_SPACE); - } else { - parentConfigureService = null; - } - configureService.initConfigurables(namespace); - } - - @Override - public boolean refreshConfigurable(String namespace) { - if (!IConfigurableService.PARENT_CHANNEL_NAME_SPACE.equals(namespace)) { - parentConfigureService.refreshConfigurable(IConfigurableService.PARENT_CHANNEL_NAME_SPACE); - } - configureService.refreshConfigurable(namespace); - return true; - } - - @Override - public List queryConfigurable(String type) { - List result = configureService.queryConfigurable(type); - if (result == null) { - result = new ArrayList<>(); - } - if (parentConfigureService == null) { - return result; - } - List parent = parentConfigureService.queryConfigurable(type); - if (parent != null) { - result.addAll(parent); - } - return result; - } - - @Override - public IConfigurable queryConfigurableByIdent(String type, String name) { - IConfigurable configurable = configureService.queryConfigurableByIdent(type, name); - if (configurable != null) { - return configurable; - } - if (parentConfigureService == null) { - return null; - } - if (configurable != null) { - return configurable; - } - return parentConfigureService.queryConfigurableByIdent(type, name); - } - - @Override - public IConfigurable queryConfigurableByIdent(String identification) { - IConfigurable configurable = configureService.queryConfigurableByIdent(identification); - if (configurable != null) { - return configurable; - } - if (parentConfigureService == null) { - return null; - } - if (configurable != null) { - return configurable; - } - return parentConfigureService.queryConfigurableByIdent(identification); - } - - @Override - protected void insertConfigurable(IConfigurable configurable) { - if (parentConfigureService != null && configurable.getNameSpace() - .equals(IConfigurableService.PARENT_CHANNEL_NAME_SPACE)) { - parentConfigureService.insert(configurable); - } else { - configureService.insert(configurable); - } - } - - @Override - protected void updateConfigurable(IConfigurable configurable) { - if (parentConfigureService != null && configurable.getNameSpace() - .equals(IConfigurableService.PARENT_CHANNEL_NAME_SPACE)) { - parentConfigureService.update(configurable); - } else { - configureService.update(configurable); - } - } - - @Override - public T queryConfigurable(String configurableType, String name) { - return (T) queryConfigurableByIdent(configurableType, name); - } - - @Override - protected GetConfigureResult loadConfigurable(String namespace) { - return null; - } - - protected List createAndQueryConfigurable(String sharedNameSpace, String sharedType, - String sharedName) { - IConfigurableService innerSharedConfigurableService = - ConfigurableServiceFactory.createConfigurableService(properties); - innerSharedConfigurableService.refreshConfigurable(sharedNameSpace); - if (StringUtil.isNotEmpty(sharedName)) { - List configurables = new ArrayList<>(); - IConfigurable configurable = innerSharedConfigurableService.queryConfigurableByIdent(sharedType, sharedName); - configurables.add(configurable); - return configurables; - } else { - return innerSharedConfigurableService.queryConfigurable(sharedType); - } - - } - - @Override - public Collection findAll() { - List configurables = new ArrayList<>(); - if (parentConfigureService != null) { - Collection tmp = parentConfigureService.findAll(); - if (tmp != null || tmp.size() > 0) { - configurables.addAll(tmp); - } - } - Collection tmp = configureService.findAll(); - if (tmp != null || tmp.size() > 0) { - configurables.addAll(tmp); - } - return configurables; - } - - public IConfigurableService getConfigureService() { - return configureService; - } - - @Override - public List loadConfigurableFromStorage(String type) { - List configurables = new ArrayList<>(); - if (parentConfigureService != null) { - Collection tmp = parentConfigureService.loadConfigurableFromStorage(type); - if (tmp != null || tmp.size() > 0) { - configurables.addAll(tmp); - } - } - Collection tmp = configureService.loadConfigurableFromStorage(type); - if (tmp != null || tmp.size() > 0) { - configurables.addAll(tmp); - } - return configurables; - } -} diff --git a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/ConfigurableServiceFactory.java b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/ConfigurableServiceFactory.java deleted file mode 100644 index a7a29da5..00000000 --- a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/ConfigurableServiceFactory.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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.configurable.service; - -import java.util.Properties; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.utils.ReflectUtil; -import org.apache.rocketmq.streams.common.utils.StringUtil; -import org.apache.rocketmq.streams.serviceloader.ServiceLoaderComponent; - -public class ConfigurableServiceFactory { - private static ServiceLoaderComponent configurableServiceLoaderComponent = ServiceLoaderComponent.getInstance(IConfigurableService.class); - public static final String CONFIGURABLE_SERVICE_TYPE = "dipper.configurable.service.type"; - private static final Log LOG = LogFactory.getLog(ConfigurableServiceFactory.class); - - public static IConfigurableService createConfigurableService(Properties properties) { - try { - Properties properties1 = new Properties(); - properties1.putAll(properties); - String type = properties1.getProperty(CONFIGURABLE_SERVICE_TYPE); - if (StringUtil.isEmpty(type)) { - type = IConfigurableService.MEMORY_SERVICE_NAME; - } - IConfigurableService configurableService = getConfigurableServiceType(type); - if (configurableService instanceof AbstractSupportParentConfigureService) { - ((AbstractSupportParentConfigureService) configurableService).initMethod(properties1); - } - return configurableService; - } catch (Exception e) { - LOG.error("create ConfigurableService error", e); - return null; - } - - } - - public static IConfigurableService getConfigurableServiceType(String type) { - IConfigurableService configurableService = configurableServiceLoaderComponent.getService().loadService(type); - return ReflectUtil.forInstance(configurableService.getClass().getName()); - } -} diff --git a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/FileConfigureService.java b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/FileConfigureService.java deleted file mode 100644 index 2d54a3e5..00000000 --- a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/FileConfigureService.java +++ /dev/null @@ -1,266 +0,0 @@ -/* - * 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.configurable.service.impl; - -import java.io.File; -import java.util.ArrayList; -import java.util.List; -import java.util.Properties; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configurable.IConfigurable; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; -import org.apache.rocketmq.streams.common.model.Entity; -import org.apache.rocketmq.streams.common.utils.AESUtil; -import org.apache.rocketmq.streams.common.utils.DateUtil; -import org.apache.rocketmq.streams.common.utils.FileUtil; -import org.apache.rocketmq.streams.common.utils.MapKeyUtil; -import org.apache.rocketmq.streams.common.utils.StringUtil; -import org.apache.rocketmq.streams.configurable.model.Configure; -import org.apache.rocketmq.streams.configurable.service.AbstractConfigurableService; - -public class FileConfigureService extends AbstractConfigurableService { - - public static final String FILE_PATH_NAME = IConfigurableService.FILE_PATH_NAME; - // 配置文件的路径 - private static final Log LOG = LogFactory.getLog(FileConfigureService.class); - private static final String DEFAULT_FILE_NAME = "dipper.cs"; // 默认文件名 - private static final String SIGN = "&&&&"; // 字段分割附号 - public String fileName; - - public FileConfigureService(Properties properties) { - super(properties); - initService(properties.getProperty(FILE_PATH_NAME)); - } - - protected void initService(String fileAndPath) { - if (StringUtil.isEmpty(fileAndPath)) { - String path = this.getClass().getProtectionDomain().getCodeSource().getLocation().getPath(); - if (path.endsWith(".jar")) { - int index = path.lastIndexOf(File.separator); - path = path.substring(0, index); - } - fileName = FileUtil.concatFilePath(path, DEFAULT_FILE_NAME); - } else { - fileName = fileAndPath; - } - LOG.info("load file from path = " + fileName); - } - - @Override - protected GetConfigureResult loadConfigurable(String namespace) { - GetConfigureResult result = new GetConfigureResult(); - try { - List configures = selectOpening(namespace); - List configurables = convert(configures); - LOG.info("load configure namespace=" + namespace + " count=" + configures.size()); - result.setConfigurables(configurables); - // 该字段标示查询是否成功,若不成功则不会更新配置 - result.setQuerySuccess(true); - } catch (Exception e) { - result.setQuerySuccess(false); - e.printStackTrace(); - LOG.error("load configurable error ", e); - } - return result; - } - - protected List selectOpening(String namespace) { - List list = loadFileLine(fileName); - List configures = convert2Configure(list); - return filter(configures, namespace); - } - - protected List filter(List configures, String namespace) { - if (configures == null) { - return new ArrayList<>(); - } - if (StringUtil.isEmpty(namespace)) { - throw new RuntimeException("namespace can not empty "); - } - List filterConfigures = new ArrayList<>(); - for (Configure configure : configures) { - if (!namespace.equals(configure.getNameSpace())) { - continue; - } - filterConfigures.add(configure); - } - return filterConfigures; - } - - @Override - protected void insertConfigurable(IConfigurable configure) { - if (configure == null) { - LOG.warn("insert configure is null"); - return; - } - String row = configure2String(configure); - - List rows = loadFileLine(fileName); - if (rows == null) { - rows = new ArrayList<>(); - } - List configures = convert2Configure(rows); - String newKey = - MapKeyUtil.createKey(configure.getNameSpace(), configure.getType(), configure.getConfigureName()); - boolean isReplace = false; - for (int i = 0; i < configures.size(); i++) { - Configure c = configures.get(i); - String old = MapKeyUtil.createKey(c.getNameSpace(), c.getType(), c.getName()); - if (old.equals(newKey)) { - rows.set(i, configure2String(configure)); - isReplace = true; - break; - } - } - if (!isReplace) { - rows.add(configure2String(configure)); - } - writeFile(fileName, rows); - } - - @Override - protected void updateConfigurable(IConfigurable configure) { - if (configure == null) { - LOG.warn("insert configure is null"); - return; - } - - List rows = FileUtil.loadFileLine(fileName); - if (rows == null) { - rows = new ArrayList<>(); - } - for (int i = 0; i < rows.size(); i++) { - String row = rows.get(i); - Configure oldConfigure = convert(row); - if (configure.getNameSpace().equals(oldConfigure.getNameSpace()) && configure.getType() - .equals(oldConfigure.getType()) && configure.getConfigureName().equals(oldConfigure.getName())) { - rows.set(i, configure2String(configure)); - } - } - writeFile(fileName, rows); - - } - - protected Configure convert(String row) { - String[] values = row.split(SIGN); - String namespace = getColumnValue(values, 0, "namespace"); - String type = getColumnValue(values, 1, "type"); - String name = getColumnValue(values, 2, "name"); - String jsonValue = getColumnValue(values, 3, "json_value"); - try { - jsonValue = AESUtil.aesDecrypt(jsonValue, ComponentCreator.getProperties().getProperty(ConfigureFileKey.SECRECY, ConfigureFileKey.SECRECY_DEFAULT)); - } catch (Exception e) { - LOG.error("failed in decrypting the value, reason:\t" + e.getCause()); - throw new RuntimeException(e); - } - String createDate = getColumnValue(values, 4, "gmt_create"); - String modifiedDate = getColumnValue(values, 5, "gmt_modified"); - String id = getColumnValue(values, 6, "id"); - Configure configure = new Configure(); - configure.setNameSpace(namespace); - configure.setType(type); - configure.setName(name); - configure.setJsonValue(jsonValue); - configure.setGmtCreate(DateUtil.parse(createDate)); - configure.setGmtCreate(DateUtil.parse(modifiedDate)); - configure.setId((id == null ? null : Long.valueOf(id))); - - return configure; - } - - protected List convert2Configure(List rows) { - List configures = new ArrayList(); - for (String row : rows) { - configures.add(convert(row)); - } - return configures; - } - - protected String getColumnValue(String[] values, int i, String namespace) { - if (values == null || values.length == 0) { - return null; - } - if (values.length <= i) { - return null; - } - if ("null".equals(values[i])) { - return null; - } - return values[i]; - } - - /** - * 解密文件,并加载到内存 - * - * @param fileName - * @return - */ - protected List loadFileLine(String fileName) { - List rows = FileUtil.loadFileLine(fileName); - if (rows == null) { - rows = new ArrayList<>(); - } - return doDecRowList(rows); - } - - protected void writeFile(String fileName, List rows) { - List rowList = doEncryptRowList(rows); - FileUtil.write(fileName, rowList); - } - - private List doEncryptRowList(List rows) { - return rows; - } - - private List doDecRowList(List rows) { - return rows; - } - - protected String configure2String(IConfigurable configure) { - Entity entity = null; - if (configure instanceof Entity) { - entity = (Entity) configure; - } else { - entity = new Entity(); - } - String theSecretValue = null; - try { - theSecretValue = AESUtil.aesEncrypt(configure.toJson(), ComponentCreator.getProperties().getProperty(ConfigureFileKey.SECRECY, ConfigureFileKey.SECRECY_DEFAULT)); - } catch (Exception e) { - LOG.error("failed in encrypting the value, reason:\t" + e.getCause()); - throw new RuntimeException(e); - } - String row = MapKeyUtil.createKeyBySign(SIGN, configure.getNameSpace(), configure.getType(), - configure.getConfigureName(), theSecretValue, DateUtil.format(entity.getGmtCreate()), - DateUtil.format(entity.getGmtModified()), entity.getId() + ""); - return row; - } - - public String getFileName() { - return fileName; - } - - @Override - public List loadConfigurableFromStorage(String type) { - refreshConfigurable(getNamespace()); - return queryConfigurableByType(type); - } - -} diff --git a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/FileSupportParentConfigureService.java b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/FileSupportParentConfigureService.java deleted file mode 100644 index 0b77b776..00000000 --- a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/FileSupportParentConfigureService.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.configurable.service.impl; - -import com.google.auto.service.AutoService; -import java.util.Properties; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.model.ServiceName; -import org.apache.rocketmq.streams.configurable.service.AbstractSupportParentConfigureService; -import org.apache.rocketmq.streams.configurable.service.ConfigurableServcieType; - -@AutoService(IConfigurableService.class) -@ServiceName(ConfigurableServcieType.FILE_SERVICE_NAME) -public class FileSupportParentConfigureService extends AbstractSupportParentConfigureService { - - @Override - protected void initBeforeInitConfigurable(Properties property) { - this.configureService = new FileConfigureService(properties); - this.parentConfigureService = new FileConfigureService(properties); - } - -} diff --git a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/HttpConfigureService.java b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/HttpConfigureService.java deleted file mode 100644 index a259a2fa..00000000 --- a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/HttpConfigureService.java +++ /dev/null @@ -1,377 +0,0 @@ -/* - * 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.configurable.service.impl; - -import com.alibaba.fastjson.JSONArray; -import com.alibaba.fastjson.JSONObject; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.io.Reader; -import java.io.UnsupportedEncodingException; -import java.nio.charset.Charset; -import java.nio.charset.UnsupportedCharsetException; -import java.security.cert.CertificateException; -import java.security.cert.X509Certificate; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import javax.net.ssl.HostnameVerifier; -import javax.net.ssl.SSLContext; -import javax.net.ssl.SSLSession; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.http.Consts; -import org.apache.http.Header; -import org.apache.http.HttpEntity; -import org.apache.http.ParseException; -import org.apache.http.client.config.RequestConfig; -import org.apache.http.client.methods.CloseableHttpResponse; -import org.apache.http.client.methods.HttpGet; -import org.apache.http.client.methods.HttpPost; -import org.apache.http.config.ConnectionConfig; -import org.apache.http.config.Registry; -import org.apache.http.config.RegistryBuilder; -import org.apache.http.conn.socket.ConnectionSocketFactory; -import org.apache.http.conn.socket.PlainConnectionSocketFactory; -import org.apache.http.conn.ssl.SSLConnectionSocketFactory; -import org.apache.http.entity.ContentType; -import org.apache.http.entity.StringEntity; -import org.apache.http.impl.client.CloseableHttpClient; -import org.apache.http.impl.client.HttpClientBuilder; -import org.apache.http.impl.client.HttpClients; -import org.apache.http.impl.conn.PoolingHttpClientConnectionManager; -import org.apache.http.protocol.HTTP; -import org.apache.http.ssl.SSLContexts; -import org.apache.http.ssl.TrustStrategy; -import org.apache.http.util.Args; -import org.apache.http.util.CharArrayBuffer; -import org.apache.rocketmq.streams.common.component.AbstractComponent; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configurable.IConfigurable; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; -import org.apache.rocketmq.streams.common.utils.AESUtil; -import org.apache.rocketmq.streams.configurable.model.Configure; -import org.apache.rocketmq.streams.configurable.service.AbstractConfigurableService; - -public class HttpConfigureService extends AbstractConfigurableService { - - public static final int NORMAL_STATUES = 200; - public static final String CHARSET = "UTF-8"; - public static final int TIMOUT = 10000; - public static final int CONNECT_TIMOUT = 10000; - - protected String accessId; - protected String accessIdSecret; - protected String endPoint; - - protected transient CloseableHttpClient client; - - private static final Log LOG = LogFactory.getLog(HttpConfigureService.class); - - public static void main(String[] args) { - HttpConfigureService service = new HttpConfigureService("", "", "http://11.158.168.161:8888/queryConfigure"); - service.loadConfigurable("test"); - } - - public HttpConfigureService() { - init(); - } - - public HttpConfigureService(String accessId, String accessIdSecret, String endPoint) { - this.accessId = accessId; - this.accessIdSecret = accessIdSecret; - this.endPoint = endPoint; - init(); - } - - public HttpConfigureService(Properties properties) { - super(properties); - this.accessId = properties.getProperty(AbstractComponent.HTTP_AK); - this.accessIdSecret = properties.getProperty(AbstractComponent.HTTP_SK); - this.endPoint = properties.getProperty(HTTP_SERVICE_ENDPOINT) + "/queryConfigure"; - init(); - } - - public void init() { - RequestConfig.Builder configBuilder = RequestConfig.custom(); - configBuilder.setConnectionRequestTimeout(CONNECT_TIMOUT); - configBuilder.setConnectTimeout(CONNECT_TIMOUT); - configBuilder.setSocketTimeout(TIMOUT); - SSLConnectionSocketFactory sslsf = null; - try { - SSLContext sslcontext = SSLContexts.custom().loadTrustMaterial(new TrustStrategy() { - @Override - public boolean isTrusted(X509Certificate[] chain, String authType) throws CertificateException { - return true; - } - }).build(); - sslsf = new SSLConnectionSocketFactory(sslcontext, new HostnameVerifier() { - @Override - public boolean verify(String s, SSLSession sslSession) { - return true; - } - }); - } catch (Exception e) { - throw new RuntimeException(e); - } - ConnectionConfig connectionConfig = ConnectionConfig.custom().setCharset(Consts.UTF_8).build(); - Registry socketFactoryRegistry = RegistryBuilder.create().register("https", - sslsf).register("http", - new PlainConnectionSocketFactory()).build(); - PoolingHttpClientConnectionManager connManager = new PoolingHttpClientConnectionManager(socketFactoryRegistry); - connManager.setDefaultConnectionConfig(connectionConfig); - connManager.setMaxTotal(500); - connManager.setDefaultMaxPerRoute(50); - HttpClientBuilder clientBuilder = HttpClients.custom(); - - clientBuilder.setDefaultRequestConfig(configBuilder.build()); - clientBuilder.setSSLSocketFactory(sslsf); - clientBuilder.setConnectionManager(connManager); - - client = clientBuilder.build(); - - } - - @Override - public GetConfigureResult loadConfigurable(String namespace) { - GetConfigureResult result = new GetConfigureResult(); -// request.setDipperNamespace(namespace); - try { - JSONObject param = new JSONObject(); - param.put("namespace", namespace); - CloseableHttpResponse response = post(endPoint, param.toJSONString(), null); - if (response == null) { - result.setQuerySuccess(false); - if (LOG.isErrorEnabled()) { - LOG.error("loadConfigurable error!namespace=" + namespace + ",response=" + JSONObject.toJSONString( - response)); - } - } else { - result.setQuerySuccess(true); - List configures = new ArrayList(); - String content = toString(response.getEntity(), Charset.forName(CHARSET)); - JSONObject object = JSONObject.parseObject(content); - JSONArray data = object.getJSONArray("data"); - System.out.println(data.size()); - configures = convert2Configure(data); - List configurables = convert(configures); - result.setConfigurables(configurables); - System.out.println(configures.size() + " " + configurables.size()); -// Data data = response.getEntity().getContent() -// if (data != null) { -// for (Item item : data.getItems()) { -// Configure configure = convert2Configure(item); -// configures.add(configure); -// } -// } -// result.setConfigurables(convert(configures)); - } - } catch (Exception e) { - result.setQuerySuccess(false); - if (LOG.isErrorEnabled()) { - LOG.error("loadConfigurable error!namespace=" + namespace, e); - } - } - return result; - } - - public CloseableHttpResponse get(String url, Map param) { - - try { - HttpGet httpGet = new HttpGet(url); - if (param != null && param.size() > 0) { - List
    headers = new ArrayList<>(); - for (Map.Entry tmp : param.entrySet()) { - httpGet.addHeader(tmp.getKey(), tmp.getValue()); - } - } - CloseableHttpResponse response = client.execute(httpGet); - return response; - } catch (Exception e) { - e.printStackTrace(); - } - return null; - - } - - public CloseableHttpResponse get(String url, Header... headers) { - try { - HttpGet httpGet = new HttpGet(url); - if (headers != null && headers.length > 0) { - for (Header header : headers) { - httpGet.addHeader(header); - } - } - CloseableHttpResponse response = client.execute(httpGet); - return response; - } catch (Exception e) { - e.printStackTrace(); - } - return null; - } - - public CloseableHttpResponse post(String url, String body, Header... headers) { - try { - HttpPost httpPost = new HttpPost(url); - StringEntity stringEntity = new StringEntity(body, CHARSET); - stringEntity.setContentEncoding(CHARSET); - stringEntity.setContentType("application/json"); - httpPost.setEntity(stringEntity); - if (headers != null && headers.length > 0) { - for (Header header : headers) { - httpPost.addHeader(header); - } - } - CloseableHttpResponse response = client.execute(httpPost); - return response; - - } catch (Exception e) { - e.printStackTrace(); - } - return null; - } - - protected List convert2Configure(JSONArray array) { - List configures = new ArrayList(); - for (int i = 0; i < array.size(); i++) { - Configure configure = array.getObject(i, Configure.class); - if (!namespace.equalsIgnoreCase(configure.getNameSpace())) { - continue; - } - try { - configure.setJsonValue(AESUtil.aesDecrypt(configure.getJsonValue(), - ComponentCreator.getProperties().getProperty(ConfigureFileKey.SECRECY, ConfigureFileKey.SECRECY_DEFAULT))); - } catch (Exception e) { - e.printStackTrace(); - } - configures.add(configure); - } - return configures; - } - - protected List convert(List configures) { - if (configures == null) { - return new ArrayList(); - } - List configurables = new ArrayList(); - for (Configure configure : configures) { - IConfigurable configurable = convert(configure); - if (configurable != null) { - configurables.add(configurable); - } - } - return configurables; - } - -// private Configure convert2Configure(Item item) { -// Configure configure = new Configure(); -// configure.setAccountId(item.getAccountId()); -// configure.setAccountName(item.getAccountName()); -// configure.setAccountNickName(item.getAccountNickName()); -// configure.setClientIp(item.getDipperClientIp()); -// configure.setJsonValue(item.getJsonValue()); -// configure.setName(item.getName()); -// configure.setNameSpace(item.getDipperNamespace()); -// configure.setRequestId(item.getDipperRequestId()); -// configure.setType(item.getType()); -// return configure; -// } - - public String toString(HttpEntity entity, Charset defaultCharset) throws IOException, ParseException { - Args.notNull(entity, "Entity"); - InputStream instream = entity.getContent(); - if (instream == null) { - return null; - } else { - try { - Args.check(entity.getContentLength() <= 2147483647L, "HTTP entity too large to be buffered in memory"); - int i = (int) entity.getContentLength(); - if (i < 0) { - i = 4096; - } - - Charset charset = null; - - try { - ContentType contentType = ContentType.get(entity); - if (contentType != null) { - charset = contentType.getCharset(); - } - } catch (UnsupportedCharsetException var13) { - if (defaultCharset == null) { - throw new UnsupportedEncodingException(var13.getMessage()); - } - } - - if (charset == null) { - charset = defaultCharset; - } - - if (charset == null) { - charset = HTTP.DEF_CONTENT_CHARSET; - } - - Reader reader = new InputStreamReader(instream, charset); - CharArrayBuffer buffer = new CharArrayBuffer(i); - char[] tmp = new char[1024]; - - int l; - while ((l = reader.read(tmp)) != -1) { - buffer.append(tmp, 0, l); - } - - String var9 = buffer.toString(); - return var9; - } finally { - instream.close(); - } - } - } - - public String getAccessId() { - return accessId; - } - - public String getAccessIdSecret() { - return accessIdSecret; - } - - public void setAccessId(String accessId) { - this.accessId = accessId; - } - - public void setAccessIdSecret(String accessIdSecret) { - this.accessIdSecret = accessIdSecret; - } - - @Override - public List loadConfigurableFromStorage(String type) { - return null; - } - - @Override - protected void updateConfigurable(IConfigurable configurable) { - - } - - @Override - protected void insertConfigurable(IConfigurable configurable) { - - } -} diff --git a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/HttpSupportParentConfigureService.java b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/HttpSupportParentConfigureService.java deleted file mode 100644 index 9f9d3768..00000000 --- a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/HttpSupportParentConfigureService.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.configurable.service.impl; - -import com.google.auto.service.AutoService; -import java.util.Properties; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.model.ServiceName; -import org.apache.rocketmq.streams.configurable.service.AbstractSupportParentConfigureService; -import org.apache.rocketmq.streams.configurable.service.ConfigurableServcieType; - -@AutoService(IConfigurableService.class) -@ServiceName(ConfigurableServcieType.HTTP_SERVICE_NAME) -public class HttpSupportParentConfigureService extends AbstractSupportParentConfigureService { - - private static final Log LOG = LogFactory.getLog(HttpConfigureService.class); - - public HttpSupportParentConfigureService() { - - } - - @Override - protected void initBeforeInitConfigurable(Properties property) { - parentConfigureService = new HttpConfigureService(properties); - configureService = new HttpConfigureService(properties); - - } -} diff --git a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/MemoryConfigureService.java b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/MemoryConfigureService.java deleted file mode 100644 index 3de2e4d7..00000000 --- a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/MemoryConfigureService.java +++ /dev/null @@ -1,120 +0,0 @@ -/* - * 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.configurable.service.impl; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import org.apache.rocketmq.streams.common.configurable.AbstractConfigurable; -import org.apache.rocketmq.streams.common.configurable.IConfigurable; -import org.apache.rocketmq.streams.common.utils.ReflectUtil; -import org.apache.rocketmq.streams.configurable.service.AbstractConfigurableService; - -public class MemoryConfigureService extends AbstractConfigurableService { - - private static Map> namespace2Configure = new HashMap<>(); - - public MemoryConfigureService(Properties properties) { - super(properties); - } - - @Override - protected GetConfigureResult loadConfigurable(String namespace) { - GetConfigureResult result = new GetConfigureResult(); - result.setQuerySuccess(true); - List configurableList = new ArrayList<>(); - List configurables = namespace2Configure.get(namespace); - if (configurables == null) { - configurableList = null; - } else { - List tmps = new ArrayList<>(); - tmps.addAll(configurables); - for (IConfigurable configurable : tmps) { - IConfigurable tmp = ReflectUtil.forInstance(configurable.getClass()); - tmp.toObject(configurable.toJson()); - tmp.setNameSpace(configurable.getNameSpace()); - tmp.setConfigureName(configurable.getConfigureName()); - configurableList.add(tmp); - } - } - result.setConfigurables(configurableList); - return result; - } - - @Override - protected void insertConfigurable(IConfigurable configurable) { - if (configurable == null) { - return; - } - - String namespace = configurable.getNameSpace(); - List list = namespace2Configure.get(namespace); - if (list == null) { - synchronized (this) { - list = namespace2Configure.get(namespace); - if (list == null) { - list = new ArrayList<>(); - namespace2Configure.put(namespace, list); - } - } - } - int removeIndex = -1; - for (int i = 0; i < list.size(); i++) { - IConfigurable config = list.get(i); - if (config.getType().equals(configurable.getType()) && config.getConfigureName() - .equals(configurable.getConfigureName())) { - removeIndex = i; - } - } - if (AbstractConfigurable.class.isInstance(configurable)) { - ((AbstractConfigurable)configurable).setConfigurableService(this); - } - if (removeIndex != -1) { - list.remove(removeIndex); - } - list.add(configurable); - } - - @Override - protected void updateConfigurable(IConfigurable configure) { - List list = namespace2Configure.get(configure.getNameSpace()); - if (list == null || list.size() == 0) { - throw new RuntimeException( - "not have exist configure " + configure.getNameSpace() + "," + configure.getType() + "," - + configure.getConfigureName()); - } - for (int i = 0; i < list.size(); i++) { - IConfigurable config = list.get(i); - if (config.getType().equals(configure.getType()) && config.getConfigureName() - .equals(configure.getConfigureName())) { - list.set(i, configure); - return; - } - } - throw new RuntimeException( - "not have exist configure " + configure.getNameSpace() + "," + configure.getType() + "," - + configure.getConfigureName()); - } - - @Override - public List loadConfigurableFromStorage(String type) { - refreshConfigurable(getNamespace()); - return queryConfigurableByType(type); - } -} diff --git a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/MemorySupportParentConfigureService.java b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/MemorySupportParentConfigureService.java deleted file mode 100644 index 0c8765b2..00000000 --- a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/MemorySupportParentConfigureService.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.configurable.service.impl; - -import com.google.auto.service.AutoService; -import java.util.Properties; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.model.ServiceName; -import org.apache.rocketmq.streams.configurable.service.AbstractSupportParentConfigureService; -import org.apache.rocketmq.streams.configurable.service.ConfigurableServcieType; - -@AutoService(IConfigurableService.class) -@ServiceName(ConfigurableServcieType.MEMORY_SERVICE_NAME) -public class MemorySupportParentConfigureService extends AbstractSupportParentConfigureService { - - @Override - protected void initBeforeInitConfigurable(Properties property) { - // this.rootConfigureService = new MemoryConfigureService(property); - this.parentConfigureService = new MemoryConfigureService(properties); - this.configureService = new MemoryConfigureService(properties); - } -} diff --git a/rocketmq-streams-configurable/src/test/java/org/apache/rocketmq/streams/configuable/ConfiguableComponentTest.java b/rocketmq-streams-configurable/src/test/java/org/apache/rocketmq/streams/configuable/ConfiguableComponentTest.java deleted file mode 100644 index 542193c0..00000000 --- a/rocketmq-streams-configurable/src/test/java/org/apache/rocketmq/streams/configuable/ConfiguableComponentTest.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * 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.configuable; - -import java.util.List; -import org.apache.rocketmq.streams.common.component.AbstractComponent; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.configuable.model.Person; -import org.apache.rocketmq.streams.configurable.ConfigurableComponent; -import org.junit.Test; - -import static junit.framework.TestCase.assertTrue; - -public class ConfiguableComponentTest { - - @Test - public void testInsertConfiguable(){ - String namespace="org.apache.configuable.test"; - ConfigurableComponent configurableComponent= ConfigurableComponent.getInstance(namespace); - Person person=createPerson(namespace); - configurableComponent.insert(person);//完成数据存储,在配置文件配置存储类型,支持内存,db和文件,默认是内存 - //查询只操作内存,存储的数据定时加载到内存,刚插入的数据,还未加载,查询不到 - assertTrue(configurableComponent.queryConfigurable("person","personName")==null); - configurableComponent.refreshConfigurable(namespace);//强制加载数据到内存,可以查询数据 - assertTrue(configurableComponent.queryConfigurable("person","peronName")!=null); - } - - - @Test - public void testConfiguableENVDependence(){ - String namespace="org.apache.configuable.test"; - ConfigurableComponent configurableComponent= ConfigurableComponent.getInstance(namespace); - Person person=createPerson(namespace); - person.setName("persion.name");//对于有ENVDependence的字段,可以不存储真值,存储一个key,把真值配置在配置文件中 - configurableComponent.insert(person);//完成数据存储,在配置文件配置存储类型,支持内存,db和文件,默认是内存 - ComponentCreator.getProperties().put("persion.name","realName");//这个代表真实的配置文件,启动时会把配置文件的内容加载到ComponentCreator.getProperties()中 - configurableComponent.refreshConfigurable(namespace);//刷新存储 - person=configurableComponent.queryConfigurable("person","peronName"); - assertTrue(person.getName().equals("realName")); - } - - - @Test - public void testSupportParentNameSpace(){ - String namespace="org.apache.configuable.test"; - ConfigurableComponent configurableComponent= ConfigurableComponent.getInstance(namespace); - Person person=createPerson(namespace); - Person otherPerson=createPerson("org.apache.configuable.test1"); - configurableComponent.insert(person); - configurableComponent.insert(otherPerson); - configurableComponent.refreshConfigurable(namespace); - //只加载自己命名空间的对象 - List personList=configurableComponent.queryConfigurableByType("person"); - assertTrue(personList.size()==1); - - /** - * 顶级命名空间的对象,所有namespace都可见 - */ - Person thirdPerson=createPerson(IConfigurableService.PARENT_CHANNEL_NAME_SPACE); - configurableComponent.insert(thirdPerson); - configurableComponent.refreshConfigurable(namespace);//只加载自己命名空间的对象 - personList=configurableComponent.queryConfigurableByType("person"); - assertTrue(personList.size()==2); - } - - - //测试定时加载逻辑,当对象的updateFlag值变化后,才会被替换旧对象 - @Test - public void testAutoLoader() throws InterruptedException { - ComponentCreator.getProperties().put(AbstractComponent.POLLING_TIME,"1");//1秒后动态加载对象 - String namespace="org.apache.configuable.test"; - ConfigurableComponent configurableComponent= ConfigurableComponent.getInstance(namespace); - Person person=createPerson(namespace); - configurableComponent.insert(person); - Thread.sleep(2000);//1秒后,新插入的对象会被加载 - person=configurableComponent.queryConfigurable("person","peronName"); - assertTrue(person!=null); - - - } - - /** - * 创建configuable对象 - * @param namespace - * @return - */ - protected Person createPerson(String namespace){ - Person person=new Person(); - person.setName("chris"); - person.setAge(18); - person.setNameSpace(namespace); - person.setConfigureName("peronName"); - person.setType("person"); - return person; - } -} diff --git a/rocketmq-streams-configurable/src/test/java/org/apache/rocketmq/streams/configuable/model/Person.java b/rocketmq-streams-configurable/src/test/java/org/apache/rocketmq/streams/configuable/model/Person.java deleted file mode 100644 index 057b3127..00000000 --- a/rocketmq-streams-configurable/src/test/java/org/apache/rocketmq/streams/configuable/model/Person.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * 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.configuable.model; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; -import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; -public class Person extends BasedConfigurable{ - @ENVDependence - private String name; - private int age; - private Boolean isMale; - private List addresses; - private Map childName2Age; - - public static Person createPerson(String namespace) { - Person person = new Person(); - person.setNameSpace(namespace); - person.setType("person"); - person.setConfigureName("Chris"); - person.setName("Chris"); - List addresses = new ArrayList<>(); - addresses.add("huilongguan"); - addresses.add("shangdi"); - person.setAddresses(addresses); - Map childName2Age = new HashMap<>(); - childName2Age.put("yuanyahan", 8); - childName2Age.put("yuanruxi", 4); - person.setChildName2Age(childName2Age); - person.setMale(true); - person.setAge(18); - return person; - } - - @Override - public String toString() { - return "Person{" + "name='" + name + '\'' + ", age=" + age + ", isMale=" + isMale + ", addresses=" + addresses - + ", childName2Age=" + childName2Age + '}'; - } - - public String getName() { - return name; - } - - public void setName(String name) { - this.name = name; - } - - public int getAge() { - return age; - } - - public void setAge(int age) { - this.age = age; - } - - public Boolean getMale() { - return isMale; - } - - public void setMale(Boolean male) { - isMale = male; - } - - public List getAddresses() { - return addresses; - } - - public void setAddresses(List addresses) { - this.addresses = addresses; - } - - public Map getChildName2Age() { - return childName2Age; - } - - public void setChildName2Age(Map childName2Age) { - this.childName2Age = childName2Age; - } - - @Override - public Object clone() { - Person person = null; - try { - person = (Person)super.clone(); - } catch (CloneNotSupportedException e) { - System.out.println("clone error " + e); - } - return person; - } -} diff --git a/rocketmq-streams-configurable/src/test/java/org/apache/rocketmq/streams/configurable/ConfigurableComponentTest.java b/rocketmq-streams-configurable/src/test/java/org/apache/rocketmq/streams/configurable/ConfigurableComponentTest.java deleted file mode 100644 index 0dba8d57..00000000 --- a/rocketmq-streams-configurable/src/test/java/org/apache/rocketmq/streams/configurable/ConfigurableComponentTest.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * 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.configurable; - -import java.util.List; -import org.apache.rocketmq.streams.common.component.AbstractComponent; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.configurable.model.Person; -import org.junit.Test; - -import static junit.framework.TestCase.assertTrue; - -public class ConfigurableComponentTest { - - @Test - public void testInsertConfigurable() { - String namespace = "org.apache.configurable.test"; - ConfigurableComponent configurableComponent = ConfigurableComponent.getInstance(namespace); - Person person = createPerson(namespace); - configurableComponent.insert(person);//完成数据存储,在配置文件配置存储类型,支持内存,db和文件,默认是内存 - //查询只操作内存,存储的数据定时加载到内存,刚插入的数据,还未加载,查询不到 - assertTrue(configurableComponent.queryConfigurable("person", "personName") == null); - configurableComponent.refreshConfigurable(namespace);//强制加载数据到内存,可以查询数据 - assertTrue(configurableComponent.queryConfigurable("person", "peronName") != null); - } - - @Test - public void testConfigurableENVDependence() { - String namespace = "org.apache.configurable.test"; - ConfigurableComponent configurableComponent = ConfigurableComponent.getInstance(namespace); - Person person = createPerson(namespace); - person.setName("persion.name");//对于有ENVDependence的字段,可以不存储真值,存储一个key,把真值配置在配置文件中 - configurableComponent.insert(person);//完成数据存储,在配置文件配置存储类型,支持内存,db和文件,默认是内存 - ComponentCreator.getProperties().put("persion.name", "realName");//这个代表真实的配置文件,启动时会把配置文件的内容加载到ComponentCreator.getProperties()中 - configurableComponent.refreshConfigurable(namespace);//刷新存储 - person = configurableComponent.queryConfigurable("person", "peronName"); - assertTrue(person.getName().equals("realName")); - } - - @Test - public void testSupportParentNameSpace() { - String namespace = "org.apache.configurable.test"; - ConfigurableComponent configurableComponent = ConfigurableComponent.getInstance(namespace); - Person person = createPerson(namespace); - Person otherPerson = createPerson("org.apache.configuable.test1"); - configurableComponent.insert(person); - configurableComponent.insert(otherPerson); - configurableComponent.refreshConfigurable(namespace); - //只加载自己命名空间的对象 - List personList = configurableComponent.queryConfigurableByType("person"); - assertTrue(personList.size() == 1); - - /** - * 顶级命名空间的对象,所有namespace都可见 - */ - Person thirdPerson = createPerson(IConfigurableService.PARENT_CHANNEL_NAME_SPACE); - configurableComponent.insert(thirdPerson); - configurableComponent.refreshConfigurable(namespace);//只加载自己命名空间的对象 - personList = configurableComponent.queryConfigurableByType("person"); - assertTrue(personList.size() == 2); - } - - //测试定时加载逻辑,当对象的updateFlag值变化后,才会被替换旧对象 - @Test - public void testAutoLoader() throws InterruptedException { - ComponentCreator.getProperties().put(AbstractComponent.POLLING_TIME, "1");//1秒后动态加载对象 - String namespace = "org.apache.configurable.test"; - ConfigurableComponent configurableComponent = ConfigurableComponent.getInstance(namespace); - Person person = createPerson(namespace); - configurableComponent.insert(person); - Thread.sleep(2000);//1秒后,新插入的对象会被加载 - person = configurableComponent.queryConfigurable("person", "peronName"); - assertTrue(person != null); - - } - - /** - * 创建configurable对象 - * - * @param namespace - * @return - */ - protected Person createPerson(String namespace) { - Person person = new Person(); - person.setName("chris"); - person.setAge(18); - person.setNameSpace(namespace); - person.setConfigureName("peronName"); - person.setType("person"); - return person; - } -} diff --git a/rocketmq-streams-configurable/src/test/java/org/apache/rocketmq/streams/configurable/model/Person.java b/rocketmq-streams-configurable/src/test/java/org/apache/rocketmq/streams/configurable/model/Person.java deleted file mode 100644 index 0c7de985..00000000 --- a/rocketmq-streams-configurable/src/test/java/org/apache/rocketmq/streams/configurable/model/Person.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * 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.configurable.model; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; -import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; - -public class Person extends BasedConfigurable { - @ENVDependence - private String name; - private int age; - private Boolean isMale; - private List addresses; - private Map childName2Age; - - public static Person createPerson(String namespace) { - Person person = new Person(); - person.setNameSpace(namespace); - person.setType("person"); - person.setConfigureName("Chris"); - person.setName("Chris"); - List addresses = new ArrayList<>(); - addresses.add("huilongguan"); - addresses.add("shangdi"); - person.setAddresses(addresses); - Map childName2Age = new HashMap<>(); - childName2Age.put("yuanyahan", 8); - childName2Age.put("yuanruxi", 4); - person.setChildName2Age(childName2Age); - person.setMale(true); - person.setAge(18); - return person; - } - - @Override - public String toString() { - return "Person{" + "name='" + name + '\'' + ", age=" + age + ", isMale=" + isMale + ", addresses=" + addresses - + ", childName2Age=" + childName2Age + '}'; - } - - public String getName() { - return name; - } - - public void setName(String name) { - this.name = name; - } - - public int getAge() { - return age; - } - - public void setAge(int age) { - this.age = age; - } - - public Boolean getMale() { - return isMale; - } - - public void setMale(Boolean male) { - isMale = male; - } - - public List getAddresses() { - return addresses; - } - - public void setAddresses(List addresses) { - this.addresses = addresses; - } - - public Map getChildName2Age() { - return childName2Age; - } - - public void setChildName2Age(Map childName2Age) { - this.childName2Age = childName2Age; - } - - @Override - public Object clone() { - Person person = null; - try { - person = (Person)super.clone(); - } catch (CloneNotSupportedException e) { - System.out.println("clone error " + e); - } - return person; - } -} diff --git a/rocketmq-streams-configurable/src/test/resources/log4j.xml b/rocketmq-streams-configurable/src/test/resources/log4j.xml deleted file mode 100755 index fb0edb15..00000000 --- a/rocketmq-streams-configurable/src/test/resources/log4j.xml +++ /dev/null @@ -1,36 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/rocketmq-streams-connectors/pom.xml b/rocketmq-streams-connectors/pom.xml index d544e3d5..65154ee7 100755 --- a/rocketmq-streams-connectors/pom.xml +++ b/rocketmq-streams-connectors/pom.xml @@ -15,14 +15,14 @@ See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 org.apache.rocketmq rocketmq-streams - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT rocketmq-streams-connectors jar @@ -38,10 +38,9 @@ org.apache.rocketmq rocketmq-streams-schedule - org.apache.rocketmq - rocketmq-streams-commons + rocketmq-streams-tasks diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/IBoundedSource.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/IBoundedSource.java index 1a383433..70d38208 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/IBoundedSource.java +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/IBoundedSource.java @@ -21,10 +21,11 @@ /** * @description */ -public interface IBoundedSource{ +public interface IBoundedSource { /** * reader完成时调用 + * * @param iSplit */ void boundedFinishedCallBack(ISplit iSplit); diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/ISourceBalance.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/ISourceBalance.java index b012b323..bac739b1 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/ISourceBalance.java +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/ISourceBalance.java @@ -23,25 +23,25 @@ public interface ISourceBalance { /** * 做负载均衡 - + * * @return */ SplitChanged doBalance(List allSplits, List ownerSplits); /** * 从启动开始,做了多少次均衡 + * * @return */ int getBalanceCount(); - - boolean getRemoveSplitLock(); void unLockRemoveSplitLock(); /** * lock the split and hold it util the instance is shutdown or remove split + * * @param split * @return */ @@ -49,12 +49,11 @@ public interface ISourceBalance { /** * unlock split lock + * * @param split */ void unlockSplit(ISplit split); - void setSourceIdentification(String sourceIdentification); - } diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/SplitChanged.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/SplitChanged.java index c01c1519..1ac4de38 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/SplitChanged.java +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/SplitChanged.java @@ -24,9 +24,10 @@ public class SplitChanged { protected int splitCount;//变动多分片个数 protected boolean isNewSplit;//是否新增,false是删除 protected List changedSplits; - public SplitChanged(int splitCount,boolean isNewSplit){ - this.splitCount=splitCount; - this.isNewSplit=isNewSplit; + + public SplitChanged(int splitCount, boolean isNewSplit) { + this.splitCount = splitCount; + this.isNewSplit = isNewSplit; } public int getSplitCount() { diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/impl/LeaseBalanceImpl.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/impl/LeaseBalanceImpl.java index dc504e5d..bd2bd371 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/impl/LeaseBalanceImpl.java +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/impl/LeaseBalanceImpl.java @@ -22,8 +22,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.channel.split.ISplit; import org.apache.rocketmq.streams.common.model.ServiceName; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; @@ -34,14 +32,15 @@ import org.apache.rocketmq.streams.lease.LeaseComponent; import org.apache.rocketmq.streams.lease.model.LeaseInfo; import org.apache.rocketmq.streams.lease.service.ILeaseService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @AutoService(ISourceBalance.class) @ServiceName(LeaseBalanceImpl.DB_BALANCE_NAME) public class LeaseBalanceImpl extends AbstractBalance { - private static final Log logger = LogFactory.getLog(LeaseBalanceImpl.class); - public static final String DB_BALANCE_NAME = "db_balance"; + private static final Logger LOGGER = LoggerFactory.getLogger(LeaseBalanceImpl.class); private static final String REMOVE_SPLIT_LOCK_NAME = "lock_remove_split"; private static final String SOURCE_LOCK_PREFIX = "SOURCE_"; private static final String SPLIT_LOCK_PREFIX = "SPLIT_"; @@ -62,7 +61,7 @@ public LeaseBalanceImpl() { @Override protected List fetchWorkingSplits(List allSplits) { List leaseInfos = leaseComponent.getService().queryLockedInstanceByNamePrefix(SPLIT_LOCK_PREFIX + this.sourceIdentification, null); - logger.info(String.format("lease SPLIT_LOCK_PREFIX is %s, sourceIdentification is %s. ", SPLIT_LOCK_PREFIX, sourceIdentification)); + LOGGER.info(String.format("lease SPLIT_LOCK_PREFIX is %s, sourceIdentification is %s. ", SPLIT_LOCK_PREFIX, sourceIdentification)); if (leaseInfos == null) { return new ArrayList<>(); } @@ -77,7 +76,7 @@ protected List fetchWorkingSplits(List allSplits) { String splitId = MapKeyUtil.getLast(leaseName); splits.add(allSplitMap.get(splitId)); } - logger.info(String.format("working split is %s", Arrays.toString(splits.toArray()))); + LOGGER.info(String.format("working split is %s", Arrays.toString(splits.toArray()))); return splits; } diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/model/PullMessage.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/model/PullMessage.java index 9bf34803..6d83e291 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/model/PullMessage.java +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/model/PullMessage.java @@ -37,13 +37,16 @@ public MessageOffset getMessageOffset() { public void setMessageOffset(MessageOffset messageOffset) { this.messageOffset = messageOffset; } + /** * 获取offset字符串,通过.把主offset和子offset串接在一起 + * * @return */ - public String getOffsetStr(){ - return this.messageOffset.getOffsetStr(); + public String getOffsetStr() { + return this.messageOffset.getOffsetStr(); } + public String getMainOffset() { return messageOffset.getMainOffset(); } diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/model/QuerySplit.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/model/QuerySplit.java new file mode 100644 index 00000000..524380e8 --- /dev/null +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/model/QuerySplit.java @@ -0,0 +1,91 @@ +/* + * 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.connectors.model; + +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; + +public class QuerySplit extends BasedConfigurable implements ISplit { + + protected String queueId; + protected Long initStartTime; + protected long dateAdd; + protected Long pollingMinute; + protected int pageNum; + + public QuerySplit(String queueId, Long initStartTime, long dateAdd, Long pollingMinute,int pageNum) { + this.queueId = queueId; + this.initStartTime = initStartTime; + this.dateAdd = dateAdd; + this.pollingMinute = pollingMinute; + this.pageNum=pageNum; + } + + public QuerySplit() { + } + + @Override + public String getQueueId() { + return queueId; + } + + public void setQueueId(String queueId) { + this.queueId = queueId; + } + + @Override + public String getQueue() { + return queueId; + } + + @Override + public int compareTo(QuerySplit o) { + return queueId.compareTo(o.queueId); + } + + public Long getInitStartTime() { + return initStartTime; + } + + public void setInitStartTime(Long initStartTime) { + this.initStartTime = initStartTime; + } + + public long getDateAdd() { + return dateAdd; + } + + public void setDateAdd(long dateAdd) { + this.dateAdd = dateAdd; + } + + public Long getPollingMinute() { + return pollingMinute; + } + + public void setPollingMinute(Long pollingMinute) { + this.pollingMinute = pollingMinute; + } + + public int getPageNum() { + return pageNum; + } + + public void setPageNum(int pageNum) { + this.pageNum = pageNum; + } +} \ No newline at end of file diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/model/ReaderStatus.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/model/ReaderStatus.java index a4889b5f..1764c737 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/model/ReaderStatus.java +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/model/ReaderStatus.java @@ -43,6 +43,36 @@ public class ReaderStatus extends Entity { int totalReader; + public static ReaderStatus queryReaderStatusByUK(String sourceName, String readerName) { + String sql = String.format(queryReaderStatusByUK, sourceName, readerName); + ReaderStatus readerStatus = ORMUtil.queryForObject(sql, null, ReaderStatus.class); + return readerStatus; + } + + public static List queryReaderStatusListBySourceName(String sourceName) { + String sql = String.format(queryReaderStatusList, sourceName); + List readerStatusList = ORMUtil.queryForList(sql, null, ReaderStatus.class); + return readerStatusList; + } + + public static void clearReaderStatus(String sourceName, String readerName) { + String sql = String.format(clearReaderStatus, sourceName, readerName); + ORMUtil.executeSQL(sql, null); + } + + public static ReaderStatus create(String sourceName, String readerName, int isFinished, int totalReader) { + + ReaderStatus readerStatus = new ReaderStatus(); + readerStatus.setSourceName(sourceName); + readerStatus.setReaderName(readerName); + readerStatus.setIsFinished(isFinished); + readerStatus.setTotalReader(totalReader); + readerStatus.setGmtCreate(new Date()); + readerStatus.setGmtModified(new Date()); + return readerStatus; + + } + public String getReaderName() { return readerName; } @@ -87,34 +117,4 @@ public String toString() { ", totalReader=" + totalReader + '}'; } - - public static ReaderStatus queryReaderStatusByUK(String sourceName, String readerName) { - String sql = String.format(queryReaderStatusByUK, sourceName, readerName); - ReaderStatus readerStatus = ORMUtil.queryForObject(sql, null, ReaderStatus.class); - return readerStatus; - } - - public static List queryReaderStatusListBySourceName(String sourceName) { - String sql = String.format(queryReaderStatusList, sourceName); - List readerStatusList = ORMUtil.queryForList(sql, null, ReaderStatus.class); - return readerStatusList; - } - - public static void clearReaderStatus(String sourceName, String readerName) { - String sql = String.format(clearReaderStatus, sourceName, readerName); - ORMUtil.executeSQL(sql, null); - } - - public static ReaderStatus create(String sourceName, String readerName, int isFinished, int totalReader) { - - ReaderStatus readerStatus = new ReaderStatus(); - readerStatus.setSourceName(sourceName); - readerStatus.setReaderName(readerName); - readerStatus.setIsFinished(isFinished); - readerStatus.setTotalReader(totalReader); - readerStatus.setGmtCreate(new Date()); - readerStatus.setGmtModified(new Date()); - return readerStatus; - - } } diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/StreamBuilder.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/AbstractFlinkSplitReader.java similarity index 62% rename from rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/StreamBuilder.java rename to rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/AbstractFlinkSplitReader.java index aa5edcfc..ce935ffa 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/StreamBuilder.java +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/AbstractFlinkSplitReader.java @@ -14,19 +14,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.rocketmq.streams.connectors.reader; -package org.apache.rocketmq.streams.client; +import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; +import java.util.List; +import org.apache.rocketmq.streams.common.channel.split.ISplit; -import org.apache.rocketmq.streams.client.source.DataStreamSource; +public abstract class AbstractFlinkSplitReader extends AbstractSplitReader { + protected List rows = new ArrayList<>(); -public class StreamBuilder { - - public static DataStreamSource dataStream(String namespace, String jobName) { - return DataStreamSource.create(namespace, jobName); + public AbstractFlinkSplitReader(ISplit split) { + super(split); } - public static DataStreamSource dataStream(String namespace, String jobName, String[] duplicateKeys, Long windowSize) { - return DataStreamSource.create(namespace, jobName, duplicateKeys, windowSize); + public synchronized void collect(JSONObject row) { + rows.add(row); } } diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/AbstractQueryReader.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/AbstractQueryReader.java new file mode 100644 index 00000000..30470011 --- /dev/null +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/AbstractQueryReader.java @@ -0,0 +1,68 @@ +/* + * 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.connectors.reader; + +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.connectors.model.QuerySplit; + +public abstract class AbstractQueryReader extends AbstractSplitReader { + protected long startTime; + protected long dateAdd; + protected long pollingMinute; + protected long pageNum ; + + public AbstractQueryReader(ISplit split) { + super(split); + } + + @Override public void open() { + QuerySplit dbSplit = (QuerySplit) split; + this.startTime = dbSplit.getInitStartTime(); + this.dateAdd = dbSplit.getDateAdd(); + this.pollingMinute = dbSplit.getPollingMinute(); + this.pageNum=dbSplit.getPageNum(); + + } + + @Override public boolean next() { + return startTime + pollingMinute * 60 * 1000 < System.currentTimeMillis(); + } + + + @Override public long getDelay() { + return System.currentTimeMillis() - startTime; + } + + @Override public long getFetchedDelay() { + return 0; + } + + @Override public void seek(String cursor) { + if (cursor == null) { + this.cursor = this.startTime + ";" + (pageNum + 10000); + return; + } + this.cursor = cursor; + String[] values = cursor.split(";"); + this.startTime = Long.parseLong(values[0]); + this.pageNum = Integer.parseInt(values[1]) - 10000; + } + + @Override public String getCursor() { + return this.startTime + ";" + (pageNum + 10000); + } +} diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/AbstractSplitReader.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/AbstractSplitReader.java new file mode 100644 index 00000000..abf77e1c --- /dev/null +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/AbstractSplitReader.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.connectors.reader; + +import org.apache.rocketmq.streams.common.channel.split.ISplit; + +public abstract class AbstractSplitReader implements ISplitReader { + protected ISplit split; + protected volatile boolean isInterrupt = false; + protected volatile boolean isClose = false; + + protected String cursor; + + public AbstractSplitReader(ISplit split) { + this.split = split; + } + + @Override @Deprecated public SplitCloseFuture close() { + interrupt(); + return new SplitCloseFuture(this, split); + } + + @Override public String getProgress() { + this.cursor = getCursor(); + return this.cursor; + } + + public abstract String getCursor(); + + @Override public boolean isClose() { + return isClose; + } + + @Override public ISplit getSplit() { + return this.split; + } + + @Override public boolean isInterrupt() { + return isInterrupt; + } + + @Override public void interrupt() { + isInterrupt = true; + } + + @Override public void finish() { + this.isClose = true; + } +} diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/ISplitReader.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/ISplitReader.java index 6b377cff..0090529f 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/ISplitReader.java +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/ISplitReader.java @@ -16,28 +16,20 @@ */ package org.apache.rocketmq.streams.connectors.reader; -import java.io.IOException; import java.io.Serializable; -import java.util.List; +import java.util.Iterator; import org.apache.rocketmq.streams.common.channel.split.ISplit; import org.apache.rocketmq.streams.connectors.model.PullMessage; public interface ISplitReader extends Serializable { /** - * Open. - * - * @param split the split - * @throws IOException the io exception + * init */ - void open(ISplit split); + void open(); /** * Next boolean. - * - * @return the boolean - * @throws IOException the io exception - * @throws InterruptedException the interrupted exception */ boolean next(); @@ -46,12 +38,10 @@ public interface ISplitReader extends Serializable { * * @return the message */ - List getMessage(); + Iterator> getMessage(); /** * Close. - * - * @throws IOException the io exception */ SplitCloseFuture close(); @@ -59,7 +49,6 @@ public interface ISplitReader extends Serializable { * Seek. * * @param cursor the cursor - * @throws IOException the io exception */ void seek(String cursor); @@ -67,19 +56,18 @@ public interface ISplitReader extends Serializable { * Gets progress. * * @return the progress - * @throws IOException the io exception */ String getProgress(); /** - * Get message delay (millseconds) + * Get message delay (milliseconds) * * @return delay */ long getDelay(); /** - * Get message delay (millseconds) from being fetched + * Get message delay (milliseconds) from being fetched * * @return delay */ @@ -87,10 +75,12 @@ public interface ISplitReader extends Serializable { boolean isClose(); - ISplit getSplit(); + ISplit getSplit(); boolean isInterrupt(); - boolean interrupt(); + void interrupt(); + + void finish(); } diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/AbstractPullSource.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/AbstractPullSource.java index 9aedc95a..78e26bd3 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/AbstractPullSource.java +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/AbstractPullSource.java @@ -16,297 +16,367 @@ */ package org.apache.rocketmq.streams.connectors.source; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.JSONObject; 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.Set; -import java.util.concurrent.ExecutionException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledThreadPoolExecutor; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import org.apache.commons.lang3.concurrent.BasicThreadFactory; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import java.util.function.Consumer; + 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.CheckPoint; -import org.apache.rocketmq.streams.common.checkpoint.CheckPointManager; -import org.apache.rocketmq.streams.common.context.Message; +import org.apache.rocketmq.streams.common.checkpoint.AbstractCheckPointStorage; +import org.apache.rocketmq.streams.common.checkpoint.ICheckPointStorage; +import org.apache.rocketmq.streams.common.checkpoint.ISplitOffset; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; import org.apache.rocketmq.streams.common.threadpool.ThreadPoolFactory; +import org.apache.rocketmq.streams.common.utils.CollectionUtil; +import org.apache.rocketmq.streams.common.utils.IdUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; -import org.apache.rocketmq.streams.connectors.balance.ISourceBalance; -import org.apache.rocketmq.streams.connectors.balance.SplitChanged; -import org.apache.rocketmq.streams.connectors.balance.impl.LeaseBalanceImpl; -import org.apache.rocketmq.streams.connectors.model.PullMessage; import org.apache.rocketmq.streams.connectors.reader.ISplitReader; -import org.apache.rocketmq.streams.connectors.reader.SplitCloseFuture; -import org.apache.rocketmq.streams.serviceloader.ServiceLoaderComponent; +import org.apache.rocketmq.streams.connectors.source.client.SplitConsumer; +import org.apache.rocketmq.streams.dispatcher.ICache; +import org.apache.rocketmq.streams.dispatcher.IDispatcherCallback; +import org.apache.rocketmq.streams.dispatcher.cache.DBCache; +import org.apache.rocketmq.streams.dispatcher.enums.DispatchMode; +import org.apache.rocketmq.streams.dispatcher.impl.LeaseDispatcher; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public abstract class AbstractPullSource extends AbstractSource implements IPullSource { + private static final Logger LOGGER = LoggerFactory.getLogger(AbstractPullSource.class); + protected static final String TASK_NAME_SPLIT = "->"; + protected final String START_TIME = "start_time"; + /** + * 可以有多种实现,通过名字选择不同的实现 + */ + protected long pullIntervalMs = 1000 * 20; + /** + * 每次拉取的条数 + */ + protected int pullSize = 1000; + protected transient volatile boolean shutDown = false; + protected boolean isTest = false; + protected boolean closeBalance = false; + protected transient Map ownerConsumers = new ConcurrentHashMap<>() {}; + protected transient ICache cache; + private transient LeaseDispatcher dispatcher; + private transient IDispatcherCallback balanceCallback; + private transient ExecutorService executorService; - private static final Log logger = LogFactory.getLog(AbstractPullSource.class); - - protected transient ISourceBalance balance;// balance interface - protected transient ScheduledExecutorService balanceExecutor;//schdeule balance - protected transient Map splitReaders = new HashMap<>();//owner split readers - protected transient Map ownerSplits = new HashMap<>();//working splits by the source instance + @Override + protected boolean initConfigurable() { + if (this.cache == null) { + this.cache = new DBCache(); + } + if (this.executorService == null) { + this.executorService = ThreadPoolFactory.createThreadPool(10, this.getNameSpace() + "_" + this.getName()); + } + return super.initConfigurable(); + } - //可以有多种实现,通过名字选择不同的实现 - protected String balanceName = LeaseBalanceImpl.DB_BALANCE_NAME; - //balance schedule time - protected int balanceTimeSecond = 10; - protected long pullIntervalMs; - protected transient CheckPointManager checkPointManager = new CheckPointManager(); - protected transient boolean shutDown=false; @Override protected boolean startSource() { - ServiceLoaderComponent serviceLoaderComponent = ServiceLoaderComponent.getInstance(ISourceBalance.class); - balance = (ISourceBalance) serviceLoaderComponent.getService().loadService(balanceName); - balance.setSourceIdentification(MapKeyUtil.createKey(getNameSpace(), getConfigureName())); - balanceExecutor = new ScheduledThreadPoolExecutor(1, new BasicThreadFactory.Builder().namingPattern("balance-task-%d").daemon(true).build()); - List allSplits = fetchAllSplits(); - SplitChanged splitChanged = balance.doBalance(allSplits, new ArrayList(ownerSplits.values())); - doSplitChanged(splitChanged); - balanceExecutor.scheduleWithFixedDelay(new Runnable() { - @Override - public void run() { - logger.info("balance running..... current splits is " + ownerSplits); - List allSplits = fetchAllSplits(); - SplitChanged splitChanged = balance.doBalance(allSplits, new ArrayList(ownerSplits.values())); - doSplitChanged(splitChanged); - } - }, balanceTimeSecond, balanceTimeSecond, TimeUnit.SECONDS); - + if (isTest || closeBalance) { + doSplitAddition(fetchAllSplits()); + } + if (!isTest) { + setOffsetStore(); + } + if (!closeBalance) { + startBalance(); + } startWorks(); return true; } - private void startWorks() { - ExecutorService workThreads= ThreadPoolFactory.createThreadPool(maxThread); - long start=System.currentTimeMillis(); - while (!shutDown) { - Iterator> it = splitReaders.entrySet().iterator(); - while (it.hasNext()) { - Map.Entry entry=it.next(); - String splitId=entry.getKey(); - ISplit split=ownerSplits.get(splitId); - ISplitReader reader=entry.getValue(); - ReaderRunner runner=new ReaderRunner(split,reader); - workThreads.execute(runner); - } - try { - long sleepTime=this.pullIntervalMs-(System.currentTimeMillis()-start); - if(sleepTime>0){ - Thread.sleep(sleepTime); + protected void startWorks() { + this.shutDown = false; + Thread thread = new Thread(() -> { + long start; + while (!shutDown) { + start = System.currentTimeMillis(); + try { + if (this.ownerConsumers != null && !this.ownerConsumers.isEmpty()) { + for (SplitConsumer consumer : ownerConsumers.values()) { + consumer.consume(); + } + } + } catch (Exception e) { + LOGGER.error("[{}][{}] Pull_Reader_Execute_Error", IdUtil.instanceId(), getName(), e); } - } catch (InterruptedException e) { - e.printStackTrace(); + waitPollingTime(start); } - } + }); + thread.start(); } @Override - public Map getAllSplitMap() { - List splits = fetchAllSplits(); + public Map> getAllSplitMap() { + List> splits = fetchAllSplits(); if (splits == null) { return new HashMap<>(); } - Map splitMap = new HashMap<>(); - for (ISplit split : splits) { + Map> splitMap = new HashMap<>(); + for (ISplit split : splits) { splitMap.put(split.getQueueId(), split); } return splitMap; } - protected void doSplitChanged(SplitChanged splitChanged) { - if (splitChanged == null) { - return; - } - if (splitChanged.getSplitCount() == 0) { - return; - } - if (splitChanged.isNewSplit()) { - doSplitAddition(splitChanged.getChangedSplits()); - } else { - doSplitRelease(splitChanged.getChangedSplits()); - } + public List> getAllSplits() { + List> splits = fetchAllSplits(); + return new ArrayList<>(splits); } - protected void doSplitAddition(List changedSplits) { + protected synchronized void doSplitAddition(List> changedSplits) { if (changedSplits == null) { return; } - Set splitIds = new HashSet<>(); - for (ISplit split : changedSplits) { - splitIds.add(split.getQueueId()); + for (ISplit split : changedSplits) { + if (ownerConsumers.containsKey(split.getQueueId())) { + continue; + } + SplitConsumer splitConsumer = new SplitConsumer(this, split, executorService); + splitConsumer.open(); + ownerConsumers.put(split.getQueueId(), splitConsumer); } - addNewSplit(splitIds); - for (ISplit split : changedSplits) { - ISplitReader reader = createSplitReader(split); - reader.open(split); - reader.seek(loadSplitOffset(split)); - splitReaders.put(split.getQueueId(), reader); - this.ownerSplits.put(split.getQueueId(), split); -// logger.info("start next"); -// Thread thread = new Thread(new Runnable() { -// -// thread.setName("reader-task-" + reader.getSplit().getQueueId()); -// thread.start(); + } + + protected synchronized void doSplitRelease(List> changedSplits) { + + try { + for (ISplit split : changedSplits) { + SplitConsumer consumer = this.ownerConsumers.get(split.getQueueId()); + if (consumer == null) { + continue; + } + consumer.destroy(); + this.ownerConsumers.remove(split.getQueueId()); + } + } catch (Exception e) { + throw new RuntimeException(e); } } @Override - public String loadSplitOffset(ISplit split) { + public String loadSplitOffset(ISplit split) { String offset = null; - CheckPoint checkPoint = checkPointManager.recover(this, split); + ISplitOffset checkPoint = checkPointManager.recover(this, split); if (checkPoint != null) { - offset = JSON.parseObject(checkPoint.getData()).getString("offset"); + offset = checkPoint.getOffset(); } return offset; } - protected abstract ISplitReader createSplitReader(ISplit split); + public ISplitReader createReader(ISplit split) { + return createSplitReader(split); + } - protected void doSplitRelease(List changedSplits) { - boolean success = balance.getRemoveSplitLock(); - if (!success) { - return; + protected abstract ISplitReader createSplitReader(ISplit split); + + protected List> getSplitByName(List names) { + if (CollectionUtil.isEmpty(names)) { + return null; } - try { - List closeFutures = new ArrayList<>(); - for (ISplit split : changedSplits) { - ISplitReader reader = this.splitReaders.get(split.getQueueId()); - if (reader == null) { - continue; - } - SplitCloseFuture future = reader.close(); - closeFutures.add(future); + Map> allSplits = getAllSplitMap(); + List> splits = new ArrayList<>(); + for (String name : names) { + ISplit split = allSplits.get(name); + if (split == null) { + LOGGER.warn("[{}][{}] Source_Get_Split_Error({})", IdUtil.instanceId(), getName(), name); + continue; } - for (SplitCloseFuture future : closeFutures) { - try { - if(!future.isDone()){ - future.get(); - } - this.splitReaders.remove(future.getSplit().getQueueId()); - this.ownerSplits.remove(future.getSplit().getQueueId()); - } catch (InterruptedException e) { - e.printStackTrace(); - } catch (ExecutionException e) { - e.printStackTrace(); + splits.add(split); + } + return splits; + + } + + protected void setOffsetStore() { + ICheckPointStorage iCheckPointStorage = new AbstractCheckPointStorage() { + + @Override + public String getStorageName() { + return "db"; + } + + @Override + public void save(List checkPointState) { + if (checkPointState == null) { + return; + } + for (T splitOffset : checkPointState) { + String namespace = MapKeyUtil.createKey(getNameSpace(), getName()); + String queueId = splitOffset.getQueueId(); + String offset = splitOffset.getOffset(); + cache.putKeyConfig(namespace, queueId, offset); } + } - } finally { - balance.unLockRemoveSplitLock(); - } + @Override + public ISplitOffset recover(ISource iSource, String queueID) { + String name = MapKeyUtil.createKey(iSource.getNameSpace(), iSource.getName()); + String offset = cache.getKeyConfig(name, queueID); + return new ISplitOffset() { + + @Override + public String getName() { + return name; + } - } + @Override + public String getQueueId() { + return queueID; + } + @Override + public String getOffset() { + return offset; + } + }; + } + }; + checkPointManager.setiCheckPointStorage(iCheckPointStorage); + } - protected class ReaderRunner implements Runnable{ - long mLastCheckTime = System.currentTimeMillis(); - protected ISplit split; - protected ISplitReader reader; + protected void startBalance() { + try { + if (this.balanceCallback == null) { + this.balanceCallback = new IDispatcherCallback<>() { + + @Override + public List start(List jobNames) { + List names = splitQueueIds(jobNames); + List> additionSplits = getSplitByName(names); + if (CollectionUtil.isEmpty(additionSplits)) { + return jobNames; + } + doSplitAddition(additionSplits); + return jobNames; + } - public ReaderRunner(ISplit split,ISplitReader reader){ - this.split=split; - this.reader=reader; - } + @Override + public List stop(List jobNames) { + List names = splitQueueIds(jobNames); + List> additionSplits = getSplitByName(names); + if (CollectionUtil.isEmpty(additionSplits)) { + return jobNames; + } + doSplitRelease(additionSplits); + return jobNames; + } - @Override - public void run() { - logger.info("start running"); - if (reader.isInterrupt() == false) { - if (reader.next()) { - List messages = reader.getMessage(); - if (messages != null) { - for (PullMessage pullMessage : messages) { - String queueId = split.getQueueId(); - String offset = pullMessage.getOffsetStr(); - JSONObject msg = createJson(pullMessage.getMessage()); - Message message = createMessage(msg, queueId, offset, false); - message.getHeader().setOffsetIsLong(pullMessage.getMessageOffset().isLongOfMainOffset()); - executeMessage(message); + @Override + public List list() { + List> splits = getAllSplits(); + List names = new ArrayList<>(); + for (ISplit split : splits) { + names.add(getName() + TASK_NAME_SPLIT + split.getQueueId()); } + return names; } - reader.notifyAll(); - } - long curTime = System.currentTimeMillis(); - if (curTime - mLastCheckTime > getCheckpointTime()) { - sendCheckpoint(reader.getSplit().getQueueId()); - mLastCheckTime = curTime; - } + @Override + public void destroy() { - }else { - Set removeSplits = new HashSet<>(); - removeSplits.add(reader.getSplit().getQueueId()); - removeSplit(removeSplits); - balance.unlockSplit(split); - reader.close(); - synchronized (reader) { - reader.notifyAll(); - } + } + }; } - + if (this.dispatcher == null) { + String jdbc = getConfiguration().getProperty(ConfigurationKey.JDBC_DRIVER); + String url = getConfiguration().getProperty(ConfigurationKey.JDBC_URL); + String userName = getConfiguration().getProperty(ConfigurationKey.JDBC_USERNAME); + String password = getConfiguration().getProperty(ConfigurationKey.JDBC_PASSWORD); + int scheduleTime = Integer.parseInt(getConfiguration().getProperty(ConfigurationKey.DIPPER_DISPATCHER_SCHEDULE_TIME, "60")); + this.dispatcher = new LeaseDispatcher<>(jdbc, url, userName, password, IdUtil.workerId(), getNameSpace() + "_" + getName(), DispatchMode.AVERAGELY, scheduleTime, this.balanceCallback, new DBCache(), "partition"); + } + this.dispatcher.start(); + } catch (Exception e) { + throw new RuntimeException(e); } + } + protected void waitPollingTime(long start) { + try { + long sleepTime = this.pullIntervalMs - (System.currentTimeMillis() - start); + if (sleepTime > 0) { + Thread.sleep(sleepTime); + } + } catch (InterruptedException e) { + e.printStackTrace(); + } } @Override - public boolean supportNewSplitFind() { - return true; + public void destroySource() { + if (this.dispatcher != null) { + this.dispatcher.close(); + } + List> splits = new ArrayList<>(); + for (SplitConsumer splitConsumer : this.ownerConsumers.values()) { + splits.add(splitConsumer.getSplit()); + } + doSplitRelease(new ArrayList<>(splits)); + this.ownerConsumers = new HashMap<>(); + this.shutDown = true; } @Override - public boolean supportRemoveSplitFind() { - return true; + public Long getPullIntervalMs() { + return pullIntervalMs; } - @Override - public boolean supportOffsetRest() { - return true; + public void setPullIntervalMs(long pullIntervalMs) { + this.pullIntervalMs = pullIntervalMs; } @Override - public Long getPullIntervalMs() { - return pullIntervalMs; + public List> ownerSplits() { + List> splits = new ArrayList<>(); + for (SplitConsumer splitConsumer : this.ownerConsumers.values()) { + splits.add(splitConsumer.getSplit()); + } + return splits; } - public String getBalanceName() { - return balanceName; + protected List splitQueueIds(List taskNames) { + if (taskNames == null) { + return new ArrayList<>(); + } + List names = new ArrayList<>(); + for (String taskName : taskNames) { + names.add(taskName.split(TASK_NAME_SPLIT)[1]); + } + return names; } - public void setBalanceName(String balanceName) { - this.balanceName = balanceName; + public boolean isTest() { + return isTest; } - public int getBalanceTimeSecond() { - return balanceTimeSecond; + public void setTest(boolean test) { + isTest = test; } - public void setBalanceTimeSecond(int balanceTimeSecond) { - this.balanceTimeSecond = balanceTimeSecond; + public boolean isCloseBalance() { + return closeBalance; } - public void setPullIntervalMs(long pullIntervalMs) { - this.pullIntervalMs = pullIntervalMs; + public void setCloseBalance(boolean closeBalance) { + this.closeBalance = closeBalance; } - @Override - public List ownerSplits() { - return new ArrayList(ownerSplits.values()); + public int getPullSize() { + return pullSize; } + public void setPullSize(int pullSize) { + this.pullSize = pullSize; + } } \ No newline at end of file diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/AbstractQuerySource.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/AbstractQuerySource.java new file mode 100644 index 00000000..7c577fd8 --- /dev/null +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/AbstractQuerySource.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.connectors.source; + +import java.util.ArrayList; +import java.util.Date; +import java.util.List; +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.connectors.model.QuerySplit; + +public abstract class AbstractQuerySource extends AbstractPullSource { + protected String initFirstTime = DateUtil.getCurrentTimeString(); + protected Long pollingMinute = 10L; + protected int splitCount = 1;//在一个周期内的并发性 + protected int pageInit=0; + + + @Override public List> fetchAllSplits() { + List> splits = new ArrayList<>(); + long baseDataAdd = pollingMinute / splitCount; + long remainder = pollingMinute % splitCount; + Date date = DateUtil.getWindowBeginTime(DateUtil.parseTime(initFirstTime).getTime(), pollingMinute*60*1000); + for (int i = 0; i < splitCount; i++) { + long dataAdd = baseDataAdd + (remainder > 0 ? 1 : 0); + remainder--; + Date endDate = DateUtil.addMinute(date, (int) dataAdd); + QuerySplit openAPISplit = new QuerySplit(String.valueOf(i), date.getTime(), dataAdd, pollingMinute,pageInit); + splits.add(openAPISplit); + date = endDate; + } + return splits; + } + + protected long getEndTime(long startTime,long dateAdd){ + return startTime + dateAdd * 60 * 1000; + } + + public String getInitFirstTime() { + return initFirstTime; + } + + public void setInitFirstTime(String initFirstTime) { + this.initFirstTime = initFirstTime; + } + + public Long getPollingMinute() { + return pollingMinute; + } + + public void setPollingMinute(Long pollingMinute) { + this.pollingMinute = pollingMinute; + } + + public int getSplitCount() { + return splitCount; + } + + public void setSplitCount(int splitCount) { + this.splitCount = splitCount; + } + + public int getPageInit() { + return pageInit; + } + + public void setPageInit(int pageInit) { + this.pageInit = pageInit; + } +} diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/IPullSource.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/IPullSource.java index 6733911d..01b30ad3 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/IPullSource.java +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/IPullSource.java @@ -17,7 +17,6 @@ package org.apache.rocketmq.streams.connectors.source; import java.util.Collection; -import java.util.List; import java.util.Map; import org.apache.rocketmq.streams.common.channel.source.ISource; import org.apache.rocketmq.streams.common.channel.split.ISplit; @@ -25,28 +24,21 @@ /** * poll message,need balance */ -public interface IPullSource extends ISource { +public interface IPullSource> extends ISource { /** * 拥有的分片格式 * * @return */ - Collection ownerSplits(); + Collection> ownerSplits(); /** * get all split for the source * * @return */ - List fetchAllSplits(); - - /** - * get all split for the source - * - * @return - */ - Map getAllSplitMap(); + Map> getAllSplitMap(); Long getPullIntervalMs(); @@ -55,6 +47,6 @@ public interface IPullSource extends ISource { * * @return */ - String loadSplitOffset(ISplit split); + String loadSplitOffset(ISplit split); } diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/MutilBatchTaskSource.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/MutilBatchTaskSource.java deleted file mode 100644 index d82ba32a..00000000 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/MutilBatchTaskSource.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * 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.connectors.source; - -import com.alibaba.fastjson.JSON; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.rocketmq.streams.common.channel.split.ISplit; -import org.apache.rocketmq.streams.common.checkpoint.CheckPoint; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; -import org.apache.rocketmq.streams.common.topology.model.Pipeline; -import org.apache.rocketmq.streams.common.topology.task.TaskAssigner; -import org.apache.rocketmq.streams.common.utils.DipperThreadLocalUtil; -import org.apache.rocketmq.streams.common.utils.RuntimeUtil; -import org.apache.rocketmq.streams.connectors.model.PullMessage; -import org.apache.rocketmq.streams.connectors.reader.ISplitReader; -import org.apache.rocketmq.streams.connectors.reader.SplitCloseFuture; - -public class MutilBatchTaskSource extends AbstractPullSource { - - @Override protected ISplitReader createSplitReader(ISplit split) { - return new ISplitReader() { - protected transient ISplit split; - protected boolean isInterrupt; - protected boolean isClose; - protected transient AtomicLong offsetGenerator=new AtomicLong(1000000000); - @Override public void open(ISplit split) { - this.split=split; - } - - @Override public boolean next() { - return true; - } - - @Override public List getMessage() { - PipelineSplit pipelineSplit=(PipelineSplit)split; - pipelineSplit.getQueue().startChannel(); - return null; - } - - @Override public SplitCloseFuture close() { - isClose=true; - return new SplitCloseFuture(this,split); - } - - @Override public void seek(String cursor) { - - } - - @Override public String getProgress() { - return RuntimeUtil.getDipperInstanceId()+"_"+offsetGenerator.incrementAndGet(); - } - - @Override public long getDelay() { - return 0; - } - - @Override public long getFetchedDelay() { - return getPullIntervalMs(); - } - - @Override public boolean isClose() { - return isClose; - } - - @Override public ISplit getSplit() { - return split; - } - - @Override public boolean isInterrupt() { - return isInterrupt; - } - - @Override public boolean interrupt() { - isInterrupt=true; - return isInterrupt; - } - }; - } - - @Override protected boolean isNotDataSplit(String queueId) { - return false; - } - - @Override public List fetchAllSplits() { - - List taskAssigners = configurableService.queryConfigurableByType(TaskAssigner.TYPE); - if (taskAssigners == null) { - return null; - } - String taskName = getConfigureName(); - List splits=new ArrayList<>(); - for (TaskAssigner taskAssigner : taskAssigners) { - if (!taskName.equals(taskAssigner.getTaskName())) { - continue; - } - String pipelineName = taskAssigner.getPipelineName(); - if(pipelineName!=null){ - ChainPipeline pipeline = configurableService.queryConfigurable(Pipeline.TYPE, pipelineName); - if (pipeline != null) { - splits.add(new PipelineSplit(pipeline)); - } - } - } - return splits; - } - - - protected class PipelineSplit implements ISplit{ - protected ChainPipeline chainPipeline; - public PipelineSplit(ChainPipeline chainPipeline){ - this.chainPipeline=chainPipeline; - } - - @Override public String getQueueId() { - return chainPipeline.getConfigureName(); - } - - @Override public ChainPipeline getQueue() { - return chainPipeline; - } - - @Override public int compareTo(PipelineSplit o) { - return chainPipeline.getConfigureName().compareTo(o.getQueueId()); - } - - @Override public String toJson() { - return chainPipeline.toJson(); - } - - @Override public void toObject(String jsonString) { - ChainPipeline pipeline=new ChainPipeline(); - pipeline.toObject(jsonString); - this.chainPipeline=pipeline; - } - } - - @Override - public String loadSplitOffset(ISplit split) { - return null; - } - -} diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/SourceInstance.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/SourceInstance.java index c0da5b6e..edb1eca3 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/SourceInstance.java +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/SourceInstance.java @@ -22,9 +22,8 @@ public class SourceInstance { protected String sourceInstanceId; - - public SourceInstance(String sourceInstanceId){ - this.sourceInstanceId=sourceInstanceId; + public SourceInstance(String sourceInstanceId) { + this.sourceInstanceId = sourceInstanceId; } public String getSourceInstanceId() { diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/client/SplitConsumer.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/client/SplitConsumer.java new file mode 100644 index 00000000..772d37c5 --- /dev/null +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/client/SplitConsumer.java @@ -0,0 +1,162 @@ +/* + * 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.connectors.source.client; + +import com.alibaba.fastjson.JSONObject; + +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; +import java.util.concurrent.ExecutorService; + +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.context.Message; +import org.apache.rocketmq.streams.common.context.MessageOffset; +import org.apache.rocketmq.streams.common.utils.IdUtil; +import org.apache.rocketmq.streams.connectors.model.PullMessage; +import org.apache.rocketmq.streams.connectors.reader.ISplitReader; +import org.apache.rocketmq.streams.connectors.source.AbstractPullSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 一个分片的pull消费 + */ +public class SplitConsumer { + private static final Logger LOGGER = LoggerFactory.getLogger(SplitConsumer.class); + private final AbstractPullSource pullSource; + private final ISplit split; + private final ExecutorService executorService; + private SplitStatus splitStatus; + private volatile boolean isFinished = true; + private Long mLastCheckTime = System.currentTimeMillis(); + private ISplitReader reader; + + public SplitConsumer(AbstractPullSource pullSource, ISplit split, ExecutorService executorService) { + this.pullSource = pullSource; + this.split = split; + this.executorService = executorService; + this.splitStatus = SplitStatus.INIT; + } + + public void open() { + try { + LOGGER.info("[{}][{}] Split_Open_On({})", IdUtil.instanceId(), pullSource.getName(), split.getQueueId()); + ISplitReader reader = pullSource.createReader(split); + reader.open(); + String offset = pullSource.loadSplitOffset(split); + reader.seek(offset); + LOGGER.info("[{}][{}] Split_Reset_To({}:{})", IdUtil.instanceId(), pullSource.getName(), split.getQueueId(), offset == null ? "" : offset); + this.reader = reader; + Set splits = new HashSet<>(); + splits.add(split.getQueueId()); + this.pullSource.addNewSplit(splits); + splitStatus = SplitStatus.CONSUME; + } catch (Exception e) { + String error = "[{}][{}] Split_Open_Error_On({})"; + LOGGER.error(error, IdUtil.instanceId(), pullSource.getName(), split.getQueueId()); + throw new RuntimeException(e); + } + + } + + public void consume() { + if (isFinished) { + isFinished = false; + this.executorService.submit(this::exePullTask); + } + } + + public void destroy() { + try { + LOGGER.info("[{}][{}] Split_Destroy_On({})", IdUtil.instanceId(), pullSource.getName(), split.getQueueId()); + reader.interrupt(); + splitStatus = SplitStatus.INTERRUPT; + Set splitIds = new HashSet<>(); + splitIds.add(reader.getSplit().getQueueId()); + pullSource.removeSplit(splitIds); + saveOffset(reader); + reader.finish(); + splitStatus = SplitStatus.CLOSED; + } catch (Exception e) { + String error = "[{}][{}] Split_Destroy_Error_On({})"; + LOGGER.error(error, IdUtil.instanceId(), pullSource.getName(), split.getQueueId()); + throw new RuntimeException(error, e); + } + + } + + protected void exePullTask() { + try { + if (!reader.isInterrupt()) { + if (splitStatus == SplitStatus.CONSUME && reader.next()) { + LOGGER.info("[{}][{}] Split_Pull_Message_On({})", IdUtil.instanceId(), pullSource.getName(), split.getQueueId()); + Iterator> pullMessageIterator = reader.getMessage(); + if (pullMessageIterator != null) { + while (pullMessageIterator.hasNext()) { + PullMessage pullMessage = pullMessageIterator.next(); + String queueId = split.getQueueId(); + String offset = pullMessage.getOffsetStr(); + JSONObject msg = pullSource.createJson(pullMessage.getMessage()); + Message message = pullSource.createMessage(msg, queueId, offset, false); + message.getHeader().setOffsetIsLong(pullMessage.getMessageOffset().isLongOfMainOffset()); + pullSource.executeMessage(message); + } + } + } + saveOffset(reader); + isFinished = true; + } + } catch (Exception e) { + String error = "[{}][{}] Split_Pull_Message_Error_On({})"; + LOGGER.error(error, IdUtil.instanceId(), pullSource.getName(), split.getQueueId(), e); + throw new RuntimeException(error, e); + } + } + + /** + * save offset + * + * @param reader + */ + private void saveOffset(ISplitReader reader) { + long curTime = System.currentTimeMillis(); + if (curTime - mLastCheckTime > pullSource.getCheckpointTime()) { + if (!pullSource.isTest()) { + LOGGER.info("[{}][{}] Split_Save_The_Progress_On({}:{})", IdUtil.instanceId(), pullSource.getName(), split.getQueueId(), reader.getProgress()); + } + LOGGER.info("[{}][{}] Source_Delayed_On_Queue({})_For({})", IdUtil.instanceId(), pullSource.getName(), split.getQueueId(), reader.getDelay()); + pullSource.sendCheckpoint(reader.getSplit().getQueueId(), new MessageOffset(reader.getProgress())); + mLastCheckTime = curTime; + } else { + if (!pullSource.isTest()) { + LOGGER.debug("[{}][{}] Split_Save_The_Progress_ON({}:{}-{})", IdUtil.instanceId(), pullSource.getName(), split.getQueueId(), curTime, mLastCheckTime); + } + } + } + + public ISplit getSplit() { + return split; + } + + protected enum SplitStatus { + INIT, + CONSUME, + INTERRUPT, + CLOSED + } +} diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/AbstractPatternFilter.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/AbstractPatternFilter.java index 0d5368a7..c171a286 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/AbstractPatternFilter.java +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/AbstractPatternFilter.java @@ -29,7 +29,6 @@ public abstract class AbstractPatternFilter implements PatternFilter, Serializab public abstract boolean filter(String sourceName, String logicTableName, String tableName); - @Override public PatternFilter setNext(PatternFilter filter) { this.next = filter; diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/BoundedPatternFilter.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/BoundedPatternFilter.java index c06de98d..a5a0a06c 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/BoundedPatternFilter.java +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/BoundedPatternFilter.java @@ -17,9 +17,9 @@ package org.apache.rocketmq.streams.connectors.source.filter; import java.io.Serializable; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.connectors.model.ReaderStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * @description 过滤掉已经完成的reader @@ -27,15 +27,15 @@ @Deprecated public class BoundedPatternFilter extends AbstractPatternFilter implements Serializable { - static final Log logger = LogFactory.getLog(BoundedPatternFilter.class); + static final Logger LOGGER = LoggerFactory.getLogger(BoundedPatternFilter.class); @Override public boolean filter(String sourceName, String logicTableName, String tableName) { ReaderStatus readerStatus = ReaderStatus.queryReaderStatusByUK(sourceName, logicTableName + "_" + tableName); if (readerStatus != null) { - logger.info(String.format("filter sourceName %s, logicTableName %s, suffix %s. ", sourceName, logicTableName, tableName)); - logger.info(String.format("query result %s", readerStatus.toString())); + LOGGER.info(String.format("filter sourceName %s, logicTableName %s, suffix %s. ", sourceName, logicTableName, tableName)); + LOGGER.info(String.format("query result %s", readerStatus.toString())); return true; } if (next == null) { diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CyclePatternFilter.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CyclePatternFilter.java index 3a0193f3..24b827c4 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CyclePatternFilter.java +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CyclePatternFilter.java @@ -28,27 +28,18 @@ */ public class CyclePatternFilter extends AbstractPatternFilter implements Serializable { - private static final long serialVersionUID = -5151597286296228754L; - public static final int INIT_CYCLE_VERSION = 0; - + private static final long serialVersionUID = -5151597286296228754L; + //历史数据读取时使用,表示比起当前相差多少个调度周期 + final long cycleDiff; CyclePeriod cyclePeriod; - Date curCycleDateTime; //当前调度周期时间 - long cycleId; - String firstStartTime; //当前最小时间 - List allPatterns; - String expression; - boolean isInit; - //历史数据读取时使用,表示比起当前相差多少个调度周期 - final long cycleDiff; - //todo expr解析 public CyclePatternFilter(String expr, Date date) throws ParseException { expression = expr; @@ -57,50 +48,66 @@ public CyclePatternFilter(String expr, Date date) throws ParseException { curCycleDateTime = calCycleDateTime(date); allPatterns = new ArrayList<>(); isInit = true; - if(cyclePeriod.isHistory){ + if (cyclePeriod.isHistory) { Date tmp = cyclePeriod.getHisDate(); - cycleDiff = curCycleDateTime.getTime()/1000 * 1000 - tmp.getTime()/1000*1000; - }else{ + cycleDiff = curCycleDateTime.getTime() / 1000 * 1000 - tmp.getTime() / 1000 * 1000; + } else { cycleDiff = 0; } } + public static void main(String[] args) throws ParseException { + + CyclePatternFilter cycle = new CyclePatternFilter("yyyyMMddHHmm - 15m", new Date()); + System.out.println(cycle); + + System.out.println(cycle.filter(null, null, "202109131650")); + System.out.println(cycle.filter(null, null, "20210902000000")); + System.out.println(cycle.filter(null, null, "20210908000000")); + System.out.println(cycle.filter(null, null, "20210910000000")); + System.out.println(cycle.filter(null, null, "20210909230000")); + + System.out.println(new SimpleDateFormat("yyyyMMddHH").parse("2021090923")); + System.out.println(new SimpleDateFormat("yyyyMMddhhmmss").parse("20210909230000")); + System.out.println(new SimpleDateFormat("yyyyMMddHHmmss").parse("20210909100000")); + System.out.println(new SimpleDateFormat("yyyyMMddhhmmss").parse("20210909100000")); + + } /** - * * @return 返回date格式的调度周期时间 */ - private Date calCycleDateTime(Date date){ + private Date calCycleDateTime(Date date) { return cyclePeriod.format(date); } - private long calCycle(Date date){ + private long calCycle(Date date) { Date tmp = calCycleDateTime(date); - if(tmp.getTime()/1000 == curCycleDateTime.getTime()/1000){ + if (tmp.getTime() / 1000 == curCycleDateTime.getTime() / 1000) { return cycleId; } return nextCycle(tmp); } - private long nextCycle(Date date){ + private long nextCycle(Date date) { curCycleDateTime = date; cycleId++; calAllPattern(); return cycleId; } - private void calAllPattern(){ + private void calAllPattern() { allPatterns.clear(); - for(int i = 1; i <= cyclePeriod.getCycle(); i++){ - long d = (curCycleDateTime.getTime()/1000)*1000 - i * cyclePeriod.getInterval() - cycleDiff; + for (int i = 1; i <= cyclePeriod.getCycle(); i++) { + long d = (curCycleDateTime.getTime() / 1000) * 1000 - i * cyclePeriod.getInterval() - cycleDiff; String s = cyclePeriod.getDateFormat().format(new Date(d)); allPatterns.add(s); } firstStartTime = allPatterns.get(allPatterns.size() - 1); } - public boolean isNextCycle(Date date){ - if(isInit){ + public boolean isNextCycle(Date date) { + if (isInit) { isInit = false; calAllPattern(); return true; @@ -117,11 +124,11 @@ public long getCycleId() { return cycleId; } - public Date getCurCycleDateTime(){ + public Date getCurCycleDateTime() { return curCycleDateTime; } - public String getCurCycleDateTimeStr(){ + public String getCurCycleDateTimeStr() { return cyclePeriod.getDateFormat().format(curCycleDateTime); } @@ -129,11 +136,11 @@ public long getCycleDiff() { return cycleDiff; } - public long getCyclePeriodDiff(){ - return cycleDiff/cyclePeriod.getInterval(); + public long getCyclePeriodDiff() { + return cycleDiff / cyclePeriod.getInterval(); } - public int getCycle(){ + public int getCycle() { return cyclePeriod.getCycle(); } @@ -146,28 +153,4 @@ public boolean filter(String sourceName, String logicTableName, String tableName return allPatterns.contains(tableName); } - - - public static void main(String[] args) throws ParseException { - - CyclePatternFilter cycle = new CyclePatternFilter("yyyyMMddHHmm - 15m", new Date()); - System.out.println(cycle); - - System.out.println(cycle.filter(null, null, "202109131650")); - System.out.println(cycle.filter(null, null, "20210902000000")); - System.out.println(cycle.filter(null, null, "20210908000000")); - System.out.println(cycle.filter(null, null, "20210910000000")); - System.out.println(cycle.filter(null, null, "20210909230000")); - - System.out.println(new SimpleDateFormat("yyyyMMddHH").parse("2021090923")); - System.out.println(new SimpleDateFormat("yyyyMMddhhmmss").parse("20210909230000")); - System.out.println(new SimpleDateFormat("yyyyMMddHHmmss").parse("20210909100000")); - System.out.println(new SimpleDateFormat("yyyyMMddhhmmss").parse("20210909100000")); - - - - - } - - } diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CyclePeriod.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CyclePeriod.java index 4e6cdd6a..ec6c0202 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CyclePeriod.java +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CyclePeriod.java @@ -19,8 +19,8 @@ import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.Date; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * @Description @@ -101,17 +101,36 @@ Date format(Date strDate) { }; + static final Logger LOGGER = LoggerFactory.getLogger(CyclePeriod.class); boolean isHistory = false; - long interval; - int cycle; - String format; - String hisDateString; - static final Log logger = LogFactory.getLog(CyclePeriod.class); + public static CyclePeriod getInstance(String expression) throws ParseException { + + String[] str = expression.split("\\-"); + assert str.length == 2 : String.format("expression error : %s. ", expression); + String expr = str[0].trim(); + String tmp = str[1].trim().toLowerCase(); + String cycleStr = tmp.substring(0, tmp.length() - 1); + int cycle = Integer.parseInt(cycleStr); + CyclePeriod cyclePeriod = null; + if (tmp.endsWith("d")) { + cyclePeriod = CYCLE_PERIOD_DATE; + } else if (tmp.endsWith("h")) { + cyclePeriod = CYCLE_PERIOD_HOUR; + } else if (tmp.endsWith("m")) { + cyclePeriod = CYCLE_PERIOD_MINUTE; + } else { + new RuntimeException(String.format("unsupported format : %s", expression)); + } + cyclePeriod.argsParser(expr); + cyclePeriod.cycle = cycle; + + return cyclePeriod; + } void argsParser(String expr) throws ParseException { if (expr.matches("^\\d+$")) { @@ -141,7 +160,7 @@ final boolean checkFormat(String expr, String format) { new SimpleDateFormat(format).parse(expr); return true; } catch (ParseException e) { - logger.error(String.format("error format, expr is %s, format is %s.", expr, format)); + LOGGER.error(String.format("error format, expr is %s, format is %s.", expr, format)); e.printStackTrace(); return false; } @@ -159,6 +178,10 @@ public long getInterval() { return interval; } + public void setInterval(long interval) { + this.interval = interval; + } + public boolean isHistory() { return isHistory; } @@ -167,10 +190,6 @@ public void setHistory(boolean history) { isHistory = history; } - public void setInterval(long interval) { - this.interval = interval; - } - public int getCycle() { return cycle; } @@ -179,14 +198,14 @@ public void setCycle(int cycle) { this.cycle = cycle; } - public void setFormat(String format) { - this.format = format; - } - public String getFormat() { return format; } + public void setFormat(String format) { + this.format = format; + } + public String getHisDateString() { return hisDateString; } @@ -195,28 +214,4 @@ public void setHisDateString(String hisDateString) { this.hisDateString = hisDateString; } - public static CyclePeriod getInstance(String expression) throws ParseException { - - String[] str = expression.split("\\-"); - assert str.length == 2 : String.format("expression error : %s. ", expression); - String expr = str[0].trim(); - String tmp = str[1].trim().toLowerCase(); - String cycleStr = tmp.substring(0, tmp.length() - 1); - int cycle = Integer.parseInt(cycleStr); - CyclePeriod cyclePeriod = null; - if (tmp.endsWith("d")) { - cyclePeriod = CYCLE_PERIOD_DATE; - } else if (tmp.endsWith("h")) { - cyclePeriod = CYCLE_PERIOD_HOUR; - } else if (tmp.endsWith("m")) { - cyclePeriod = CYCLE_PERIOD_MINUTE; - } else { - new RuntimeException(String.format("unsupported format : %s", expression)); - } - cyclePeriod.argsParser(expr); - cyclePeriod.cycle = cycle; - - return cyclePeriod; - } - } diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CycleSchedule.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CycleSchedule.java index ba9a2797..22fc11d8 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CycleSchedule.java +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CycleSchedule.java @@ -30,15 +30,15 @@ */ public class CycleSchedule implements Serializable { - private static final long serialVersionUID = -5151597286296228754L; public static final int INIT_CYCLE_VERSION = 0; + private static final long serialVersionUID = -5151597286296228754L; private static CycleSchedule INSTANCE; + //历史数据读取时使用,表示比起当前相差多少个调度周期 + final long cycleDiff; CyclePeriod cyclePeriod; AtomicLong cycleId = new AtomicLong(0); String expression; boolean isInit; - //历史数据读取时使用,表示比起当前相差多少个调度周期 - final long cycleDiff; public CycleSchedule(String expr, Date date) throws ParseException { Date local = subMs(date); @@ -55,6 +55,21 @@ public CycleSchedule(String expr, Date date) throws ParseException { } } + public static CycleSchedule getInstance(String expr, Date date) { + if (INSTANCE == null) { + synchronized (CycleSchedule.class) { + if (INSTANCE == null) { + try { + INSTANCE = new CycleSchedule(expr, date); + } catch (ParseException e) { + e.printStackTrace(); + } + } + } + } + return INSTANCE; + } + /** * 去掉毫秒时间戳 * @@ -144,21 +159,6 @@ public long getCycleDiff() { return cycleDiff; } - public static CycleSchedule getInstance(String expr, Date date) { - if (INSTANCE == null) { - synchronized (CycleSchedule.class) { - if (INSTANCE == null) { - try { - INSTANCE = new CycleSchedule(expr, date); - } catch (ParseException e) { - e.printStackTrace(); - } - } - } - } - return INSTANCE; - } - public static class Cycle extends BasedConfigurable implements Serializable { private static final long serialVersionUID = 4842560538716388622L; @@ -169,6 +169,9 @@ public static class Cycle extends BasedConfigurable implements Serializable { String curDateStr; long cycleDiff; + public Cycle() { + } + public Integer getCycleCount() { return cycleCount; } @@ -177,9 +180,6 @@ public void setCycleCount(Integer cycleCount) { this.cycleCount = cycleCount; } - public Cycle() { - } - public Long getCycleId() { return cycleId; } diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CycleScheduleFilter.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CycleScheduleFilter.java index 507739d8..338b0010 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CycleScheduleFilter.java +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CycleScheduleFilter.java @@ -26,7 +26,7 @@ public class CycleScheduleFilter extends AbstractPatternFilter implements Serial List allPattern; - public CycleScheduleFilter(List allPattern){ + public CycleScheduleFilter(List allPattern) { this.allPattern = allPattern; } diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/DataFormatPatternFilter.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/DataFormatPatternFilter.java index 0cdc0762..7ae14aaf 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/DataFormatPatternFilter.java +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/DataFormatPatternFilter.java @@ -19,26 +19,31 @@ import java.io.Serializable; import java.text.ParseException; import java.text.SimpleDateFormat; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * @description */ public class DataFormatPatternFilter extends AbstractPatternFilter implements Serializable { - private static final long serialVersionUID = 3604787588465242642L; - - static final Log logger = LogFactory.getLog(DataFormatPatternFilter.class); - + static final Logger LOGGER = LoggerFactory.getLogger(DataFormatPatternFilter.class); static final String yyyyMMddHHmmss = "yyyyMMddHHmmss"; static final String yyyyMMdd = "yyyyMMdd"; static final String yyyyMMddHH = "yyyyMMddHH"; - + private static final long serialVersionUID = 3604787588465242642L; SimpleDateFormat format1 = new SimpleDateFormat(yyyyMMdd); SimpleDateFormat format2 = new SimpleDateFormat(yyyyMMddHH); SimpleDateFormat format3 = new SimpleDateFormat(yyyyMMddHHmmss); + public static void main(String[] args) { + DataFormatPatternFilter filter = new DataFormatPatternFilter(); +// System.out.println(filter.filter("20200101")); +// System.out.println(filter.filter("2020010101")); +// System.out.println(filter.filter("20200101010101")); + + } + @Override public boolean filter(String sourceName, String logicTableName, String tableNameSuffix) { @@ -76,7 +81,7 @@ public boolean filter(String sourceName, String logicTableName, String tableName } if (isFilter) { - logger.info(String.format("filter sourceName %s, logicTableName %s, suffix %s", sourceName, logicTableName, tableNameSuffix)); + LOGGER.info(String.format("filter sourceName %s, logicTableName %s, suffix %s", sourceName, logicTableName, tableNameSuffix)); return true; } if (next != null) { @@ -95,12 +100,4 @@ public PatternFilter getNext() { return next; } - public static void main(String[] args) { - DataFormatPatternFilter filter = new DataFormatPatternFilter(); -// System.out.println(filter.filter("20200101")); -// System.out.println(filter.filter("2020010101")); -// System.out.println(filter.filter("20200101010101")); - - } - } diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/PatternFilter.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/PatternFilter.java index 42365007..da46c26c 100644 --- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/PatternFilter.java +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/PatternFilter.java @@ -26,9 +26,9 @@ public interface PatternFilter { String yyyyMMddHH = "yyyyMMddHH"; String yyyyMMddHHmm = "yyyyMMddHHmm"; - /** * 根据sourceName和tableName判断是否符合 + * * @param sourceName * @param tableName * @return @@ -37,5 +37,4 @@ public interface PatternFilter { PatternFilter setNext(PatternFilter filter); - } diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/impl/FilesPullSource.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/impl/FilesPullSource.java new file mode 100644 index 00000000..744db07e --- /dev/null +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/impl/FilesPullSource.java @@ -0,0 +1,179 @@ +/* + * 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.connectors.source.impl; + +import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.rocketmq.streams.common.channel.impl.file.FileSource; +import org.apache.rocketmq.streams.common.channel.impl.file.FileSplit; +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.context.MessageOffset; +import org.apache.rocketmq.streams.connectors.model.PullMessage; +import org.apache.rocketmq.streams.connectors.reader.AbstractSplitReader; +import org.apache.rocketmq.streams.connectors.reader.ISplitReader; +import org.apache.rocketmq.streams.connectors.source.AbstractPullSource; + +public class FilesPullSource extends AbstractPullSource { + + protected String filePath;//一个文件一个分片 + protected String namePattern;//可以配置正则过滤需要的文件 + protected boolean isCSV = false; + + protected transient FileSource fileSource; + + @Override protected boolean initConfigurable() { + FileSource fileSource = new FileSource(); + fileSource.setFilePath(filePath); + fileSource.setNamePattern(namePattern); + fileSource.setCSV(isCSV); + fileSource.init(); + this.fileSource = fileSource; + return super.initConfigurable(); + } + + @Override protected ISplitReader createSplitReader(ISplit split) { + return new AbstractSplitReader(split) { + private FileSource.FileIterator iterator; + private AtomicInteger offsetGenerator = new AtomicInteger(0); + private transient List csvFileMsgs; + private transient int nextCurrentIndex = 0; + private transient boolean hasNext = true; + + @Override public String getCursor() { + if (isCSV) { + return this.nextCurrentIndex + ""; + } + return offsetGenerator.get() + ""; + } + + @Override public void open() { + FileSplit fileSplit = (FileSplit) split; + this.cursor = 0 + ""; + try { + iterator = new FileSource.FileIterator(fileSplit.getFile()); + } catch (Exception e) { + throw new RuntimeException("创建文件迭代器错误:" + fileSplit.getFile().getAbsolutePath(), e); + } + + } + + @Override public boolean next() { + if (isCSV) { + if (csvFileMsgs == null) { + return true; + } + if (nextCurrentIndex <= csvFileMsgs.size() - 1) { + return true; + } + return false; + } + return hasNext; + } + + @Override public Iterator> getMessage() { + List> pullMessages = new ArrayList<>(); + if (isCSV) { + if (csvFileMsgs == null) { + csvFileMsgs = fileSource.parseCSV(iterator.getFile()); + ; + } + int startIndex = nextCurrentIndex; + for (int i = startIndex; i < csvFileMsgs.size(); i++) { + PullMessage pullMessage = new PullMessage(); + JSONObject jsonObject = csvFileMsgs.get(i); + pullMessage.setMessage(jsonObject); + MessageOffset messageOffset = new MessageOffset(i + 1); + pullMessage.setMessageOffset(messageOffset); + pullMessages.add(pullMessage); + nextCurrentIndex = i + 1; + } + return pullMessages.iterator(); + } else { + while (iterator.hasNext()) { + String line = iterator.next(); + if (line == null) { + hasNext = false; + break; + } + PullMessage pullMessage = new PullMessage<>(); + JSONObject jsonObject = fileSource.create(line); + pullMessage.setMessage(jsonObject); + MessageOffset messageOffset = new MessageOffset(offsetGenerator.incrementAndGet()); + pullMessage.setMessageOffset(messageOffset); + pullMessages.add(pullMessage); + } + + return pullMessages.iterator(); + + } + } + + @Override public void seek(String cursor) { + if (isCSV) { + nextCurrentIndex = Integer.parseInt(cursor); + } else { + offsetGenerator = new AtomicInteger(Integer.parseInt(cursor)); + } + } + + @Override public long getDelay() { + return 0; + } + + @Override public long getFetchedDelay() { + return 0; + } + }; + } + + @Override public List> fetchAllSplits() { + Iterator it = fileSource.createIteratorList().iterator(); + List> splits = new ArrayList<>(); + while (it.hasNext()) { + FileSource.FileIterator fileIterator = it.next(); + splits.add(new FileSplit(fileIterator.getFile())); + } + return splits; + } + + public String getFilePath() { + return filePath; + } + + public void setFilePath(String filePath) { + this.filePath = filePath; + } + + public String getNamePattern() { + return namePattern; + } + + public void setNamePattern(String namePattern) { + this.namePattern = namePattern; + } + + public boolean isCSV() { + return isCSV; + } + + public void setCSV(boolean CSV) { + isCSV = CSV; + } +} diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/impl/NonHomologousPullSource.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/impl/NonHomologousPullSource.java new file mode 100644 index 00000000..57a1c2c3 --- /dev/null +++ b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/impl/NonHomologousPullSource.java @@ -0,0 +1,108 @@ +/* + * 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.connectors.source.impl; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; +import org.apache.rocketmq.streams.connectors.reader.ISplitReader; +import org.apache.rocketmq.streams.connectors.source.AbstractPullSource; + +/** + * 多个pull pipeline合并成一个 大的调度体 + */ +public class NonHomologousPullSource extends AbstractPullSource { + protected Map pipelines = new HashMap<>(); + + @Override protected ISplitReader createSplitReader(ISplit split) { + SplitProxy splitProxy = (SplitProxy) split; + AbstractPullSource source = (AbstractPullSource) splitProxy.pipeline.getSource(); + return source.createReader(split); + } + + @Override public synchronized List> fetchAllSplits() { + List> splits = new ArrayList<>(); + for (ChainPipeline pipeline : pipelines.values()) { + if (!(pipeline.getSource() instanceof AbstractPullSource)) { + throw new RuntimeException("MultiSplitSource can support AbstractPullSource sql only"); + } + List> pipelineSplits = pipeline.getSource().fetchAllSplits(); + for (ISplit split : pipelineSplits) { + splits.add(new SplitProxy<>(pipeline, split)); + } + + } + return splits; + } + + public synchronized void addSubPipeline(ChainPipeline chainPipeline) { + if (!(chainPipeline.getSource() instanceof AbstractPullSource)) { + throw new RuntimeException("MultiSplitSource can support AbstractPullSource sql only"); + } + this.pipelines.put(chainPipeline.getName(), chainPipeline); + chainPipeline.startPipeline(); + + } + + public synchronized void removeSubPipeline(String chainPipelineName) { + ChainPipeline chainPipeline = this.pipelines.remove(chainPipelineName); + if (chainPipeline != null) { + chainPipeline.destroy(); + } + } + + @Override public synchronized void destroySource() { + super.destroySource(); + for (ChainPipeline pipeline : pipelines.values()) { + pipeline.destroy(); + } + + } + + protected class SplitProxy implements ISplit { + protected ISplit split; + protected ChainPipeline pipeline; + + public SplitProxy(ChainPipeline pipeline, ISplit split) { + this.split = split; + this.pipeline = pipeline; + } + + @Override public String getQueueId() { + return split.getQueueId(); + } + + @Override public Q getQueue() { + return split.getQueue(); + } + + @Override public int compareTo(T o) { + return split.compareTo(o); + } + + @Override public String toJson() { + return split.toJson(); + } + + @Override public void toObject(String jsonString) { + split.toObject(jsonString); + } + } +} diff --git a/rocketmq-streams-connectors/src/test/java/org/apache/rocketmq/streams/connector/FileSourceTest.java b/rocketmq-streams-connectors/src/test/java/org/apache/rocketmq/streams/connector/FileSourceTest.java new file mode 100644 index 00000000..888bf3f8 --- /dev/null +++ b/rocketmq-streams-connectors/src/test/java/org/apache/rocketmq/streams/connector/FileSourceTest.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.connector; + +import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.rocketmq.streams.common.context.AbstractContext; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; +import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.common.utils.FileUtil; +import org.apache.rocketmq.streams.connectors.source.impl.FilesPullSource; +import org.junit.Test; + +public class FileSourceTest { + @Test + public void testPullFileSource() throws InterruptedException { + FilesPullSource filesPullSource = new FilesPullSource(); + filesPullSource.setFilePath("/Users/junjie.cheng/Desktop/TEST_DATA"); + //filesPullSource.setTest(true); + //filesPullSource.setCloseBalance(true); + filesPullSource.setName("test"); + filesPullSource.setMaxThread(3); + //filesPullSource.setPullSize(2); + filesPullSource.setPullIntervalMs(100); + filesPullSource.setCheckpointTime(1000 * 10); + filesPullSource.init(); + ConcurrentHashMap split2Count = new ConcurrentHashMap<>(); + filesPullSource.start(new IStreamOperator() { + @Override public Object doMessage(IMessage message, AbstractContext context) { + if (message.getHeader().isSystemMessage()) { + return null; + } +// try { +// Thread.sleep(10000); +// } catch (InterruptedException e) { +// e.printStackTrace(); +// } + synchronized (this) { + AtomicInteger count = split2Count.get(message.getHeader().getQueueId()); + if (count == null) { + count = new AtomicInteger(1); + split2Count.putIfAbsent(message.getHeader().getQueueId(), count); + System.out.println(DateUtil.getCurrentTime() + " " + message.getHeader().getQueueId() + ":" + count.get()); + return null; + } + + System.out.println(DateUtil.getCurrentTime() + " " + message.getHeader().getQueueId() + ":" + count.incrementAndGet()); + return null; + } +// System.out.println(DateUtil.getCurrentTime()+" "+message.getMessageBody()); +// return null; + + } + }); + + while (true) { + Thread.sleep(1000); + } + + } + + @Test + public void testWriteRows() { + List rows = new ArrayList<>(); + for (int i = 1; i < 101; i++) { + JSONObject row = new JSONObject(); + row.put("num", i); + rows.add(row.toJSONString()); + } + FileUtil.write("/tmp/chris/test.txt", rows); + } +} diff --git a/rocketmq-streams-cep/src/test/resources/log4j.xml b/rocketmq-streams-connectors/src/test/resources/log4j.xml old mode 100755 new mode 100644 similarity index 100% rename from rocketmq-streams-cep/src/test/resources/log4j.xml rename to rocketmq-streams-connectors/src/test/resources/log4j.xml diff --git a/rocketmq-streams-dbinit/pom.xml b/rocketmq-streams-db-init/pom.xml similarity index 86% rename from rocketmq-streams-dbinit/pom.xml rename to rocketmq-streams-db-init/pom.xml index 120dfc29..7d7f60ad 100644 --- a/rocketmq-streams-dbinit/pom.xml +++ b/rocketmq-streams-db-init/pom.xml @@ -15,17 +15,17 @@ See the License for the specific language governing permissions and limitations under the License. --> - rocketmq-streams org.apache.rocketmq - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT 4.0.0 - rocketmq-streams-dbinit - ROCKETMQ STREAMS :: dbinit + rocketmq-streams-db-init + ROCKETMQ STREAMS :: db-init diff --git a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBDelegate.java b/rocketmq-streams-db-init/src/main/java/org/apache/rocketmq/streams/db/init/mysql/delegate/DBDelegate.java similarity index 82% rename from rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBDelegate.java rename to rocketmq-streams-db-init/src/main/java/org/apache/rocketmq/streams/db/init/mysql/delegate/DBDelegate.java index cc8e2891..0773a9c0 100644 --- a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBDelegate.java +++ b/rocketmq-streams-db-init/src/main/java/org/apache/rocketmq/streams/db/init/mysql/delegate/DBDelegate.java @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.rocketmq.streams.dbinit.mysql.delegate; +package org.apache.rocketmq.streams.db.init.mysql.delegate; public interface DBDelegate { - public void init(String driver, String url, String userName, - String password); + void init(String driver, String url, String userName, String password); - public void init(); + void init(); } diff --git a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBDelegateFactory.java b/rocketmq-streams-db-init/src/main/java/org/apache/rocketmq/streams/db/init/mysql/delegate/DBDelegateFactory.java similarity index 83% rename from rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBDelegateFactory.java rename to rocketmq-streams-db-init/src/main/java/org/apache/rocketmq/streams/db/init/mysql/delegate/DBDelegateFactory.java index 4a7974b9..7c9ce194 100644 --- a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBDelegateFactory.java +++ b/rocketmq-streams-db-init/src/main/java/org/apache/rocketmq/streams/db/init/mysql/delegate/DBDelegateFactory.java @@ -15,15 +15,12 @@ * limitations under the License. */ -package org.apache.rocketmq.streams.dbinit.mysql.delegate; - -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +package org.apache.rocketmq.streams.db.init.mysql.delegate; public class DBDelegateFactory { public static DBDelegate getDelegate() { - String dbType = ComponentCreator.getProperties().getProperty(ConfigureFileKey.DB_TYPE); + String dbType = null; if (dbType == null || "".equalsIgnoreCase(dbType)) { dbType = DBType.DB_MYSQL; } diff --git a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBType.java b/rocketmq-streams-db-init/src/main/java/org/apache/rocketmq/streams/db/init/mysql/delegate/DBType.java similarity index 93% rename from rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBType.java rename to rocketmq-streams-db-init/src/main/java/org/apache/rocketmq/streams/db/init/mysql/delegate/DBType.java index 83e79788..0a580473 100644 --- a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBType.java +++ b/rocketmq-streams-db-init/src/main/java/org/apache/rocketmq/streams/db/init/mysql/delegate/DBType.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.rocketmq.streams.dbinit.mysql.delegate; +package org.apache.rocketmq.streams.db.init.mysql.delegate; public class DBType { diff --git a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/MysqlDelegate.java b/rocketmq-streams-db-init/src/main/java/org/apache/rocketmq/streams/db/init/mysql/delegate/MysqlDelegate.java similarity index 83% rename from rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/MysqlDelegate.java rename to rocketmq-streams-db-init/src/main/java/org/apache/rocketmq/streams/db/init/mysql/delegate/MysqlDelegate.java index 4c0beeca..c2a71ab5 100644 --- a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/MysqlDelegate.java +++ b/rocketmq-streams-db-init/src/main/java/org/apache/rocketmq/streams/db/init/mysql/delegate/MysqlDelegate.java @@ -15,23 +15,22 @@ * limitations under the License. */ -package org.apache.rocketmq.streams.dbinit.mysql.delegate; +package org.apache.rocketmq.streams.db.init.mysql.delegate; import java.io.IOException; import java.net.URL; -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.db.driver.orm.ORMUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class MysqlDelegate implements DBDelegate { - public static final Log LOG = LogFactory.getLog(MysqlDelegate.class); - + public static final Logger LOGGER = LoggerFactory.getLogger(MysqlDelegate.class); @Override public void init(String driver, final String url, final String userName, - final String password) { + final String password) { String[] sqls = loadSqls(); for (String sql : sqls) { ORMUtil.executeSQL(sql, null, driver, url, userName, password); @@ -52,8 +51,8 @@ private String[] loadSqls() { try { String tables = FileUtil.loadFileContent(url.openStream()); sqls = tables.split(";"); - if (LOG.isDebugEnabled()) { - LOG.debug("Init db sqls : " + tables); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Init db sqls : " + tables); } } catch (IOException e) { e.printStackTrace(); @@ -61,5 +60,4 @@ private String[] loadSqls() { return sqls; } - } diff --git a/rocketmq-streams-db-init/src/main/resources/tables_mysql_innodb.sql b/rocketmq-streams-db-init/src/main/resources/tables_mysql_innodb.sql new file mode 100644 index 00000000..1d57fd1c --- /dev/null +++ b/rocketmq-streams-db-init/src/main/resources/tables_mysql_innodb.sql @@ -0,0 +1,227 @@ +CREATE TABLE IF NOT EXISTS `window_max_value` +( + `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT, + `gmt_create` datetime NOT NULL, + `gmt_modified` datetime NOT NULL, + `max_offset` varchar(256) NOT NULL, + `is_max_offset_long` int(1) DEFAULT NULL, + `max_value` bigint(20) unsigned NOT NULL, + `max_event_time` bigint(20) unsigned NOT NULL, + `msg_key` varchar(256) NOT NULL, + PRIMARY KEY(`id`), + UNIQUE KEY `uk__ket`(`msg_key`(250)), + KEY `idx_modifytime`(`gmt_modified`) + ) ENGINE=InnoDB DEFAULT CHARSET=utf8; + +CREATE TABLE IF NOT EXISTS `window_value` +( + `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT, + `gmt_create` datetime NOT NULL, + `gmt_modified` datetime NOT NULL, + `start_time` varchar(20) NOT NULL, + `end_time` varchar(20) NOT NULL, + `max_offset` longtext, + `group_by` text, + `agg_column_result` longtext, + `computed_column_result` longtext, + `version` varchar(64) DEFAULT NULL, + `name_space` varchar(256) DEFAULT NULL, + `configure_name` varchar(256) DEFAULT NULL, + `msg_key` varchar(64) NOT NULL, + `window_instance_id` varchar(64) NOT NULL, + `partition` varchar(512) DEFAULT NULL, + `partition_num` bigint(20) DEFAULT NULL, + `fire_time` varchar(20) DEFAULT NULL, + `update_version` bigint(20) unsigned DEFAULT NULL, + `update_flag` bigint(20) DEFAULT NULL, + `window_instance_partition_id` varchar(64) DEFAULT NULL, + `type` varchar(64) DEFAULT NULL, + `group_by_field_name` varchar(128) DEFAULT NULL COMMENT '分组字段', + PRIMARY KEY(`id`), + UNIQUE KEY `uk_window_state`(`msg_key`), + KEY `idx_window_instance_shuffle`(`window_instance_partition_id`,`partition_num`), + KEY `idx_window_instance_firetime`(`window_instance_partition_id`,`fire_time`), + KEY `idx_window`(`name_space`(128),`configure_name`(128),`partition`(128)) + ) ENGINE=InnoDB AUTO_INCREMENT=5 DEFAULT CHARSET=utf8 +; + + +CREATE TABLE IF NOT EXISTS `window_task` +( + `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT, + `gmt_create` datetime NOT NULL DEFAULT CURRENT_TIMESTAMP, + `gmt_modified` datetime NOT NULL DEFAULT CURRENT_TIMESTAMP, + `task_id` varchar(64) NOT NULL, + `untreated_flag` int(11) NOT NULL DEFAULT '0', + `group_by_value` varchar(1024) NOT NULL, + `task_owner` varchar(256) NOT NULL, + `task_send_time` datetime DEFAULT NULL, + `send_task_msg` longtext NOT NULL, + `msg_send_time` bigint(20) DEFAULT NULL, + `name` varchar(128) NOT NULL, + `start_time` varchar(20) NOT NULL, + `end_time` varchar(20) NOT NULL, + PRIMARY KEY(`id`), + UNIQUE KEY `uk_taskid`(`task_id`), + KEY `idx_flag_modifytime`(`name`,`untreated_flag`,`gmt_modified`) + ) ENGINE=InnoDB DEFAULT CHARSET=utf8; + +CREATE TABLE IF NOT EXISTS `window_instance` +( + `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT, + `gmt_create` datetime NOT NULL, + `gmt_modified` datetime NOT NULL, + `start_time` varchar(20) NOT NULL, + `end_time` varchar(20) NOT NULL, + `fire_time` varchar(20) NOT NULL, + `window_name` varchar(128) NOT NULL, + `window_name_space` varchar(128) NOT NULL, + `status` tinyint(4) NOT NULL DEFAULT '0', + `version` int(11) DEFAULT '0', + `window_instance_key` varchar(128) DEFAULT NULL, + `window_instance_name` varchar(128) DEFAULT NULL, + `window_Instance_split_name` varchar(128) DEFAULT NULL, + `split_id` varchar(128) DEFAULT NULL, + `can_clear_resource` tinyint(4) DEFAULT NULL COMMENT '是否清理资源', + PRIMARY KEY(`id`), + UNIQUE KEY `uk_window_instance_uniq_index`(`window_instance_key`), + KEY `idx_gmt_modified`(`fire_time`,`window_name`,`window_name_space`,`status`), + KEY `idx_windowinstance_name`(`window_instance_name`), + KEY `idx_windowinstance_split_name`(`window_Instance_split_name`), + KEY `idx_windowinstance_split_name_firetime`(`window_Instance_split_name`,`fire_time`) + ) ENGINE=InnoDB AUTO_INCREMENT=5 DEFAULT CHARSET=utf8 +; + + +CREATE TABLE IF NOT EXISTS `lease_info` +( + `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT, + `gmt_create` datetime NOT NULL, + `gmt_modified` datetime NOT NULL, + `lease_name` varchar(255) NOT NULL, + `lease_user_ip` varchar(255) NOT NULL, + `lease_end_time` varchar(255) NOT NULL, + `status` int(11) NOT NULL DEFAULT '1', + `version` bigint(20) NOT NULL, + `candidate_lease_ip` varchar(255) DEFAULT NULL, + PRIMARY KEY(`id`), + UNIQUE KEY `uk_name`(`lease_name`) + ) ENGINE=InnoDB DEFAULT CHARSET=utf8; + +CREATE TABLE IF NOT EXISTS `dipper_sql_configure` +( + `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT, + `gmt_create` datetime NOT NULL, + `gmt_modified` datetime NOT NULL, + `namespace` varchar(32) NOT NULL, + `type` varchar(32) NOT NULL, + `name` varchar(128) NOT NULL, + `json_value` longtext NOT NULL, + `request_id` varchar(128) NOT NULL, + `account_id` varchar(32) NOT NULL, + `account_name` varchar(32) NOT NULL, + `account_nickname` varchar(32) NOT NULL, + `client_ip` varchar(64) NOT NULL, + `status` tinyint(3) unsigned NOT NULL DEFAULT '0', + `is_publish` int(11) NOT NULL DEFAULT '0', + PRIMARY KEY(`id`), + UNIQUE KEY `uk_namespace_type_name`(`namespace`,`type`,`name`), + KEY `idx_namespace`(`namespace`) + ) ENGINE=InnoDB DEFAULT CHARSET=utf8; + +CREATE TABLE IF NOT EXISTS `dipper_configure` +( + `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT, + `gmt_create` datetime NOT NULL, + `gmt_modified` datetime NOT NULL, + `namespace` varchar(32) NOT NULL, + `type` varchar(32) NOT NULL, + `name` varchar(128) NOT NULL, + `json_value` longtext NOT NULL, + `request_id` varchar(128) DEFAULT NULL, + `account_id` varchar(32) DEFAULT NULL, + `account_name` varchar(32) DEFAULT NULL, + `account_nickname` varchar(32) DEFAULT NULL, + `client_ip` varchar(64) DEFAULT NULL, + `status` tinyint(3) unsigned NOT NULL DEFAULT '0', + `isPublish` int(1) NOT NULL DEFAULT '0', + PRIMARY KEY(`id`), + UNIQUE KEY `uk_namespace_type_name`(`namespace`,`type`, `name`), + KEY `idx_namespace`(`namespace`) + ) ENGINE=InnoDB DEFAULT CHARSET=utf8; + +CREATE TABLE IF NOT EXISTS `join_right_state` +( + `id` bigint(20) NOT NULL AUTO_INCREMENT, + `gmt_create` datetime DEFAULT NULL, + `gmt_modified` datetime DEFAULT NULL, + `window_id` bigint(20) DEFAULT NULL, + `window_name` varchar(200) DEFAULT NULL, + `window_name_space` varchar(45) DEFAULT NULL, + `message_id` varchar(200) DEFAULT NULL, + `message_key` varchar(32) DEFAULT NULL, + `message_time` datetime DEFAULT NULL, + `message_body` longtext, + `msg_key` varchar(400) DEFAULT NULL, + `window_instance_id` varchar(200) DEFAULT NULL, + `partition` varchar(200) DEFAULT NULL, + `partition_num` bigint(20) DEFAULT NULL, + `window_instance_partition_id` varchar(200) DEFAULT NULL, + `version` varchar(64) DEFAULT NULL, + `update_flag` bigint(20) DEFAULT NULL, + `name_space` varchar(256) DEFAULT NULL, + `configure_name` varchar(256) DEFAULT NULL, + `type` varchar(64) DEFAULT NULL, + `name` varchar(64) DEFAULT NULL, + `update_version` bigint(20) unsigned DEFAULT NULL, + PRIMARY KEY(`id`), + UNIQUE KEY `uk_message_id_unique`(`message_id`), + KEY `idx_message_key_index`(`message_key`), + KEY `idx_gmt_create_index`(`gmt_create`), + KEY `idx_window_name_index`(`window_name`(70)), + KEY `idx_message_key_gmt_create_index`(`message_key`,`gmt_create`) + ) ENGINE=InnoDB DEFAULT CHARSET=utf8; + +CREATE TABLE IF NOT EXISTS `join_left_state` +( + `id` bigint(20) NOT NULL AUTO_INCREMENT, + `gmt_create` datetime DEFAULT NULL, + `gmt_modified` datetime DEFAULT NULL, + `window_id` bigint(20) DEFAULT NULL, + `window_name` varchar(200) DEFAULT NULL, + `window_name_space` varchar(45) DEFAULT NULL, + `message_id` varchar(200) DEFAULT NULL, + `message_key` varchar(32) DEFAULT NULL, + `message_time` datetime DEFAULT NULL, + `message_body` longtext, + `msg_key` varchar(400) DEFAULT NULL, + `window_instance_id` varchar(20) DEFAULT NULL, + `partition` varchar(200) DEFAULT NULL, + `partition_num` bigint(20) DEFAULT NULL, + `window_instance_partition_id` varchar(200) DEFAULT NULL, + `version` varchar(64) DEFAULT NULL, + `update_flag` bigint(20) DEFAULT NULL, + `name_space` varchar(256) DEFAULT NULL, + `configure_name` varchar(256) DEFAULT NULL, + `type` varchar(64) DEFAULT NULL, + `name` varchar(64) DEFAULT NULL, + `update_version` bigint(20) unsigned DEFAULT NULL, + PRIMARY KEY(`id`), + UNIQUE KEY `uk_message_id_unique`(`message_id`), + KEY `idx_message_key_index`(`message_key`), + KEY `idx_gmt_create_index`(`gmt_create`), + KEY `idx_window_name_index`(`window_name`(70)), + KEY `idx_message_key_gmt_create_index`(`message_key`, `gmt_create`) + ) ENGINE=InnoDB DEFAULT CHARSET=utf8; + +CREATE TABLE IF NOT EXISTS `dipper_dispatcher_kv_config` +( + `namespace` varchar(255) NOT NULL, + `config_key` varchar(255) NOT NULL, + `config_value` longtext NOT NULL, + `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT '创建时间', + `update_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP COMMENT '更新时间', + UNIQUE KEY `uk_message_id_unique`(`namespace`,`config_key`), + KEY `idx_namespace_index`(`namespace`), + KEY `idx_key_index`(`config_key`) + ) ENGINE=InnoDB DEFAULT CHARSET=utf8; \ No newline at end of file diff --git a/rocketmq-streams-db-operator/pom.xml b/rocketmq-streams-db-operator/pom.xml index 9ce26f7f..b46301a8 100755 --- a/rocketmq-streams-db-operator/pom.xml +++ b/rocketmq-streams-db-operator/pom.xml @@ -15,14 +15,14 @@ See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 org.apache.rocketmq rocketmq-streams - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT rocketmq-streams-db-operator ROCKETMQ STREAMS :: db-operator @@ -31,7 +31,7 @@ org.apache.rocketmq - rocketmq-streams-configurable + rocketmq-streams-commons diff --git a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/DBConfigureService.java b/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/DBConfigureService.java deleted file mode 100644 index 234b4ab7..00000000 --- a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/DBConfigureService.java +++ /dev/null @@ -1,275 +0,0 @@ -/* - * 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.db.configuable; - -import com.alibaba.fastjson.JSONObject; -import java.util.ArrayList; -import java.util.Date; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.component.AbstractComponent; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configurable.AbstractConfigurable; -import org.apache.rocketmq.streams.common.configurable.IConfigurable; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; -import org.apache.rocketmq.streams.common.interfaces.IPropertyEnable; -import org.apache.rocketmq.streams.common.utils.AESUtil; -import org.apache.rocketmq.streams.common.utils.MapKeyUtil; -import org.apache.rocketmq.streams.common.utils.SQLUtil; -import org.apache.rocketmq.streams.common.utils.StringUtil; -import org.apache.rocketmq.streams.configurable.model.Configure; -import org.apache.rocketmq.streams.configurable.service.AbstractConfigurableService; -import org.apache.rocketmq.streams.db.driver.DriverBuilder; -import org.apache.rocketmq.streams.db.driver.JDBCDriver; - -/** - * Configuable对象存储在db中,是生成环境常用的一种模式 数据库参数可以配置在配置文件中,ConfiguableComponent在启动时,会把参数封装在Properties中,调用DBConfigureService(Properties properties) 构造方法完成实例创建 - */ - -public class DBConfigureService extends AbstractConfigurableService implements IPropertyEnable { - - private static final Log LOG = LogFactory.getLog(DBConfigureService.class); - private String jdbcdriver; - private String url; - private String userName; - private String password; - private String tableName = "dipper_configure"; - @Deprecated - private boolean isCompatibilityOldRuleEngine = false;//兼容老规则引擎使用,正常场景不需要理会 - - public DBConfigureService(String jdbcdriver, String url, String userName, String password) { - this(jdbcdriver, url, userName, password, null); - } - - public DBConfigureService(String jdbcdriver, String url, String userName, String password, String tableName) { - this.url = url; - this.jdbcdriver = jdbcdriver; - this.userName = userName; - this.password = password; - this.tableName = tableName; - LOG.info("DBConfigureService resource ,the info is: driver:" + this.jdbcdriver + ",url:" + this.url - + ",username:" + userName + ",password:" + password); - regJdbcDriver(jdbcdriver); - } - - public DBConfigureService() { - } - - /** - * @param properties - */ - public DBConfigureService(Properties properties) { - super(properties); - initProperty(properties); - } - - @Override - protected GetConfigureResult loadConfigurable(String namespace) { - GetConfigureResult result = new GetConfigureResult(); - try { - List configures = selectOpening(namespace); - List configurables = convert(configures); - result.setConfigurables(configurables); - result.setQuerySuccess(true);// 该字段标示查询是否成功,若不成功则不会更新配置 - } catch (Exception e) { - result.setQuerySuccess(false); - LOG.error("load configurable error ", e); - } - return result; - } - - protected List selectOpening(String namespace) { - return queryConfigureByNamespace(namespace); - } - - protected List queryConfigureByNamespace(String... namespaces) { - return queryConfigureByNamespaceInner(null, namespaces); - } - - protected List queryConfigureByNamespaceInner(String type, String... namespaces) { - JDBCDriver resource = createResouce(); - try { - String namespace = "namespace"; - if (isCompatibilityOldRuleEngine && AbstractComponent.JDBC_COMPATIBILITY_RULEENGINE_TABLE_NAME.equals(tableName)) { - namespace = "name_space"; - } - String sql = "SELECT * FROM `" + tableName + "` WHERE " + namespace + " in (" + SQLUtil.createInSql(namespaces) + ") and status =1"; - if (StringUtil.isNotEmpty(type)) { - sql = sql + " and type='" + type + "'"; - } - JSONObject jsonObject = new JSONObject(); - jsonObject.put("namespace", MapKeyUtil.createKeyBySign(",", namespaces)); - sql = SQLUtil.parseIbatisSQL(jsonObject, sql); - // String builder = "SELECT * FROM `" + tableName + "` WHERE namespace ='" + namespace + "' and status =1"; - List> result = resource.queryForList(sql); - if (result == null) { - return new ArrayList(); - } - // LOG.info("load configurable's count is " + result.size()); - return convert2Configure(result); - } finally { - if (resource != null) { - resource.destroy(); - } - } - } - - @Override - public List queryConfiguableByNamespace(String... namespaces) { - List configures = queryConfigureByNamespace(namespaces); - return convert(configures); - } - - protected void saveOrUpdate(IConfigurable configure) { - JDBCDriver jdbcDataSource = createResouce(); - String sql = AbstractConfigurable.createSQL(configure, this.tableName); - try { - jdbcDataSource.executeInsert(sql); - } catch (Exception e) { - LOG.error("DBConfigureService saveOrUpdate error,sqlnode:" + sql); - throw new RuntimeException(e); - } finally { - if (jdbcDataSource != null) { - jdbcDataSource.destroy(); - } - } - } - - protected List convert2Configure(List> rows) { - List configures = new ArrayList(); - for (Map row : rows) { - Configure configure = new Configure(); - Long id = getColumnValue(row, "id"); - configure.setId(id); - Date create = getColumnValue(row, "gmt_create"); - configure.setGmtCreate(create); - Date modify = getColumnValue(row, "gmt_modified"); - configure.setGmtModified(modify); - String namespace = getColumnValue(row, "namespace"); - if (StringUtil.isEmpty(namespace)) { - namespace = getColumnValue(row, "name_space"); - } - configure.setNameSpace(namespace); - String type = getColumnValue(row, "type"); - configure.setType(type); - String name = getColumnValue(row, "name"); - configure.setName(name); - String jsonValue = getColumnValue(row, "json_value"); - try { - jsonValue = AESUtil.aesDecrypt(jsonValue, ComponentCreator.getProperties().getProperty(ConfigureFileKey.SECRECY, ConfigureFileKey.SECRECY_DEFAULT)); - } catch (Exception e) { - LOG.error("failed in decrypting the value, reason:\t" + e.getCause()); - throw new RuntimeException(e); - } - configure.setJsonValue(jsonValue); - configures.add(configure); - } - return configures; - } - - @SuppressWarnings("unchecked") - protected T getColumnValue(Map row, String columnName) { - Object value = row.get(columnName); - if (value == null) { - return null; - } - if (java.math.BigInteger.class.isInstance(value)) { - return (T) Long.valueOf(value.toString()); - } - return (T) value; - - } - - protected JDBCDriver createResouce() { - JDBCDriver resource = DriverBuilder.createDriver(this.jdbcdriver, this.url, this.userName, this.password); - return resource; - } - - public void setJdbcdriver(String jdbcdriver) { - this.jdbcdriver = jdbcdriver; - } - - public void setUrl(String url) { - this.url = url; - } - - public void setUserName(String userName) { - this.userName = userName; - } - - public void setPassword(String password) { - this.password = password; - } - - private void regJdbcDriver(String jdbcdriver) { - try { - if (StringUtil.isEmpty(jdbcdriver)) { - jdbcdriver = AbstractComponent.DEFAULT_JDBC_DRIVER; - } - Class.forName(jdbcdriver); - } catch (ClassNotFoundException e) { - LOG.error("DBConfigureService regJdbcDriver ClassNotFoundException error", e); - } catch (Exception e) { - LOG.error("DBConfigureService regJdbcDriver error", e); - } - } - - @Override - public void initProperty(Properties properties) { - this.jdbcdriver = properties.getProperty(AbstractComponent.JDBC_DRIVER); - regJdbcDriver(jdbcdriver); - this.url = properties.getProperty(AbstractComponent.JDBC_URL); - this.userName = properties.getProperty(AbstractComponent.JDBC_USERNAME); - this.password = properties.getProperty(AbstractComponent.JDBC_PASSWORD); - String tableName = properties.getProperty(AbstractComponent.JDBC_TABLE_NAME); - String isCompatibilityOldRuleEngine = properties.getProperty(AbstractComponent.JDBC_COMPATIBILITY_OLD_RULEENGINE); - if (StringUtil.isNotEmpty(isCompatibilityOldRuleEngine)) { - this.isCompatibilityOldRuleEngine = true; - } - if (StringUtil.isNotEmpty(tableName)) { - this.tableName = tableName; - } - LOG.info( - "Properties resource ,the info is: driver:" + this.jdbcdriver + ",url:" + this.url + ",username:" + userName - + ",password:" + password); - } - - @Override - protected void insertConfigurable(IConfigurable configurable) { - saveOrUpdate(configurable); - } - - @Override - protected void updateConfigurable(IConfigurable configurable) { - saveOrUpdate(configurable); - } - - @Override - public List loadConfigurableFromStorage(String type) { - - List configures = queryConfigureByNamespaceInner(type, namespace); - List configurables = convert(configures); - List result = new ArrayList<>(); - for (IConfigurable configurable : configurables) { - result.add((T) configurable); - } - return result; - } -} diff --git a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/DBSupportParentConfigureService.java b/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/DBSupportParentConfigureService.java deleted file mode 100644 index 7582a891..00000000 --- a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/DBSupportParentConfigureService.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.db.configuable; - -import com.google.auto.service.AutoService; -import java.util.Properties; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.model.ServiceName; -import org.apache.rocketmq.streams.configurable.service.AbstractSupportParentConfigureService; -import org.apache.rocketmq.streams.configurable.service.ConfigurableServcieType; - -@AutoService(IConfigurableService.class) -@ServiceName(ConfigurableServcieType.DEFAULT_SERVICE_NAME) -public class DBSupportParentConfigureService extends AbstractSupportParentConfigureService { - - @Override - protected void initBeforeInitConfigurable(Properties property) { - this.parentConfigureService = new DBConfigureService(property); - this.configureService = new DBConfigureService(property); - - } -} diff --git a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/DriverBuilder.java b/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/DriverBuilder.java index a7d2e866..3159deff 100644 --- a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/DriverBuilder.java +++ b/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/DriverBuilder.java @@ -20,22 +20,20 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.component.AbstractComponent; import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.configuration.SystemContext; import org.apache.rocketmq.streams.common.utils.ReflectUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * 创建JDBCDriver,如果没有 */ public class DriverBuilder { - private static final Log LOG = LogFactory.getLog(DriverBuilder.class); - - public static final String DEFALUT_JDBC_DRIVER = "com.mysql.jdbc.Driver"; - + private static final Logger LOGGER = LoggerFactory.getLogger(DriverBuilder.class); private static final Map dataSourceMap = new ConcurrentHashMap<>(); private static AtomicInteger count = new AtomicInteger(0); @@ -46,10 +44,10 @@ public class DriverBuilder { * @return */ public static JDBCDriver createDriver() { - String driver = ComponentCreator.getProperties().getProperty(AbstractComponent.JDBC_DRIVER); - String url = ComponentCreator.getProperties().getProperty(AbstractComponent.JDBC_URL); - String userName = ComponentCreator.getProperties().getProperty(AbstractComponent.JDBC_USERNAME); - String password = ComponentCreator.getProperties().getProperty(AbstractComponent.JDBC_PASSWORD); + String driver = SystemContext.getProperty(ConfigurationKey.JDBC_DRIVER); + String url = SystemContext.getProperty(ConfigurationKey.JDBC_URL); + String userName = SystemContext.getProperty(ConfigurationKey.JDBC_USERNAME); + String password = SystemContext.getProperty(ConfigurationKey.JDBC_PASSWORD); return createDriver(driver, url, userName, password); } @@ -62,19 +60,16 @@ public static JDBCDriver createDriver() { * @param password 密码 * @return JDBCDriver */ - public static JDBCDriver createDriver(String driver, final String url, final String userName, - final String password) { + public static JDBCDriver createDriver(String driver, final String url, final String userName, final String password) { if (StringUtil.isEmpty(driver)) { - driver = DEFALUT_JDBC_DRIVER; + driver = ConfigurationKey.DEFAULT_JDBC_DRIVER; } String className = ComponentCreator.getDBProxyClassName(); if (StringUtil.isNotEmpty(className)) { Class clazz = ReflectUtil.forClass(className); try { - Constructor constructor = clazz.getConstructor( - new Class[] {String.class, String.class, String.class, String.class}); - JDBCDriver abstractDBDataSource = (JDBCDriver)constructor.newInstance(url, userName, password, - driver); + Constructor constructor = clazz.getConstructor(new Class[] {String.class, String.class, String.class, String.class}); + JDBCDriver abstractDBDataSource = (JDBCDriver) constructor.newInstance(url, userName, password, driver); abstractDBDataSource.init(); return abstractDBDataSource; } catch (Exception e) { @@ -85,7 +80,7 @@ public static JDBCDriver createDriver(String driver, final String url, final Str final String jdbcdriver = driver; ReflectUtil.forClass(jdbcdriver); JDBCDriver resource = new JDBCDriver(); - LOG.debug("jdbcdriver=" + jdbcdriver + ",url=" + url); + LOGGER.debug("jdbcdriver=" + jdbcdriver + ",url=" + url); resource.setJdbcDriver(jdbcdriver); resource.setUrl(url); resource.setUserName(userName); @@ -102,7 +97,7 @@ public static JDBCDriver createDriver(String driver, final String url, final Str * @param password * @return */ - private static String genereateKey(String url, String userName, String password) { + private static String generateKey(String url, String userName, String password) { return url + "_" + userName + "_" + password; } diff --git a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/JDBCDriver.java b/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/JDBCDriver.java index 17624788..6d3a9155 100644 --- a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/JDBCDriver.java +++ b/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/JDBCDriver.java @@ -21,12 +21,12 @@ import java.sql.Statement; import java.util.ArrayList; import java.util.Collection; -import java.util.Iterator; import java.util.List; import java.util.Map; import org.apache.rocketmq.streams.common.channel.sink.ISink; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; import org.apache.rocketmq.streams.common.dboperator.IDBDriver; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.springframework.dao.DataAccessException; @@ -45,19 +45,14 @@ *

    */ public class JDBCDriver extends BasedConfigurable implements IDriverBudiler, IDBDriver { - private String jdbcDriver = DriverBuilder.DEFALUT_JDBC_DRIVER; - @ENVDependence - protected String url; - @ENVDependence - protected String userName; - @ENVDependence - protected String password; - + @ENVDependence protected String url; + @ENVDependence protected String userName; + @ENVDependence protected String password; protected transient javax.sql.DataSource dataSource; - private transient IDBDriver dbDriver = null; + private String jdbcDriver = ConfigurationKey.DEFAULT_JDBC_DRIVER; + private transient volatile IDBDriver dbDriver = null; - public JDBCDriver(String url, String userName, String password, - String driver) { + public JDBCDriver(String url, String userName, String password, String driver) { setType(ISink.TYPE); this.url = url; this.userName = userName; @@ -84,76 +79,41 @@ protected IDBDriver createOrGetDriver() { } return dbDriver; } - static class BatchUpdateStatementCallback implements StatementCallback, SqlProvider { - private String currSql; - private String[] sql; - public BatchUpdateStatementCallback(String... sqls){ - this.sql=sqls; - } - @Override - public int[] doInStatement(Statement stmt) throws SQLException, DataAccessException { - int[] rowsAffected = new int[sql.length]; - if (JdbcUtils.supportsBatchUpdates(stmt.getConnection())) { - // stmt.getConnection().setAutoCommit(false); - for (String sqlStmt : sql) { - this.currSql = sqlStmt; - stmt.addBatch(sqlStmt); - } - rowsAffected = stmt.executeBatch(); - // stmt.getConnection().commit(); - } - else { - for (int i = 0; i < sql.length; i++) { - this.currSql = sql[i]; - if (!stmt.execute(sql[i])) { - rowsAffected[i] = stmt.getUpdateCount(); - } - else { - throw new InvalidDataAccessApiUsageException("Invalid batch SQL statement: " + sql[i]); - } - } - } - return rowsAffected; - } - @Override - public String getSql() { - return this.currSql; - } - } - @Override - public IDBDriver createDBDriver() { + @Override public IDBDriver createDBDriver() { javax.sql.DataSource dataSource = createDBDataSource(); return new IDBDriver() { private final JdbcTemplate jdbcTemplate = new JdbcTemplate(dataSource); - - @Override - public int update(String sql) { + @Override public int update(String sql) { return jdbcTemplate.update(sql); } - @Override - public void execute(String sql) { + @Override public void execute(String sql) { jdbcTemplate.execute(sql); } - @Override - public List> queryForList(String sql) { + @Override public int execute(String sql, Object[] params) { + return jdbcTemplate.update(sql, params); + } + + @Override public List> executeQuery(String sql, Object[] params) { + return jdbcTemplate.queryForList(sql, params); + } + + @Override public List> queryForList(String sql) { return jdbcTemplate.queryForList(sql); } - @Override - public Map queryOneRow(String sql) { + @Override public Map queryOneRow(String sql) { return jdbcTemplate.queryForMap(sql); } - @Override - public long executeInsert(String sql) { + @Override public long executeInsert(String sql) { try { KeyHolder keyHolder = new GeneratedKeyHolder(); jdbcTemplate.update(con -> con.prepareStatement(sql, PreparedStatement.RETURN_GENERATED_KEYS), keyHolder); - if (keyHolder.getKeyList() == null || keyHolder.getKeyList().size() > 1 || keyHolder.getKey() == null) { + if (keyHolder.getKeyList().size() > 1 || keyHolder.getKey() == null) { return 0; } return keyHolder.getKey().longValue(); @@ -163,26 +123,22 @@ public long executeInsert(String sql) { } } - @Override - public void executSqls(String... sqls) { + @Override public void executeSqls(String... sqls) { Assert.notEmpty(sqls, "SQL array must not be empty"); jdbcTemplate.execute(new BatchUpdateStatementCallback(sqls)); } - @Override - public void executSqls(Collection sqlCollection) { + @Override public void executeSqls(Collection sqlCollection) { if (sqlCollection == null || sqlCollection.size() == 0) { return; } String[] sqls = new String[sqlCollection.size()]; int i = 0; - Iterator it = sqlCollection.iterator(); - while (it.hasNext()) { - String sql = it.next(); + for (String sql : sqlCollection) { sqls[i] = sql; i++; } - executSqls(sqls); + executeSqls(sqls); } /** @@ -190,8 +146,7 @@ public void executSqls(Collection sqlCollection) { * @param sql 可执行的SQL * @return 结果数据 */ - @Override - public List> batchQueryBySql(String sql, int batchSize) { + @Override public List> batchQueryBySql(String sql, int batchSize) { List> rows = new ArrayList<>(); int startBatch; String baseSql = sql; @@ -216,17 +171,14 @@ public List> batchQueryBySql(String sql, int batchSize) { } protected javax.sql.DataSource createDBDataSource() { - SingleConnectionDataSource dataSource = new SingleConnectionDataSource(url, userName, password, true); - dataSource.setDriverClassName(jdbcDriver); dataSource.setSuppressClose(true); this.dataSource = dataSource; return dataSource; } - @Override - public boolean isValidate() { + @Override public boolean isValidate() { try { if (dataSource == null) { dataSource = createDBDataSource(); @@ -238,10 +190,9 @@ public boolean isValidate() { return true; } - @Override - public void destroy() { + @Override public void destroy() { if (dataSource instanceof SingleConnectionDataSource) { - SingleConnectionDataSource data = (SingleConnectionDataSource)dataSource; + SingleConnectionDataSource data = (SingleConnectionDataSource) dataSource; data.destroy(); } } @@ -278,43 +229,80 @@ public void setPassword(String password) { this.password = password; } - @Override - public int update(String sql) { + @Override public int update(String sql) { return createOrGetDriver().update(sql); } - @Override - public void execute(String sql) { + @Override public void execute(String sql) { createOrGetDriver().execute(sql); } - @Override - public List> queryForList(String sql) { + @Override public int execute(String sql, Object[] params) { + return createOrGetDriver().execute(sql, params); + } + + @Override public List> executeQuery(String sql, Object[] params) { + return createOrGetDriver().executeQuery(sql, params); + } + + @Override public List> queryForList(String sql) { return createOrGetDriver().queryForList(sql); } - @Override - public Map queryOneRow(String sql) { + @Override public Map queryOneRow(String sql) { return createOrGetDriver().queryOneRow(sql); } - @Override - public long executeInsert(String sql) { + @Override public long executeInsert(String sql) { return createOrGetDriver().executeInsert(sql); } - @Override - public void executSqls(String... sqls) { - createOrGetDriver().executSqls(sqls); + @Override public void executeSqls(String... sqls) { + createOrGetDriver().executeSqls(sqls); } - @Override - public void executSqls(Collection sqls) { - createOrGetDriver().executSqls(sqls); + @Override public void executeSqls(Collection sqls) { + createOrGetDriver().executeSqls(sqls); } - @Override - public List> batchQueryBySql(String sql, int batchSize) { + @Override public List> batchQueryBySql(String sql, int batchSize) { return createOrGetDriver().batchQueryBySql(sql, batchSize); } + + static class BatchUpdateStatementCallback implements StatementCallback, SqlProvider { + private String currSql; + private String[] sql; + + public BatchUpdateStatementCallback(String... sqls) { + this.sql = sqls; + } + + @Override public int[] doInStatement(Statement stmt) throws SQLException, DataAccessException { + int[] rowsAffected = new int[sql.length]; + if (JdbcUtils.supportsBatchUpdates(stmt.getConnection())) { + // stmt.getConnection().setAutoCommit(false); + for (String sqlStmt : sql) { + this.currSql = sqlStmt; + stmt.addBatch(sqlStmt); + } + + rowsAffected = stmt.executeBatch(); + // stmt.getConnection().commit(); + } else { + for (int i = 0; i < sql.length; i++) { + this.currSql = sql[i]; + if (!stmt.execute(sql[i])) { + rowsAffected[i] = stmt.getUpdateCount(); + } else { + throw new InvalidDataAccessApiUsageException("Invalid batch SQL statement: " + sql[i]); + } + } + } + return rowsAffected; + } + + @Override public String getSql() { + return this.currSql; + } + } } diff --git a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/batchloader/BatchRowLoader.java b/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/batchloader/BatchRowLoader.java index 53c192ba..0f454a3d 100644 --- a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/batchloader/BatchRowLoader.java +++ b/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/batchloader/BatchRowLoader.java @@ -21,91 +21,112 @@ import java.util.Map; 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 org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.cache.compress.impl.IntValueKV; +import org.apache.rocketmq.streams.common.threadpool.ThreadPoolFactory; import org.apache.rocketmq.streams.common.utils.SQLUtil; import org.apache.rocketmq.streams.db.driver.DriverBuilder; import org.apache.rocketmq.streams.db.driver.JDBCDriver; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * 多线程批量加载数据,每加载一批数据后,通过IRowOperator回调接口处理数据 需要有递增的字段,这个字段有索引,不重复,如id字段 */ public class BatchRowLoader { - private static final Log LOG = LogFactory.getLog(BatchRowLoader.class); protected static final int MAX_LINE = 5000;//每个批次最大行数,根据这个值划分并行任务 + private static final Logger LOGGER = LoggerFactory.getLogger(BatchRowLoader.class); protected static ExecutorService executorService = null; + private final JDBCDriver jdbcDriver; protected String idFieldName;//配置字段名称,这个字段的值是数字的,且是递增的 protected String sql;//查询的sql语句,类似select * from table where idFieldName>#{idFieldName=0} order by idFieldName.不要加limit,系统会自动添加 protected int batchSize = 1000;//每批从数据库加载的数据量 protected IRowOperator dataRowProcessor;//加载的数据由这个回调接口处理 - private JDBCDriver jdbcDriver; public BatchRowLoader(String idFieldName, String sql, IRowOperator dataRowProcessor) { this.idFieldName = idFieldName; this.sql = sql; this.dataRowProcessor = dataRowProcessor; this.jdbcDriver = DriverBuilder.createDriver(); - executorService = new ThreadPoolExecutor(20, 20, - 0L, TimeUnit.MILLISECONDS, - new LinkedBlockingQueue(1000)); + executorService = ThreadPoolFactory.createFixedThreadPool(20, BatchRowLoader.class.getName() + "-" + idFieldName); + } + + public BatchRowLoader(String idFieldName, String sql, IRowOperator dataRowProcessor, String driver, String url, String userName, String password) { + this.idFieldName = idFieldName; + this.sql = sql; + this.dataRowProcessor = dataRowProcessor; + this.jdbcDriver = DriverBuilder.createDriver(driver, url, userName, password); + executorService = ThreadPoolFactory.createFixedThreadPool(20, BatchRowLoader.class.getName() + "-" + idFieldName); } public void startLoadData() { + startLoadData(Integer.MAX_VALUE); + } + + public boolean startLoadData(Integer maxValue) { try { + if (maxValue == null) { + maxValue = Integer.MAX_VALUE; + } String statisticalSQL = sql; int startIndex = sql.toLowerCase().indexOf("from"); - statisticalSQL = "select count(1) as c, min(" + idFieldName + ") as min, max(" + idFieldName + ") as max " - + sql.substring(startIndex); + statisticalSQL = "select count(1) as c, min(" + idFieldName + ") as min, max(" + idFieldName + ") as max " + sql.substring(startIndex); List> rows = jdbcDriver.queryForList(statisticalSQL); Map row = rows.get(0); - int count = Integer.valueOf(row.get("c").toString()); + int count = Integer.parseInt(row.get("c").toString()); if (count == 0) { - LOG.warn("there is no data during execute sql: " + statisticalSQL); - return; + LOGGER.debug("there is no data during execute sql: " + statisticalSQL); + return true; + } + int realCount = count; + if (count > maxValue) { + count = maxValue; } IntValueKV intValueKV = new IntValueKV(count); //int maxBatch=count/maxSyncCount;//每1w条数据,一个并发。如果数据量比较大,为了提高性能,并行执行 - long min = Long.valueOf(row.get("min").toString()); - long max = Long.valueOf(row.get("max").toString()); + long min = Long.parseLong(row.get("min").toString()); + long max = Long.parseLong(row.get("max").toString()); int maxSyncCount = count / MAX_LINE + 1; long step = (max - min + 1) / maxSyncCount; CountDownLatch countDownLatch = new CountDownLatch(maxSyncCount + 1); AtomicInteger finishedCount = new AtomicInteger(0); String taskSQL = null; - if (sql.indexOf(" where ") != -1) { - taskSQL = sql + " and " + idFieldName + ">#{startIndex} and " + idFieldName + "<=#{endIndex} order by " - + idFieldName + " limit " + batchSize; + if (sql.contains(" where ")) { + taskSQL = sql + " and " + idFieldName + ">#{startIndex} and " + idFieldName + "<=#{endIndex} order by " + idFieldName + " limit " + batchSize; } else { - taskSQL = sql + " where " + idFieldName + ">#{startIndex} and " + idFieldName - + "<=#{endIndex} order by " + idFieldName + " limit " + batchSize; + taskSQL = sql + " where " + idFieldName + ">#{startIndex} and " + idFieldName + "<=#{endIndex} order by " + idFieldName + " limit " + batchSize; } int i = 0; for (; i < maxSyncCount; i++) { - FetchDataTask - fetchDataTask = new FetchDataTask(taskSQL, (min - 1) + step * i, - (min - 1) + step * (i + 1), countDownLatch, finishedCount, jdbcDriver, count); + FetchDataTask fetchDataTask = new FetchDataTask(taskSQL, (min - 1) + step * i, (min - 1) + step * (i + 1), countDownLatch, finishedCount, jdbcDriver, count); executorService.execute(fetchDataTask); } - FetchDataTask - fetchDataTask = new FetchDataTask(taskSQL, (min - 1) + step * i, (min - 1) + step * (i + 1), - countDownLatch, finishedCount, jdbcDriver, count); + FetchDataTask fetchDataTask = new FetchDataTask(taskSQL, (min - 1) + step * i, (min - 1) + step * (i + 1), countDownLatch, finishedCount, jdbcDriver, count); executorService.execute(fetchDataTask); countDownLatch.await(); - - LOG.info(getClass().getSimpleName() + " load data finish, load data line size is " + count); + LOGGER.info(getClass().getSimpleName() + " load data finish, load data line size is " + count); + return realCount < maxValue; } catch (Exception e) { - LOG.error("failed loading data batch!", e); + LOGGER.error("failed loading data batch!", e); + return false; } finally { - jdbcDriver.destroy(); + if (jdbcDriver != null) { + jdbcDriver.destroy(); + } + if (executorService != null) { + executorService.shutdown(); + executorService = null; + } + } + } + + private void doProcess(List> rows) { + for (Map row : rows) { + dataRowProcessor.doProcess(row); } } @@ -118,8 +139,7 @@ protected class FetchDataTask implements Runnable { AtomicInteger finishedCount;//完成了多少条 int totalSize;//一共有多少条数据 - public FetchDataTask(String sql, long startIndex, long endIndex, CountDownLatch countDownLatch, - AtomicInteger finishedCount, JDBCDriver resource, int totalSize) { + public FetchDataTask(String sql, long startIndex, long endIndex, CountDownLatch countDownLatch, AtomicInteger finishedCount, JDBCDriver resource, int totalSize) { this.startIndex = startIndex; this.endIndex = endIndex; this.countDownLatch = countDownLatch; @@ -129,31 +149,27 @@ public FetchDataTask(String sql, long startIndex, long endIndex, CountDownLatch this.totalSize = totalSize; } - @Override - public void run() { + @Override public void run() { long currentIndex = startIndex; JSONObject msg = new JSONObject(); msg.put("endIndex", endIndex); while (true) { try { - msg.put("startIndex", currentIndex); - String sql = SQLUtil.parseIbatisSQL(msg, this.sql); List> rows = resource.queryForList(sql); if (rows == null || rows.size() == 0) { break; } - currentIndex = Long.valueOf(rows.get(rows.size() - 1).get(idFieldName).toString()); + currentIndex = Long.parseLong(rows.get(rows.size() - 1).get(idFieldName).toString()); int size = rows.size(); int count = finishedCount.addAndGet(size); - double progress = (double)count / (double)totalSize; + double progress = (double) count / (double) totalSize; progress = progress * 100; System.out.println(" finished count is " + count + " the total count is " + totalSize + ", the progress is " + String.format("%.2f", progress) + "%"); if (size < batchSize) { if (size > 0) { - doProcess(rows); } break; @@ -167,11 +183,5 @@ public void run() { countDownLatch.countDown(); } } - - private void doProcess(List> rows) { - for (Map row : rows) { - dataRowProcessor.doProcess(row); - } - } } diff --git a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/orm/ORMUtil.java b/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/orm/ORMUtil.java index e4e825de..acaa5e65 100644 --- a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/orm/ORMUtil.java +++ b/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/orm/ORMUtil.java @@ -18,17 +18,15 @@ import java.lang.reflect.Field; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Date; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.component.ComponentCreator; import org.apache.rocketmq.streams.common.configurable.IFieldProcessor; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.configuration.SystemContext; import org.apache.rocketmq.streams.common.datatype.DataType; import org.apache.rocketmq.streams.common.metadata.MetaData; import org.apache.rocketmq.streams.common.model.Entity; @@ -40,12 +38,14 @@ import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.db.driver.DriverBuilder; import org.apache.rocketmq.streams.db.driver.JDBCDriver; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * 轻量级的orm框架,如果pojo和table 符合驼峰的命名和下划线的命名规范,可以自动实现对象的orm */ public class ORMUtil { - private static final Log LOG = LogFactory.getLog(ORMUtil.class); + private static final Logger LOGGER = LoggerFactory.getLogger(ORMUtil.class); public static T queryForObject(String sql, Object paras, Class convertClass) { return queryForObject(sql, paras, convertClass, null, null, null); @@ -63,10 +63,9 @@ public static T queryForObject(String sql, Object paras, Class convertCla * @param * @return 转换后的对象 */ - public static T queryForObject(String sql, Object paras, Class convertClass, String url, String userName, - String password) { + public static T queryForObject(String sql, Object paras, Class convertClass, String url, String userName, String password) { List result = queryForList(sql, paras, convertClass, url, userName, password); - if (result == null || result.size() == 0) { + if (result.size() == 0) { return null; } if (result.size() > 1) { @@ -89,7 +88,7 @@ public static List queryForList(String sql, Object paras, Class conver } public static boolean hasConfigueDB() { - return ComponentCreator.getProperties().getProperty(ConfigureFileKey.JDBC_URL) != null; + return SystemContext.getProperty(ConfigurationKey.JDBC_URL) != null; } /** @@ -102,19 +101,18 @@ public static boolean hasConfigueDB() { * @param * @return 返回对象列表 */ - public static List queryForList(String sql, Object paras, Class convertClass, String url, String userName, - String password) { + public static List queryForList(String sql, Object paras, Class convertClass, String url, String userName, String password) { sql = SQLUtil.parseIbatisSQL(paras, sql); JDBCDriver dataSource = null; try { if (StringUtil.isEmpty(url) || StringUtil.isEmpty(userName)) { dataSource = DriverBuilder.createDriver(); } else { - dataSource = DriverBuilder.createDriver(DriverBuilder.DEFALUT_JDBC_DRIVER, url, userName, password); + dataSource = DriverBuilder.createDriver(ConfigurationKey.DEFAULT_JDBC_DRIVER, url, userName, password); } List> rows = dataSource.queryForList(sql); - List result = new ArrayList(); + List result = new ArrayList<>(); for (Map row : rows) { T t = convert(row, convertClass); result.add(t); @@ -122,7 +120,7 @@ public static List queryForList(String sql, Object paras, Class conver return result; } catch (Exception e) { String errorMsg = ("query for list error ,the builder is " + sql + ". the error msg is " + e.getMessage()); - LOG.error(errorMsg); + LOGGER.error(errorMsg); e.printStackTrace(); throw new RuntimeException(errorMsg, e); } finally { @@ -150,7 +148,7 @@ public static boolean executeSQL(String sql, Object paras) { return true; } catch (Exception e) { String errorMsg = ("execute sql error ,the sql is " + sql + ". the error msg is " + e.getMessage()); - LOG.error(errorMsg); + LOGGER.error(errorMsg); e.printStackTrace(); throw new RuntimeException(errorMsg, e); } finally { @@ -160,9 +158,7 @@ public static boolean executeSQL(String sql, Object paras) { } } - - public static boolean executeSQL(String sql, Object paras, String driver, final String url, final String userName, - final String password) { + public static boolean executeSQL(String sql, Object paras, String driver, final String url, final String userName, final String password) { if (paras != null) { sql = SQLUtil.parseIbatisSQL(paras, sql); } @@ -173,7 +169,7 @@ public static boolean executeSQL(String sql, Object paras, String driver, final return true; } catch (Exception e) { String errorMsg = ("execute sql error ,the sql is " + sql + ". the error msg is " + e.getMessage()); - LOG.error(errorMsg); + LOGGER.error(errorMsg); e.printStackTrace(); throw new RuntimeException(errorMsg, e); } finally { @@ -185,6 +181,7 @@ public static boolean executeSQL(String sql, Object paras, String driver, final /** * 执行sql,sql中可以有mybatis的参数#{name} + * * @param sql insert语句 * @param paras 可以是map,json或对象,只要key名或字段名和sql的参数名相同即可 * @return @@ -200,7 +197,7 @@ public static boolean executeSQL(String url, String userName, String password, S return true; } catch (Exception e) { String errorMsg = ("execute sql error ,the sql is " + sql + ". the error msg is " + e.getMessage()); - LOG.error(errorMsg); + LOGGER.error(errorMsg); e.printStackTrace(); throw new RuntimeException(errorMsg, e); } finally { @@ -234,7 +231,7 @@ public static String createQueryWhereSql(Object object, String... fieldNames) { stringBuilder.append(" and "); } String columnName = getColumnNameFromFieldName(fieldName); - stringBuilder.append(" " + columnName + "=#{" + fieldName + "} "); + stringBuilder.append(" ").append(columnName).append("=#{").append(fieldName).append("} "); } return stringBuilder.toString(); } @@ -249,16 +246,14 @@ public static String createQueryWhereSql(Object object, String... fieldNames) { */ public static T convert(Map row, Class clazz) { T t = ReflectUtil.forInstance(clazz); - Iterator> it = row.entrySet().iterator(); - while (it.hasNext()) { - Map.Entry entry = it.next(); + for (Map.Entry entry : row.entrySet()) { String columnName = entry.getKey(); Object value = entry.getValue(); if (value == null) { continue; } String fieldName = getFieldNameFromColumnName(columnName); - DataType datatype = DataTypeUtil.createFieldDataType(clazz, fieldName); + DataType datatype = DataTypeUtil.createFieldDataType(clazz, fieldName); Object columnValue = datatype.convert(value); ReflectUtil.setBeanFieldValue(t, fieldName, columnValue); } @@ -291,18 +286,15 @@ protected static String getFieldNameFromColumnName(String columnName) { * * @param values 待插入对象 */ - public static void batchReplaceInto(Collection values) { - List list = new ArrayList<>(); - list.addAll(values); + public static void batchReplaceInto(Collection values) { + List list = new ArrayList<>(values); batchReplaceInto(list); } public static void batchReplaceInto(Object... valueArray) { - List values = new ArrayList(); + List values = new ArrayList<>(); if (valueArray != null) { - for (Object value : valueArray) { - values.add(value); - } + values.addAll(Arrays.asList(valueArray)); } batchReplaceInto(values); } @@ -328,23 +320,22 @@ public static void batchIgnoreInto(List values) { public static void batchInsertInto(List values) { batchIntoByFlag(values, 0); } - public static String createBatchReplacetSQL(Object...values) { - if(values==null){ + + public static String createBatchReplaceSQL(Object... values) { + if (values == null) { return null; } - List list=new ArrayList<>(); - for(Object value:values){ - list.add(value); - } - return createBatchReplacetSQL(list); + List list = new ArrayList<>(Arrays.asList(values)); + return createBatchReplaceSQL(list); } + /** * 批量插入对象,多个对象会拼接成一个sql flag==1 then replace into flag=-1 then insert ignore int flag=0 then insert int * * @param values */ - public static String createBatchReplacetSQL(List values) { - return createBatchInsertSQL(values,1); + public static String createBatchReplaceSQL(List values) { + return createBatchInsertSQL(values, 1); } /** @@ -365,19 +356,16 @@ public static String createBatchInsertSQL(List values, int flag) { for (Object o : values) { Object id = ReflectUtil.getDeclaredField(o, metaData.getIdFieldName()); if (id == null) { - containsIdField = false; break; } if (id instanceof Number) { - if (Long.valueOf(id.toString()) == 0) { - containsIdField = false; + if (Long.parseLong(id.toString()) == 0) { break; } } if (id instanceof String) { - String idStr = (String)id; + String idStr = (String) id; if (StringUtil.isEmpty(idStr)) { - containsIdField = false; break; } } @@ -404,6 +392,7 @@ public static String createBatchInsertSQL(List values, int flag) { sql = sql + valuesSQL + " ON DUPLICATE KEY UPDATE " + SQLUtil.createDuplicateKeyUpdateSQL(metaData); return sql; } + /** * 批量插入对象,多个对象会拼接成一个sql flag==1 then replace into flag=-1 then insert ignore int flag=0 then insert int * @@ -411,7 +400,7 @@ public static String createBatchInsertSQL(List values, int flag) { * @param flag */ protected static void batchIntoByFlag(List values, int flag) { - String sql=createBatchInsertSQL(values,flag); + String sql = createBatchInsertSQL(values, flag); JDBCDriver dataSource = DriverBuilder.createDriver(); try { dataSource.execute(sql); @@ -419,17 +408,14 @@ protected static void batchIntoByFlag(List values, int flag) { e.printStackTrace(); throw new RuntimeException(e); } finally { - if (dataSource != null) { - dataSource.destroy(); - } + dataSource.destroy(); } } private static Map createRow(MetaData metaData, Object object) { Map row = new HashMap<>(); ReflectUtil.scanFields(object, new IFieldProcessor() { - @Override - public void doProcess(Object o, Field field) { + @Override public void doProcess(Object o, Field field) { String fieldName = field.getName(); String columnName = getColumnNameFromFieldName(fieldName); Object value = ReflectUtil.getBeanFieldValue(o, fieldName); @@ -454,8 +440,8 @@ public static void replaceInto(Object object) { */ public static void replaceInto(Object object, String url, String userName, String password) { Map paras = new HashMap<>(); - if (Entity.class.isInstance(object)) { - Entity newEntity = (Entity)object; + if (object instanceof Entity) { + Entity newEntity = (Entity) object; newEntity.setGmtModified(new Date()); if (newEntity.getGmtCreate() == null) { newEntity.setGmtCreate(new Date()); @@ -468,16 +454,16 @@ public static void replaceInto(Object object, String url, String userName, Strin if (StringUtil.isEmpty(url) || StringUtil.isEmpty(userName)) { dataSource = DriverBuilder.createDriver(); } else { - dataSource = DriverBuilder.createDriver(DriverBuilder.DEFALUT_JDBC_DRIVER, url, userName, password); + dataSource = DriverBuilder.createDriver(ConfigurationKey.DEFAULT_JDBC_DRIVER, url, userName, password); } long id = dataSource.executeInsert(sql); - if (Entity.class.isInstance(object)) { - Entity newEntity = (Entity)object; + if (object instanceof Entity) { + Entity newEntity = (Entity) object; newEntity.setId(id); } } catch (Exception e) { String errorMsg = ("replace into error ,the builder is " + sql + ". the error msg is " + e.getMessage()); - LOG.error(errorMsg); + LOGGER.error(errorMsg); throw new RuntimeException(errorMsg, e); } finally { if (dataSource != null) { @@ -499,11 +485,10 @@ public static void insertInto(Object object, boolean ignoreRepeateRow) { * @param userName * @param password */ - public static void insertInto(Object object, boolean ignoreRepeateRow, String url, String userName, - String password) { + public static void insertInto(Object object, boolean ignoreRepeateRow, String url, String userName, String password) { Map paras = new HashMap<>(); - if (Entity.class.isInstance(object)) { - Entity newEntity = (Entity)object; + if (object instanceof Entity) { + Entity newEntity = (Entity) object; newEntity.setGmtCreate(DateUtil.getCurrentTime()); newEntity.setGmtModified(DateUtil.getCurrentTime()); } @@ -519,16 +504,16 @@ public static void insertInto(Object object, boolean ignoreRepeateRow, String ur if (StringUtil.isEmpty(url) || StringUtil.isEmpty(userName)) { dataSource = DriverBuilder.createDriver(); } else { - dataSource = DriverBuilder.createDriver(DriverBuilder.DEFALUT_JDBC_DRIVER, url, userName, password); + dataSource = DriverBuilder.createDriver(ConfigurationKey.DEFAULT_JDBC_DRIVER, url, userName, password); } long id = dataSource.executeInsert(sql); - if (Entity.class.isInstance(object)) { - Entity newEntity = (Entity)object; + if (object instanceof Entity) { + Entity newEntity = (Entity) object; newEntity.setId(id); } } catch (Exception e) { String errorMsg = ("insert into error ,the builder is " + sql + ". the error msg is " + e.getMessage()); - LOG.error(errorMsg); + LOGGER.error(errorMsg); throw new RuntimeException(errorMsg, e); } finally { if (dataSource != null) { @@ -545,8 +530,7 @@ public static void insertInto(Object object, boolean ignoreRepeateRow, String ur * @return */ public static MetaData createMetaDate(Object object, Map paras) { - MetaData metaData = MetaData.createMetaDate(object, paras); - return metaData; + return MetaData.createMetaDate(object, paras); } public static String getTableName(Class clazz) { diff --git a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/operator/SQLOperator.java b/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/operator/SQLOperator.java index 846f20d1..1c51d496 100644 --- a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/operator/SQLOperator.java +++ b/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/operator/SQLOperator.java @@ -18,18 +18,16 @@ import java.util.List; import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.component.AbstractComponent; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; import org.apache.rocketmq.streams.common.configurable.annotation.Changeable; import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; -import org.apache.rocketmq.streams.common.topology.ChainStage; -import org.apache.rocketmq.streams.common.topology.builder.IStageBuilder; +import org.apache.rocketmq.streams.common.topology.IStageBuilder; import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; +import org.apache.rocketmq.streams.common.topology.model.AbstractChainStage; import org.apache.rocketmq.streams.common.topology.stages.NewSQLChainStage; import org.apache.rocketmq.streams.common.utils.SQLUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; @@ -39,12 +37,11 @@ /** * sql算法,执行一个sql,sql中可以有变量,会用message的值做替换。 */ -public class SQLOperator extends BasedConfigurable implements IStreamOperator, IStageBuilder { - private static final Log LOG = LogFactory.getLog(SQLOperator.class); +public class SQLOperator extends BasedConfigurable implements IStreamOperator, IStageBuilder { public static final String DEFALUT_DATA_KEY = "data"; @ENVDependence - protected String jdbcDriver = AbstractComponent.DEFAULT_JDBC_DRIVER; + protected String jdbcDriver = ConfigurationKey.DEFAULT_JDBC_DRIVER; @ENVDependence protected String url; @ENVDependence @@ -76,7 +73,7 @@ public SQLOperator(String sql, String url, String userName, String password) { public SQLOperator(String sql, String dbInfoNamePrex) { this(); if (StringUtil.isEmpty(dbInfoNamePrex)) { - dbInfoNamePrex = getConfigureName(); + dbInfoNamePrex = getName(); } if (StringUtil.isEmpty(dbInfoNamePrex)) { dbInfoNamePrex = this.getClass().getSimpleName(); @@ -128,7 +125,7 @@ public void addConfigurables(PipelineBuilder pipelineBuilder) { } @Override - public ChainStage createStageChain(PipelineBuilder pipelineBuilder) { + public AbstractChainStage createStageChain(PipelineBuilder pipelineBuilder) { NewSQLChainStage sqlChainStage = new NewSQLChainStage(); sqlChainStage.setMessageProcessor(this); return sqlChainStage; diff --git a/rocketmq-streams-db-operator/src/test/java/org/apache/rocketmq/streams/db/Person.java b/rocketmq-streams-db-operator/src/test/java/org/apache/rocketmq/streams/db/Person.java index 90551b71..493dd20a 100644 --- a/rocketmq-streams-db-operator/src/test/java/org/apache/rocketmq/streams/db/Person.java +++ b/rocketmq-streams-db-operator/src/test/java/org/apache/rocketmq/streams/db/Person.java @@ -35,7 +35,7 @@ public static Person createPerson(String namespace) { Person person = new Person(); person.setNameSpace(namespace); person.setType("person"); - person.setConfigureName("Chris"); + person.setName("Chris"); person.setName("Chris"); List addresses = new ArrayList<>(); addresses.add("huilongguan"); @@ -100,7 +100,7 @@ public void setChildName2Age(Map childName2Age) { public Object clone() { Person person = null; try { - person = (Person)super.clone(); + person = (Person) super.clone(); } catch (CloneNotSupportedException e) { System.out.println("clone error " + e); } diff --git a/rocketmq-streams-db-operator/src/test/java/org/apache/rocketmq/streams/db/configuable/DBSupportParentConfigureServiceTest.java b/rocketmq-streams-db-operator/src/test/java/org/apache/rocketmq/streams/db/configuable/DBSupportParentConfigureServiceTest.java deleted file mode 100644 index 3baa65d6..00000000 --- a/rocketmq-streams-db-operator/src/test/java/org/apache/rocketmq/streams/db/configuable/DBSupportParentConfigureServiceTest.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * 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.db.configuable; - -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; -import org.apache.rocketmq.streams.configurable.ConfigurableComponent; -import org.apache.rocketmq.streams.configurable.model.Configure; -import org.apache.rocketmq.streams.db.Person; -import org.apache.rocketmq.streams.db.driver.DriverBuilder; -import org.junit.Test; - -import static junit.framework.TestCase.assertTrue; - -/** - * 数据库的存储,需要配置存储的连接参数,请先完成配置,后执行单元用例 如果未建表,可以通过Configure.createTableSQL() 获取建表语句,创建表后,测试 - */ -public class DBSupportParentConfigureServiceTest { - private String URL = ""; - protected String USER_NAME = ""; - protected String PASSWORD = ""; - protected String TABLE_NAME = "dipper_configure_source"; - - @Test - public void testDBConfigurableService() { - String namespace = "streams.db.configuable"; - - //正式使用时,在配置文件配置 - ComponentCreator.getProperties().put(ConfigureFileKey.CONNECT_TYPE, "DB"); - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_URL, URL);//数据库连接url - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_USERNAME, USER_NAME);//用户名 - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_PASSWORD, PASSWORD);//password - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_TABLE_NAME, TABLE_NAME); - - //如果表不存在,创建表 - String sql = (Configure.createTableSQL(TABLE_NAME)); - DriverBuilder.createDriver().execute(sql); - ConfigurableComponent configurableComponent = ConfigurableComponent.getInstance(namespace); - configurableComponent.insert(createPerson(namespace)); - configurableComponent.refreshConfigurable(namespace); - Person person = configurableComponent.queryConfigurable("person", "peronName"); - assertTrue(person != null); - } - - /** - * 创建configuable对象 - * - * @param namespace - * @return - */ - protected Person createPerson(String namespace) { - Person person = new Person(); - person.setName("chris"); - person.setAge(18); - person.setNameSpace(namespace); - person.setConfigureName("peronName"); - person.setType("person"); - return person; - } -} diff --git a/rocketmq-streams-db-operator/src/test/java/org/apache/rocketmq/streams/db/driver/orm/ORMUtilTest.java b/rocketmq-streams-db-operator/src/test/java/org/apache/rocketmq/streams/db/driver/orm/ORMUtilTest.java index 1c8b2099..449893fb 100644 --- a/rocketmq-streams-db-operator/src/test/java/org/apache/rocketmq/streams/db/driver/orm/ORMUtilTest.java +++ b/rocketmq-streams-db-operator/src/test/java/org/apache/rocketmq/streams/db/driver/orm/ORMUtilTest.java @@ -20,21 +20,19 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; import org.apache.rocketmq.streams.db.Person; import org.junit.Test; public class ORMUtilTest { - private String URL = ""; protected String USER_NAME = ""; protected String PASSWORD = ""; + private String URL = ""; public ORMUtilTest() { //正式使用时,在配置文件配置 - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_URL, URL);//数据库连接url - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_USERNAME, USER_NAME);//用户名 - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_PASSWORD, PASSWORD);//password +// ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_URL, URL);//数据库连接url +// ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_USERNAME, USER_NAME);//用户名 +// ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_PASSWORD, PASSWORD);//password } @Test @@ -78,7 +76,7 @@ protected Person createPerson(String namespace, String name) { person.setName(name); person.setAge(18); person.setNameSpace(namespace); - person.setConfigureName("peronName"); + person.setName("peronName"); person.setType("person"); return person; } diff --git a/rocketmq-streams-dbinit/src/main/resources/tables_mysql_innodb.sql b/rocketmq-streams-dbinit/src/main/resources/tables_mysql_innodb.sql deleted file mode 100644 index 2f022c67..00000000 --- a/rocketmq-streams-dbinit/src/main/resources/tables_mysql_innodb.sql +++ /dev/null @@ -1,201 +0,0 @@ -CREATE TABLE IF NOT EXISTS `window_max_value` ( - `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT, - `gmt_create` datetime NOT NULL, - `gmt_modified` datetime NOT NULL, - `max_offset` varchar(20) NOT NULL, - `is_max_offset_long` int(11) DEFAULT NULL, - `max_value` bigint(20) unsigned NOT NULL, - `max_event_time` bigint(20) unsigned NOT NULL, - `msg_key` varchar(256) NOT NULL, - PRIMARY KEY (`id`), - UNIQUE KEY `uk__ket` (`msg_key`(250)), - KEY `idx_modifytime` (`gmt_modified`) -) ENGINE=InnoDB DEFAULT CHARSET=utf8; - -CREATE TABLE IF NOT EXISTS `window_value` ( - `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT, - `gmt_create` datetime NOT NULL, - `gmt_modified` datetime NOT NULL, - `start_time` varchar(20) NOT NULL, - `end_time` varchar(20) NOT NULL, - `max_offset` longtext, - `group_by` text, - `agg_column_result` longtext, - `computed_column_result` longtext, - `version` varchar(64) DEFAULT NULL, - `name_space` varchar(256) DEFAULT NULL, - `configure_name` varchar(256) DEFAULT NULL, - `msg_key` varchar(64) NOT NULL, - `window_instance_id` varchar(64) NOT NULL, - `partition` varchar(512) DEFAULT NULL, - `partition_num` bigint(20) DEFAULT NULL, - `fire_time` varchar(20) DEFAULT NULL, - `update_version` bigint(20) unsigned DEFAULT NULL, - `update_flag` bigint(20) DEFAULT NULL, - `window_instance_partition_id` varchar(64) DEFAULT NULL, - `type` varchar(64) DEFAULT NULL, - PRIMARY KEY (`id`), - UNIQUE KEY `uk_window_state` (`msg_key`), - KEY `idx_window_instance_shuffle` (`window_instance_partition_id`,`partition_num`), - KEY `idx_window_instance_firetime` (`window_instance_partition_id`,`fire_time`), - KEY `idx_window` (`name_space`(128),`configure_name`(128),`partition`(128)) -) ENGINE=InnoDB DEFAULT CHARSET=utf8; - -CREATE TABLE IF NOT EXISTS `window_task` ( - `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT, - `gmt_create` datetime NOT NULL DEFAULT CURRENT_TIMESTAMP, - `gmt_modified` datetime NOT NULL DEFAULT CURRENT_TIMESTAMP, - `task_id` varchar(64) NOT NULL, - `untreated_flag` int(11) NOT NULL DEFAULT '0', - `group_by_value` varchar(1024) NOT NULL, - `task_owner` varchar(256) NOT NULL, - `task_send_time` datetime DEFAULT NULL, - `send_task_msg` text NOT NULL, - `msg_send_time` bigint(20) DEFAULT NULL, - `name` varchar(128) NOT NULL, - `start_time` varchar(20) NOT NULL, - `end_time` varchar(20) NOT NULL, - PRIMARY KEY (`id`), - UNIQUE KEY `uk_taskid` (`task_id`), - KEY `idx_flag_modifytime` (`name`,`untreated_flag`,`gmt_modified`) -) ENGINE=InnoDB DEFAULT CHARSET=utf8; - -CREATE TABLE IF NOT EXISTS `window_instance` ( - `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT, - `gmt_create` datetime NOT NULL, - `gmt_modified` datetime NOT NULL, - `start_time` varchar(20) NOT NULL, - `end_time` varchar(20) NOT NULL, - `fire_time` varchar(20) NOT NULL, - `window_name` varchar(128) NOT NULL, - `window_name_space` varchar(128) NOT NULL, - `status` tinyint(4) NOT NULL DEFAULT '0', - `version` int(11) DEFAULT '0', - `window_instance_key` varchar(128) DEFAULT NULL, - `window_instance_name` varchar(128) DEFAULT NULL, - `window_Instance_split_name` varchar(128) DEFAULT NULL, - `split_id` varchar(128) DEFAULT NULL, - PRIMARY KEY (`id`), - UNIQUE KEY `uk_window_instance_uniq_index` (`window_instance_key`), - KEY `idx_gmt_modified` (`fire_time`,`window_name`,`window_name_space`,`status`), - KEY `idx_windowinstance_name` (`window_instance_name`), - KEY `idx_windowinstance_split_name` (`window_Instance_split_name`), - KEY `idx_windowinstance_split_name_firetime` (`window_Instance_split_name`,`fire_time`) -) ENGINE=InnoDB DEFAULT CHARSET=utf8; - -CREATE TABLE IF NOT EXISTS `lease_info` ( - `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT, - `gmt_create` datetime NOT NULL, - `gmt_modified` datetime NOT NULL, - `lease_name` varchar(255) NOT NULL, - `lease_user_ip` varchar(255) NOT NULL, - `lease_end_time` varchar(255) NOT NULL, - `status` int(11) NOT NULL DEFAULT '1', - `version` bigint(20) NOT NULL, - `candidate_lease_ip` varchar(255) DEFAULT NULL, - PRIMARY KEY (`id`), - UNIQUE KEY `uk_name` (`lease_name`) -) ENGINE=InnoDB DEFAULT CHARSET=utf8; - -CREATE TABLE IF NOT EXISTS `dipper_sql_configure` ( - `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT, - `gmt_create` datetime NOT NULL, - `gmt_modified` datetime NOT NULL, - `namespace` varchar(32) NOT NULL, - `type` varchar(32) NOT NULL, - `name` varchar(128) NOT NULL, - `json_value` longtext NOT NULL, - `request_id` varchar(128) NOT NULL, - `account_id` varchar(32) NOT NULL, - `account_name` varchar(32) NOT NULL, - `account_nickname` varchar(32) NOT NULL, - `client_ip` varchar(64) NOT NULL, - `status` tinyint(3) unsigned NOT NULL DEFAULT '0', - `is_publish` int(11) NOT NULL DEFAULT '0', - PRIMARY KEY (`id`), - UNIQUE KEY `uk_namespace_type_name` (`namespace`,`type`,`name`), - KEY `idx_namespace` (`namespace`) -) ENGINE=InnoDB DEFAULT CHARSET=utf8; - -CREATE TABLE IF NOT EXISTS `dipper_configure` ( - `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT, - `gmt_create` datetime NOT NULL, - `gmt_modified` datetime NOT NULL, - `namespace` varchar(32) NOT NULL, - `type` varchar(32) NOT NULL, - `name` varchar(128) NOT NULL, - `json_value` text NOT NULL, - `request_id` varchar(128) DEFAULT NULL, - `account_id` varchar(32) DEFAULT NULL, - `account_name` varchar(32) DEFAULT NULL, - `account_nickname` varchar(32) DEFAULT NULL, - `client_ip` varchar(64) DEFAULT NULL, - `status` tinyint(3) unsigned NOT NULL DEFAULT '0', - `isPublish` int(1) NOT NULL DEFAULT '0', - PRIMARY KEY (`id`), - UNIQUE KEY `uk_namespace_type_name` (`namespace`,`type`,`name`), - KEY `idx_namespace` (`namespace`) -) ENGINE=InnoDB DEFAULT CHARSET=utf8; - -CREATE TABLE IF NOT EXISTS `join_right_state` ( - `id` bigint(20) NOT NULL AUTO_INCREMENT, - `gmt_create` datetime DEFAULT NULL, - `gmt_modified` datetime DEFAULT NULL, - `window_id` bigint(20) DEFAULT NULL, - `window_name` varchar(200) DEFAULT NULL, - `window_name_space` varchar(45) DEFAULT NULL, - `message_id` varchar(200) DEFAULT NULL, - `message_key` varchar(32) DEFAULT NULL, - `message_time` datetime DEFAULT NULL, - `message_body` longtext, - `msg_key` varchar(400) DEFAULT NULL, - `window_instance_id` varchar(200) DEFAULT NULL, - `partition` varchar(200) DEFAULT NULL, - `partition_num` bigint(20) DEFAULT NULL, - `window_instance_partition_id` varchar(200) DEFAULT NULL, - `version` varchar(64) DEFAULT NULL, - `update_flag` bigint(20) DEFAULT NULL, - `name_space` varchar(256) DEFAULT NULL, - `configure_name` varchar(256) DEFAULT NULL, - `type` varchar(64) DEFAULT NULL, - `name` varchar(64) DEFAULT NULL, - `update_version` bigint(20) unsigned DEFAULT NULL, - PRIMARY KEY (`id`), - UNIQUE KEY `uk_message_id_unique` (`message_id`), - KEY `idx_message_key_index` (`message_key`), - KEY `idx_gmt_create_index` (`gmt_create`), - KEY `idx_window_name_index` (`window_name`(70)), - KEY `idx_message_key_gmt_create_index` (`message_key`,`gmt_create`) -) ENGINE=InnoDB DEFAULT CHARSET=utf8; - -CREATE TABLE IF NOT EXISTS `join_left_state` ( - `id` bigint(20) NOT NULL AUTO_INCREMENT, - `gmt_create` datetime DEFAULT NULL, - `gmt_modified` datetime DEFAULT NULL, - `window_id` bigint(20) DEFAULT NULL, - `window_name` varchar(200) DEFAULT NULL, - `window_name_space` varchar(45) DEFAULT NULL, - `message_id` varchar(200) DEFAULT NULL, - `message_key` varchar(32) DEFAULT NULL, - `message_time` datetime DEFAULT NULL, - `message_body` longtext, - `msg_key` varchar(400) DEFAULT NULL, - `window_instance_id` varchar(200) DEFAULT NULL, - `partition` varchar(200) DEFAULT NULL, - `partition_num` bigint(20) DEFAULT NULL, - `window_instance_partition_id` varchar(200) DEFAULT NULL, - `version` varchar(64) DEFAULT NULL, - `update_flag` bigint(20) DEFAULT NULL, - `name_space` varchar(256) DEFAULT NULL, - `configure_name` varchar(256) DEFAULT NULL, - `type` varchar(64) DEFAULT NULL, - `name` varchar(64) DEFAULT NULL, - `update_version` bigint(20) unsigned DEFAULT NULL, - PRIMARY KEY (`id`), - UNIQUE KEY `uk_message_id_unique` (`message_id`), - KEY `idx_message_key_index` (`message_key`), - KEY `idx_gmt_create_index` (`gmt_create`), - KEY `idx_window_name_index` (`window_name`(70)), - KEY `idx_message_key_gmt_create_index` (`message_key`,`gmt_create`) -) ENGINE=InnoDB DEFAULT CHARSET=utf8; - diff --git a/rocketmq-streams-dim/pom.xml b/rocketmq-streams-dim/pom.xml index 72d35c26..7473bf95 100644 --- a/rocketmq-streams-dim/pom.xml +++ b/rocketmq-streams-dim/pom.xml @@ -15,14 +15,14 @@ See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 org.apache.rocketmq rocketmq-streams - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT rocketmq-streams-dim ROCKETMQ STREAMS :: dim @@ -49,5 +49,4 @@ - diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/DimComponent.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/DimComponent.java deleted file mode 100644 index fbe79f17..00000000 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/DimComponent.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.dim; - -import java.util.Properties; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.component.AbstractComponent; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.configurable.ConfigurableComponent; -import org.apache.rocketmq.streams.dim.service.IDimService; -import org.apache.rocketmq.streams.dim.service.impl.DimServiceImpl; - -public class DimComponent extends AbstractComponent { - - private static final Log LOG = LogFactory.getLog(DimComponent.class); - - // private transient Map nameListMap = new HashMap<>(); - - protected transient ConfigurableComponent configurableComponent; - private transient IDimService dimService; - - public static DimComponent getInstance(String namespace) { - return ComponentCreator.getComponent(namespace, ComponentCreator.class); - } - - @Override - protected boolean startComponent(String namespace) { - configurableComponent = ComponentCreator.getComponent(namespace, ConfigurableComponent.class); - dimService = new DimServiceImpl(configurableComponent); - return true; - } - - @Override - protected boolean initProperties(Properties properties) { - return true; - } - - @Override - public boolean stop() { - return true; - } - - @Override - public IDimService getService() { - return dimService; - } -} diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/AbstractDimParser.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/AbstractDimParser.java index c5e41f4d..4aee226c 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/AbstractDimParser.java +++ b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/AbstractDimParser.java @@ -17,22 +17,23 @@ package org.apache.rocketmq.streams.dim.builder; import java.util.Properties; +import org.apache.rocketmq.streams.common.interfaces.IDim; import org.apache.rocketmq.streams.common.metadata.MetaData; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.dim.model.AbstractDim; public abstract class AbstractDimParser implements IDimSQLParser { @Override - public AbstractDim parseDim(String namespace, Properties properties, MetaData metaData) { + public IDim parseDim(String namespace, Properties properties, MetaData metaData) { AbstractDim dim = createDim(properties, metaData); String cacheTTLMs = properties.getProperty("cacheTTLMs"); long pollingTime = 30;//默认更新时间是30分钟 if (StringUtil.isNotEmpty(cacheTTLMs)) { - pollingTime = (Long.valueOf(cacheTTLMs) / 1000 / 60); + pollingTime = (Long.valueOf(cacheTTLMs) / 1000); } dim.setNameSpace(namespace); - dim.setPollingTimeMinute(pollingTime); + dim.setPollingTimeSeconds(pollingTime); String isLarge = properties.getProperty("isLarge"); if (isLarge == null || "false".equalsIgnoreCase(isLarge)) { diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/DimBuilder.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/DimBuilder.java index 45c45ce5..e2f3bddc 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/DimBuilder.java +++ b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/DimBuilder.java @@ -16,16 +16,17 @@ */ package org.apache.rocketmq.streams.dim.builder; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.dim.model.DBDim; public class DimBuilder { + protected Long pollingTimeSeconds = 60 * 10L; // 同步数据的事件间隔 private String url; private String password; private String userName; - protected Long pollingTime = 60L; // 同步数据的事件间隔 - private String jdbcDriver = "com.mysql.jdbc.Driver"; + private String jdbcDriver = ConfigurationKey.DEFAULT_JDBC_DRIVER; public DimBuilder(String url, String userName, String password) { this.url = url; @@ -37,15 +38,15 @@ public DBDim createDim(String namespace, String name, String sqlOrTableName) { DBDim nameList = new DBDim(); nameList.setNameSpace(namespace); if (StringUtil.isNotEmpty(name)) { - nameList.setConfigureName(name); + nameList.setName(name); } String sql = sqlOrTableName; if (sqlOrTableName.split(" ").length == 1) { sql = "select * from " + sqlOrTableName + " limit 500000"; } nameList.setSql(sql); - nameList.setJdbcdriver(jdbcDriver); - nameList.setPollingTimeMinute(pollingTime); + nameList.setJdbcDriver(jdbcDriver); + nameList.setPollingTimeSeconds(pollingTimeSeconds); nameList.setUrl(url); nameList.setUserName(userName); nameList.setPassword(password); @@ -76,12 +77,12 @@ public void setUserName(String userName) { this.userName = userName; } - public Long getPollingTime() { - return pollingTime; + public Long getPollingTimeSeconds() { + return pollingTimeSeconds; } - public void setPollingTime(Long pollingTime) { - this.pollingTime = pollingTime; + public void setPollingTimeSeconds(Long pollingTimeSeconds) { + this.pollingTimeSeconds = pollingTimeSeconds; } public String getJdbcDriver() { diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/FileDimSQLParser.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/FileDimSQLParser.java index ae24a7f5..be515cdf 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/FileDimSQLParser.java +++ b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/FileDimSQLParser.java @@ -26,16 +26,16 @@ @AutoService(IDimSQLParser.class) @ServiceName(value = FileDimSQLParser.TYPE, aliasName = "FILE") -public class FileDimSQLParser extends AbstractDimParser{ - public static final String TYPE="file"; +public class FileDimSQLParser extends AbstractDimParser { + public static final String TYPE = "file"; @Override protected AbstractDim createDim(Properties properties, MetaData data) { - String filePath=properties.getProperty("filePath"); - if(StringUtil.isEmpty(filePath)){ - filePath=properties.getProperty("file_path"); + String filePath = properties.getProperty("filePath"); + if (StringUtil.isEmpty(filePath)) { + filePath = properties.getProperty("file_path"); } - FileDim fileDim=new FileDim(); + FileDim fileDim = new FileDim(); fileDim.setFilePath(filePath); return fileDim; } diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/IDimSQLParser.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/IDimSQLParser.java index e94f0687..778a82b2 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/IDimSQLParser.java +++ b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/IDimSQLParser.java @@ -17,14 +17,14 @@ package org.apache.rocketmq.streams.dim.builder; import java.util.Properties; +import org.apache.rocketmq.streams.common.interfaces.IDim; import org.apache.rocketmq.streams.common.metadata.MetaData; -import org.apache.rocketmq.streams.dim.model.AbstractDim; /** * create dim by sql parser */ public interface IDimSQLParser { - AbstractDim parseDim(String namespace,Properties properties, MetaData metaData); + IDim parseDim(String namespace, Properties properties, MetaData metaData); } diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/expression/InExpressionResource.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/expression/InExpressionResource.java deleted file mode 100644 index c7a0ca4a..00000000 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/expression/InExpressionResource.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * 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.dim.function.expression; - -import java.util.Map; -import org.apache.rocketmq.streams.common.context.AbstractContext; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.dim.model.DBDim; -import org.apache.rocketmq.streams.filter.function.expression.AbstractExpressionFunction; -import org.apache.rocketmq.streams.filter.operator.expression.Expression; -import org.apache.rocketmq.streams.script.annotation.Function; -import org.apache.rocketmq.streams.script.annotation.FunctionMethod; -import org.apache.rocketmq.streams.script.annotation.FunctionMethodAilas; - -@Function -public class InExpressionResource extends AbstractExpressionFunction { - - /** - * value格式 :resourceName.fieldName。如果只有单列,可以省略.fieldname - * - * @param expression - * @param context - * @return - */ - @FunctionMethod(value = "in_expression_resouce", alias = "in_resouce") - @FunctionMethodAilas("in_expression_resouce(resourceName->(varName,functionName,value)&((varName,functionName," - + "value)|(varName,functionName,value)))") - @Override - public Boolean doExpressionFunction(IMessage message, AbstractContext context, Expression expression) { - return match(message,context,expression, false); - } - - protected Boolean match(IMessage message, AbstractContext context, Expression expression, boolean supportRegex) { - Object value = expression.getValue(); - if (value == null) { - return null; - } - String valueStr = String.valueOf(value); - String[] valueArray = valueStr.split("->"); - String dataResourceNamespace = expression.getNameSpace(); - String dataResourceName = null; - String expressionStr = null; - if (valueArray.length == 2) { - dataResourceName = valueArray[0]; - expressionStr = valueArray[1]; - } - if (valueArray.length > 2) { - dataResourceNamespace = valueArray[0]; - dataResourceName = valueArray[1]; - expressionStr = valueArray[2]; - } - - DBDim dataResource = - (DBDim)context.getConfigurableService().queryConfigurableByIdent(DBDim.TYPE, dataResourceName); - if (dataResource == null) { - return null; - } - Map row = dataResource.matchExpression(expressionStr, context.getMessage().getMessageBody()); - if (row != null && row.size() > 0) { - return true; - } - return false; - } -} diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/expression/NotInExpressionResource.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/expression/NotInExpressionResource.java deleted file mode 100644 index 001c5500..00000000 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/expression/NotInExpressionResource.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.dim.function.expression; - -import org.apache.rocketmq.streams.common.context.AbstractContext; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.filter.operator.expression.Expression; -import org.apache.rocketmq.streams.script.annotation.Function; -import org.apache.rocketmq.streams.script.annotation.FunctionMethod; -import org.apache.rocketmq.streams.script.annotation.FunctionMethodAilas; - -@Function -public class NotInExpressionResource extends InExpressionResource { - - /** - * value格式 :resourceName.fieldName。如果只有单列,可以省略.fieldname - * - * @param expression - * @param context - * @return - */ - @FunctionMethod("not_in_expression_resouce") - @FunctionMethodAilas("not_in_expression_resouce(resourceName->(varName,functionName,value)&((varName," - + "functionName,value)|(varName,functionName,value)))") - @Override - public Boolean doExpressionFunction(IMessage message, AbstractContext context, Expression expression) { - return !match(message,context,expression, false); - } - -} diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/IntelligenceFunction.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/IntelligenceFunction.java index 34ebc043..35ef148f 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/IntelligenceFunction.java +++ b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/IntelligenceFunction.java @@ -16,28 +16,14 @@ */ package org.apache.rocketmq.streams.dim.function.script; -import java.util.Map; -import java.util.Map.Entry; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.component.ComponentCreator; import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.utils.StringUtil; -import org.apache.rocketmq.streams.configurable.ConfigurableComponent; -import org.apache.rocketmq.streams.dim.DimComponent; -import org.apache.rocketmq.streams.dim.intelligence.AbstractIntelligenceCache; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; import org.apache.rocketmq.streams.script.context.FunctionContext; -import org.apache.rocketmq.streams.script.utils.FunctionUtils; @Function public class IntelligenceFunction { - private static final Log LOG = LogFactory.getLog(IntelligenceFunction.class); - - private DimComponent nameListComponent; - @FunctionMethod(value = "intelligence", alias = "qingbao") public void intelligence(IMessage message, FunctionContext context, String namespace, String nameListName, String intelligenceFieldName, String asName) { intelligenceInner(message, context, namespace, nameListName, intelligenceFieldName, asName, true); @@ -49,31 +35,31 @@ public void intelligenceLeftJoin(IMessage message, FunctionContext context, Stri } public void intelligenceInner(IMessage message, FunctionContext context, String namespace, String nameListName, String intelligenceFieldName, String asName, boolean isInner) { - String key = FunctionUtils.getValueString(message, context, intelligenceFieldName); - namespace = FunctionUtils.getValueString(message, context, namespace); - nameListName = FunctionUtils.getValueString(message, context, nameListName); - ConfigurableComponent configurableComponent = ComponentCreator.getComponent(namespace, ConfigurableComponent.class); - AbstractIntelligenceCache intelligenceCache = configurableComponent.queryConfigurable(AbstractIntelligenceCache.TYPE, nameListName); - if (intelligenceCache == null) { - throw new RuntimeException("can not query intelligence. the namespace is " + namespace + ", the name is " + nameListName); - } - Map row = intelligenceCache.getRow(key); - if (row != null) { - asName = FunctionUtils.getValueString(message, context, asName); - if (StringUtil.isNotEmpty(asName)) { - asName = asName + "."; - } else { - asName = ""; - } - for (Entry entry : row.entrySet()) { - String elementKey = asName + entry.getKey(); - message.getMessageBody().put(elementKey, entry.getValue()); - } - } else { - if (isInner) { - context.breakExecute(); - } - } +// String key = FunctionUtils.getValueString(message, context, intelligenceFieldName); +// namespace = FunctionUtils.getValueString(message, context, namespace); +// nameListName = FunctionUtils.getValueString(message, context, nameListName); +// ConfigurableComponent configurableComponent = ComponentCreator.getComponent(namespace, ConfigurableComponent.class); +// AbstractIntelligenceCache intelligenceCache = configurableComponent.queryConfigurable(AbstractIntelligenceCache.TYPE, nameListName); +// if (intelligenceCache == null) { +// throw new RuntimeException("can not query intelligence. the namespace is " + namespace + ", the name is " + nameListName); +// } +// Map row = intelligenceCache.getRow(key); +// if (row != null) { +// asName = FunctionUtils.getValueString(message, context, asName); +// if (StringUtil.isNotEmpty(asName)) { +// asName = asName + "."; +// } else { +// asName = ""; +// } +// for (Entry entry : row.entrySet()) { +// String elementKey = asName + entry.getKey(); +// message.getMessageBody().put(elementKey, entry.getValue()); +// } +// } else { +// if (isInner) { +// context.breakExecute(); +// } +// } } } \ No newline at end of file diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/NameListFunction.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/NameListFunction.java deleted file mode 100644 index 08f3bcc2..00000000 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/NameListFunction.java +++ /dev/null @@ -1,202 +0,0 @@ -/* - * 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.dim.function.script; - -import com.alibaba.fastjson.JSONArray; -import com.alibaba.fastjson.JSONObject; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.utils.StringUtil; -import org.apache.rocketmq.streams.dim.DimComponent; -import org.apache.rocketmq.streams.script.annotation.Function; -import org.apache.rocketmq.streams.script.annotation.FunctionMethod; -import org.apache.rocketmq.streams.script.context.FunctionContext; -import org.apache.rocketmq.streams.script.utils.FunctionUtils; - -@Function -public class NameListFunction { - - private static final Log LOG = LogFactory.getLog(NameListFunction.class); - - private DimComponent nameListComponent; - - @FunctionMethod(value = "innerJoin", alias = "inner_join") - public JSONArray innerJoin(IMessage message, FunctionContext context, String namespace, String nameListName, - String expressionStr, String alias, String script, String... fieldNames) { - JSONArray rows = getRows(message, context, namespace, nameListName, expressionStr, alias, script, fieldNames); - if (rows == null || rows.size() == 0) { - context.breakExecute(); - return null; - } - return rows; - } - - @FunctionMethod(value = "leftJoin", alias = "left_join") - public JSONArray leftJoin(IMessage message, FunctionContext context, String namespace, String nameListName, - String expressionStr, String alias, String script, String... fieldNames) { - JSONArray rows = getRows(message, context, namespace, nameListName, expressionStr, alias, script, fieldNames); - if (rows == null || rows.size() == 0) { - return null; - } - return rows; - } - - @FunctionMethod(value = "mark_rows", alias = "namelist_rows") - public String namelist(IMessage message, FunctionContext context, String namespace, String nameListName, - String expressionStr, String... fieldNames) { - JSONArray rows = getRows(message, context, namespace, nameListName, expressionStr, null, null, fieldNames); - if (rows == null || rows.size() == 0) { - return null; - } - return rows.toJSONString(); - } - - @FunctionMethod(value = "mark", alias = "namelist") - public String namelist(IMessage message, FunctionContext context, String namespace, String nameListName, - String expressionStr, String fieldName) { - String tmp = fieldName; - nameListName = FunctionUtils.getValueString(message, context, nameListName); - namespace = FunctionUtils.getValueString(message, context, namespace); - expressionStr = FunctionUtils.getValueString(message, context, expressionStr); - fieldName = FunctionUtils.getValueString(message, context, fieldName); - if (fieldName == null) { - fieldName = tmp; - } - nameListComponent = ComponentCreator.getComponent(namespace, DimComponent.class); - Map row = - nameListComponent.getService().match(nameListName, expressionStr, message.getMessageBody()); - if (row != null) { - Object value = row.get(fieldName); - if (value == null) { - return null; - } - return value.toString(); - } - return null; - } - - @FunctionMethod(value = "mark", alias = "namelist") - public String namelist(IMessage message, FunctionContext context, String namespace, String nameListName, - String expressionStr, String fieldNames, String joinMark) { - nameListName = FunctionUtils.getValueString(message, context, nameListName); - namespace = FunctionUtils.getValueString(message, context, namespace); - expressionStr = FunctionUtils.getValueString(message, context, expressionStr); - fieldNames = FunctionUtils.getValueString(message, context, fieldNames); - joinMark = FunctionUtils.getValueString(message, context, joinMark); - nameListComponent = ComponentCreator.getComponent(namespace, DimComponent.class); - Map row = - nameListComponent.getService().match(nameListName, expressionStr, message.getMessageBody()); - if (row != null) { - String[] fieldNameTem = fieldNames.split(","); - StringBuilder result = new StringBuilder(); - for (int i = 0; i < fieldNameTem.length; i++) { - Object tem = row.get(fieldNameTem[i]); - if (tem != null) { - if (result.length() == 0) { - if (StringUtil.isNotEmpty(tem.toString()) && !("null".equalsIgnoreCase(tem.toString()))) { - result.append(tem); - } - } else { - if (StringUtil.isNotEmpty(tem.toString()) && !("null".equalsIgnoreCase(tem.toString()))) { - result.append(joinMark + tem); - } - } - } - - } - return result.toString(); - } - return null; - } - - @FunctionMethod(value = "in_namelist", alias = "in_namelist") - public Boolean inNameList(IMessage message, FunctionContext context, String namespace, String nameListName, - String expressionStr) { - nameListComponent = ComponentCreator.getComponent(namespace, DimComponent.class); - Map row = - nameListComponent.getService().match(nameListName, expressionStr, message.getMessageBody()); - if (row != null && row.size() > 0) { - return true; - } - return false; - } - - /** - * 根据表达式,从namelist中获取符合条件的数据 - * - * @param message - * @param context - * @param namespace - * @param nameListName - * @param expressionStr (varname,functionName,value)&(varname,functionName,value) - * @param fieldNames 需要返回的字段名 - * @return - */ - protected JSONArray getRows(IMessage message, FunctionContext context, String namespace, String nameListName, - String expressionStr, String alias, String script, String... fieldNames) { - nameListName = FunctionUtils.getValueString(message, context, nameListName); - namespace = FunctionUtils.getValueString(message, context, namespace); - expressionStr = FunctionUtils.getValueString(message, context, expressionStr); - script = FunctionUtils.getValueString(message, context, script); - if (StringUtil.isEmpty(script)) { - script = null; - } - nameListComponent = ComponentCreator.getComponent(namespace, DimComponent.class); - List> rows = - nameListComponent.getService().matchSupportMultiRow(nameListName, expressionStr, message.getMessageBody(), script); - if (rows == null || rows.size() == 0) { - return null; - } - JSONArray jsonArray = new JSONArray(); - for (Map row : rows) { - JSONObject jsonObject = new JSONObject(); - if (fieldNames == null || fieldNames.length == 0) { - if (StringUtil.isEmpty(FunctionUtils.getConstant(alias))) { - jsonObject.putAll(row); - } else { - Iterator> it = row.entrySet().iterator(); - while (it.hasNext()) { - Entry entry = it.next(); - String fieldName = entry.getKey(); - if (alias != null) { - fieldName = alias + "." + fieldName; - } - jsonObject.put(fieldName, entry.getValue()); - } - } - - } else { - for (String fieldName : fieldNames) { - String tmp = fieldName; - if (alias != null) { - fieldName = alias + "." + fieldName; - } - jsonObject.put(fieldName, row.get(tmp)); - } - } - jsonArray.add(jsonObject); - } - return jsonArray; - } - -} \ No newline at end of file diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/index/DimIndex.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/index/DimIndex.java index ef28a246..de0072a8 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/index/DimIndex.java +++ b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/index/DimIndex.java @@ -23,27 +23,26 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.cache.compress.AbstractMemoryTable; import org.apache.rocketmq.streams.common.cache.compress.KVAddress; import org.apache.rocketmq.streams.common.cache.compress.impl.MapAddressListKV; import org.apache.rocketmq.streams.common.datatype.IntDataType; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class DimIndex { - private static final Log LOG = LogFactory.getLog(DimIndex.class); - + private static final Logger LOGGER = LoggerFactory.getLogger(DimIndex.class); + public static IntDataType INTDATATYPE = new IntDataType(); /** * 索引字段名,支持多个索引,每个索引一行,支持组合索引,多个字段用;拼接 name 单索引 name;age 组合索引 */ protected List indexs = new ArrayList<>(); - /** * 如果是非唯一索引,用这个结构存储 每个索引一行,后面的map:key:索引值;value:row id 列表,rowid用字节表示 */ - protected Map mutilIndex = new HashMap<>(); + protected Map mutilIndexForRowIds = new HashMap<>(); public DimIndex(List indexs) { this.indexs = formatIndexs(indexs); @@ -89,7 +88,7 @@ protected List formatIndexs(List indexs) { * @return */ public List getRowIds(String indexName, String indexValue) { - MapAddressListKV indexs = this.mutilIndex.get(indexName); + MapAddressListKV indexs = this.mutilIndexForRowIds.get(indexName); if (indexs == null) { return null; } @@ -109,17 +108,24 @@ public void buildIndex(AbstractMemoryTable tableCompress) { AbstractMemoryTable.RowElement row = it.next(); long rowIndex = row.getRowIndex(); // KVAddress mapAddress = new KVAddress(new ByteArray(NumberUtils.toByte(rowIndex))); - KVAddress mapAddress = KVAddress.createMapAddressFromLongValue(rowIndex); - addRowIndex(row.getRow(), mapAddress, tableCompress.getRowCount()); + addRowIndex(row.getRow(), rowIndex, tableCompress.getRowCount()); // addRowIndex(row.getRow(),row.getRowIndex(),tableCompress.getRowCount()); if ((i % 100000) == 0) { - LOG.info("dim build continue...." + i); + LOGGER.debug("dim build continue...." + i); } i++; } + LOGGER.debug(" finish poll data , the row count is " + i + ". byte is " + tableCompress.getByteCount()); + } - LOG.info(" finish poll data , the row count is " + i + ". byte is " + tableCompress - .getByteCount()); + /** + * 必须是唯一索引, + * + * @param row + */ + public void addRowIndex(Map row, long value, int rowSize) { + KVAddress rowIndex = KVAddress.createMapAddressFromLongValue(value); + addRowIndex(row, rowIndex, rowSize); } /** @@ -135,13 +141,13 @@ public void addRowIndex(Map row, KVAddress rowIndex, int rowSize return; } for (String indexName : indexs) { - MapAddressListKV name2RowIndexs = this.mutilIndex.get(indexName); + MapAddressListKV name2RowIndexs = this.mutilIndexForRowIds.get(indexName); if (name2RowIndexs == null) { synchronized (this) { - name2RowIndexs = this.mutilIndex.get(indexName); + name2RowIndexs = this.mutilIndexForRowIds.get(indexName); if (name2RowIndexs == null) { name2RowIndexs = new MapAddressListKV(rowSize); - this.mutilIndex.put(indexName, name2RowIndexs); + this.mutilIndexForRowIds.put(indexName, name2RowIndexs); } } @@ -165,9 +171,8 @@ protected String createIndexValue(Map row, String[] nameIndexs) for (int i = 0; i < nameIndexs.length; i++) { indexValues[i] = row.get(nameIndexs[i]); } - if (indexValues != null && indexValues.length > 0) { - String indexValue = MapKeyUtil.createKey(indexValues); - return indexValue; + if (indexValues.length > 0) { + return MapKeyUtil.createKey(indexValues); } return null; } @@ -180,10 +185,8 @@ protected String createIndexValue(Map row, String[] nameIndexs) */ protected Map createRow(Map row) { Map cacheValues = new HashMap();//一行数据 - Iterator> iterator = row.entrySet().iterator(); //把数据value从object转化成string - while (iterator.hasNext()) { - Map.Entry entry = iterator.next(); + for (Map.Entry entry : row.entrySet()) { if (entry != null && entry.getValue() != null && entry.getKey() != null) { cacheValues.put(entry.getKey(), entry.getValue().toString()); } @@ -191,8 +194,6 @@ protected Map createRow(Map row) { return cacheValues; } - public static IntDataType INTDATATYPE = new IntDataType(); - public List getIndexs() { return indexs; } diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/index/IndexExecutor.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/index/IndexExecutor.java index 7a4ab7b6..3745f4ba 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/index/IndexExecutor.java +++ b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/index/IndexExecutor.java @@ -38,22 +38,15 @@ */ public class IndexExecutor { + private static IntDataType intDataType = new IntDataType(); private String expressionStr;//表达式 - private boolean isSupport = false;//是否支持索引,比如表达式等值部分无所以,则不能走索引逻辑 - private String namespace; - private String indexNameKey;//索引的名字,多个字段";"分隔 - private List msgNames;// - private Rule rule;//表达式会被编译成rule - private List index; //标准化后的索引name - private Set indexNames = new HashSet<>(); - private Set columnNames; public IndexExecutor(String expressionStr, String namespace, List index, Set columns) { @@ -96,14 +89,15 @@ protected void parse() { this.isSupport = true; List indexExpressions = new ArrayList<>(); - List otherExpressions = new ArrayList<>(); + // List otherExpressions = new ArrayList<>(); + boolean hasOtherExpression = false; if (relationExpression != null) { Map map = new HashMap<>(); for (Expression tmp : expressions) { - map.put(tmp.getConfigureName(), tmp); + map.put(tmp.getName(), tmp); } for (Expression tmp : relationExpressions) { - map.put(tmp.getConfigureName(), tmp); + map.put(tmp.getName(), tmp); } List expressionNames = relationExpression.getValue(); relationExpression.setValue(new ArrayList<>()); @@ -112,8 +106,9 @@ protected void parse() { if (subExpression != null && !RelationExpression.class.isInstance(subExpression) && this.indexNames.contains(subExpression.getValue())) { indexExpressions.add(subExpression); } else { - otherExpressions.add(subExpression); - relationExpression.getValue().add(subExpression.getConfigureName()); + hasOtherExpression = true; + // otherExpressions.add(subExpression); + relationExpression.getValue().add(expressionName); } } @@ -142,7 +137,7 @@ protected void parse() { return; } this.msgNames = createMsgNames(fieldNames, msgNames); - if (otherExpressions.size() == 0) { + if (hasOtherExpression == false) { return; } Rule rule = ExpressionBuilder.createRule("tmp", "tmp", expressionStr); @@ -170,13 +165,14 @@ public boolean isSupport() { return isSupport; } - private static IntDataType intDataType = new IntDataType(); + public boolean isSupportIndex() { + return isSupport; + } public List> match(JSONObject msg, AbstractDim nameList, boolean needAll, String script) { List> rows = new ArrayList<>(); String msgValue = createValue(msg); - List rowIds = nameList.getNameListIndex() == null ? Collections.emptyList() : nameList.getNameListIndex().getRowIds(indexNameKey, msgValue); - ; + List rowIds = getMatchRowIds(msg, nameList); if (rowIds == null) { return null; } @@ -193,6 +189,12 @@ public List> match(JSONObject msg, AbstractDim nameList, boo return rows; } + public List getMatchRowIds(JSONObject msg, AbstractDim nameList) { + String msgValue = createValue(msg); + List rowIds = nameList.getNameListIndex() == null ? Collections.emptyList() : nameList.getNameListIndex().getRowIds(indexNameKey, msgValue); + return rowIds; + } + /** * 按顺序创建msg的key * @@ -206,4 +208,8 @@ private String createValue(JSONObject msg) { } return MapKeyUtil.createKey(value); } + + public List getMsgNames() { + return msgNames; + } } diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/AbstractIntelligenceCache.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/AbstractIntelligenceCache.java index 4f0f8612..1b2798bd 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/AbstractIntelligenceCache.java +++ b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/AbstractIntelligenceCache.java @@ -27,39 +27,30 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledThreadPoolExecutor; -import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.lang3.StringUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.cache.compress.impl.IntValueKV; -import org.apache.rocketmq.streams.common.channel.sink.ISink; -import org.apache.rocketmq.streams.common.component.ComponentCreator; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.configurable.annotation.ConfigurableReference; import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; import org.apache.rocketmq.streams.common.dboperator.IDBDriver; +import org.apache.rocketmq.streams.common.threadpool.ScheduleFactory; import org.apache.rocketmq.streams.common.threadpool.ThreadPoolFactory; import org.apache.rocketmq.streams.common.utils.NumberUtils; import org.apache.rocketmq.streams.common.utils.SQLUtil; import org.apache.rocketmq.streams.db.driver.JDBCDriver; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public abstract class AbstractIntelligenceCache extends BasedConfigurable implements - IAfterConfigurableRefreshListener { - - private static final Log LOG = LogFactory.getLog(AbstractIntelligenceCache.class); +public abstract class AbstractIntelligenceCache extends BasedConfigurable { public static final String TYPE = "intelligence"; - protected static final int FILE_MAX_LINE = 50000;//每个文件最大行数 - + private static final Logger LOGGER = LoggerFactory.getLogger(AbstractIntelligenceCache.class); + protected static ExecutorService executorService; protected transient IntValueKV intValueKV = new IntValueKV(0) { @Override public Integer get(String key) { @@ -71,37 +62,26 @@ public void put(String key, Integer value) { } }; - protected String idFieldName = "id";//必须有id字段 - protected int batchSize = 3000; - @ENVDependence - protected Long pollingTimeMintue = 30L; - - protected String datasourceName;//情报对应的存储 - - protected transient IDBDriver outputDataSource; - - protected static ExecutorService executorService; - - protected transient ScheduledExecutorService scheduledExecutorService; + protected Long pollingTimeMinute = 30L; + @ConfigurableReference protected IDBDriver outputDataSource; + protected transient AtomicBoolean hasInit = new AtomicBoolean(false); public AbstractIntelligenceCache() { setType(TYPE); - executorService = ThreadPoolFactory.createThreadPool(20); - scheduledExecutorService = new ScheduledThreadPoolExecutor(3); + executorService = ThreadPoolFactory.createFixedThreadPool(20, AbstractIntelligenceCache.class.getName() + "-intelligence"); } public IntValueKV startLoadData(String sql, IDBDriver resource) { try { String statisticalSQL = sql; int startIndex = sql.toLowerCase().indexOf("from"); - statisticalSQL = "select count(1) as c, min(" + idFieldName + ") as min, max(" + idFieldName + ") as max " - + sql.substring(startIndex); + statisticalSQL = "select count(1) as c, min(" + idFieldName + ") as min, max(" + idFieldName + ") as max " + sql.substring(startIndex); List> rows = resource.queryForList(statisticalSQL); Map row = rows.get(0); - int count = Integer.valueOf(row.get("c").toString()); + int count = Integer.parseInt(row.get("c").toString()); IntValueKV intValueKV = new IntValueKV(count); //int maxBatch=count/maxSyncCount;//每1w条数据,一个并发。如果数据量比较大,为了提高性能,并行执行 if (count == 0) { @@ -117,37 +97,31 @@ public void put(String key, Integer value) { } }; } - long min = Long.valueOf(row.get("min").toString()); - long max = Long.valueOf(row.get("max").toString()); + long min = Long.parseLong(row.get("min").toString()); + long max = Long.parseLong(row.get("max").toString()); int maxSyncCount = count / FILE_MAX_LINE + 1; long step = (max - min + 1) / maxSyncCount; CountDownLatch countDownLatch = new CountDownLatch(maxSyncCount + 1); AtomicInteger finishedCount = new AtomicInteger(0); String taskSQL = null; - if (sql.indexOf(" where ") != -1) { - taskSQL = sql + " and " + idFieldName + ">#{startIndex} and " + idFieldName + "<=#{endIndex} order by " - + idFieldName + " limit " + batchSize; + if (sql.contains(" where ")) { + taskSQL = sql + " and " + idFieldName + ">#{startIndex} and " + idFieldName + "<=#{endIndex} order by " + idFieldName + " limit " + batchSize; } else { - taskSQL = sql + " where " + idFieldName + ">#{startIndex} and " + idFieldName - + "<=#{endIndex} order by " + idFieldName + " limit " + batchSize; + taskSQL = sql + " where " + idFieldName + ">#{startIndex} and " + idFieldName + "<=#{endIndex} order by " + idFieldName + " limit " + batchSize; } int i = 0; for (; i < maxSyncCount; i++) { - FetchDataTask fetchDataTask = new FetchDataTask(taskSQL, (min - 1) + step * i, - (min - 1) + step * (i + 1), countDownLatch, finishedCount, resource, i, intValueKV, this, count); + FetchDataTask fetchDataTask = new FetchDataTask(taskSQL, (min - 1) + step * i, (min - 1) + step * (i + 1), countDownLatch, finishedCount, resource, i, intValueKV, this, count); executorService.execute(fetchDataTask); } - FetchDataTask fetchDataTask = new FetchDataTask(taskSQL, (min - 1) + step * i, (min - 1) + step * (i + 1), - countDownLatch, finishedCount, resource, i, intValueKV, this, count); + FetchDataTask fetchDataTask = new FetchDataTask(taskSQL, (min - 1) + step * i, (min - 1) + step * (i + 1), countDownLatch, finishedCount, resource, i, intValueKV, this, count); executorService.execute(fetchDataTask); - countDownLatch.await(); - - LOG.info(getClass().getSimpleName() + " load data finish, load data line size is " + intValueKV.getSize()); + LOGGER.info("[{}] {} load data finish, load data line size is {}", getName(), getClass().getSimpleName(), intValueKV.getSize()); return intValueKV; } catch (Exception e) { - LOG.error("failed loading intelligence data!", e); + LOGGER.error("[{}] failed loading intelligence data!", getName(), e); return new IntValueKV(0) { @Override public Integer get(String key) { @@ -162,13 +136,6 @@ public void put(String key, Integer value) { } } - protected transient AtomicBoolean hasInit = new AtomicBoolean(false); - - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - this.outputDataSource = configurableService.queryConfigurable(ISink.TYPE, datasourceName); - } - public void startIntelligence() { boolean success = dbInit(); if (success) { @@ -197,12 +164,12 @@ public void startIntelligenceInner() { String sql = getSQL(); if (hasInit.compareAndSet(false, true)) { this.intValueKV = startLoadData(sql, outputDataSource); - scheduledExecutorService.scheduleWithFixedDelay(new Runnable() { + ScheduleFactory.getInstance().execute(getNameSpace() + "-" + getName() + "-intelligence_schedule", new Runnable() { @Override public void run() { intValueKV = startLoadData(sql, outputDataSource); } - }, pollingTimeMintue, pollingTimeMintue, TimeUnit.MINUTES); + }, pollingTimeMinute, pollingTimeMinute, TimeUnit.MINUTES); } } @@ -210,100 +177,26 @@ public void run() { /** * 查询情报需要的sql - * - * @return */ protected abstract String getSQL(); /** * 情报中的 情报字段名 - * - * @return */ public abstract String getKeyName(); /** * 情报对应的表名 - * - * @return */ public abstract String getTableName(); - protected class FetchDataTask implements Runnable { - IntValueKV intValueKV; - long startIndex; - long endIndex; - String sql; - CountDownLatch countDownLatch; - int index; - IDBDriver resource; - AtomicInteger finishedCount;//完成了多少条 - AbstractIntelligenceCache cache; - int totalSize;//一共有多少条数据 - - public FetchDataTask(String sql, long startIndex, long endIndex, CountDownLatch countDownLatch, - AtomicInteger finishedCount, IDBDriver resource, int i, IntValueKV intValueKV, - AbstractIntelligenceCache cache, int totalSize) { - this.startIndex = startIndex; - this.endIndex = endIndex; - this.countDownLatch = countDownLatch; - this.sql = sql; - this.finishedCount = finishedCount; - this.resource = resource; - this.index = i; - this.intValueKV = intValueKV; - this.cache = cache; - this.totalSize = totalSize; - } - - @Override - public void run() { - long currentIndex = startIndex; - JSONObject msg = new JSONObject(); - msg.put("endIndex", endIndex); - while (true) { - try { - - msg.put("startIndex", currentIndex); - - String sql = SQLUtil.parseIbatisSQL(msg, this.sql); - List> rows = resource.queryForList(sql); - if (rows == null || rows.size() == 0) { - break; - } - currentIndex = Long.valueOf(rows.get(rows.size() - 1).get(idFieldName).toString()); - - int size = rows.size(); - int count = finishedCount.addAndGet(size); - double progress = (double) count / (double) totalSize; - progress = progress * 100; - System.out.println(cache.getClass().getSimpleName() + ", finished count is " + count + " the total count is " + totalSize + ", the progress is " + String.format("%.2f", progress) + "%"); - if (size < batchSize) { - if (size > 0) { - doProccRows(intValueKV, rows, index); - } - break; - } - doProccRows(intValueKV, rows, index); - } catch (Exception e) { - throw new RuntimeException("put data error ", e); - } - } - - countDownLatch.countDown(); - } - } - public boolean dbInit() { try { int successCode = 200; - String region = ComponentCreator.getProperties().getProperty(ConfigureFileKey.INTELLIGENCE_REGION); - String ak = ComponentCreator.getProperties().getProperty( - ConfigureFileKey.INTELLIGENCE_AK); - String sk = ComponentCreator.getProperties().getProperty( - ConfigureFileKey.INTELLIGENCE_SK); - String endpoint = ComponentCreator.getProperties().getProperty( - ConfigureFileKey.INTELLIGENCE_TIP_DB_ENDPOINT); + String region = getConfiguration().getProperty(ConfigurationKey.INTELLIGENCE_REGION); + String ak = getConfiguration().getProperty(ConfigurationKey.INTELLIGENCE_AK); + String sk = getConfiguration().getProperty(ConfigurationKey.INTELLIGENCE_SK); + String endpoint = getConfiguration().getProperty(ConfigurationKey.INTELLIGENCE_TIP_DB_ENDPOINT); if (StringUtils.isNotBlank(region) && StringUtils.isNotBlank(ak) && StringUtils.isNotBlank(sk) && StringUtils.isNotBlank(endpoint)) { DefaultProfile profile = DefaultProfile.getProfile(region, ak, sk); IAcsClient client = new DefaultAcsClient(profile); @@ -319,8 +212,7 @@ public boolean dbInit() { JSONObject obj = JSON.parseObject(content); JSONObject dbInfo = obj.getJSONObject("dBInfo"); if (dbInfo != null) { - String dbUrl = "jdbc:mysql://" + dbInfo.getString("dbConnection") + ":" + dbInfo.getInteger( - "port") + "/" + dbInfo.getString("dBName"); + String dbUrl = "jdbc:mysql://" + dbInfo.getString("dbConnection") + ":" + dbInfo.getInteger("port") + "/" + dbInfo.getString("dBName"); String dbUserName = dbInfo.getString("userName"); String dbPassword = dbInfo.getString("passWord"); JDBCDriver dataSource = (JDBCDriver) this.outputDataSource; @@ -329,25 +221,22 @@ public boolean dbInit() { dataSource.setUserName(dbUserName); dataSource.setHasInit(false); dataSource.init(); - LOG.debug("succeed in getting db information from tip service!"); + LOGGER.debug("[{}] succeed in getting db information from tip service!", getName()); return true; } } } } - LOG.error("failed in getting db information from tip service!"); + LOGGER.error("[{}] failed in getting db information from tip service!", getName()); return false; } catch (Exception e) { - LOG.error("failed in getting db information from tip service!", e); + LOGGER.error("[{}] failed in getting db information from tip service!", getName(), e); return false; } } /** * 把存储0/1字符串的值,转化成bit - * - * @param values - * @return */ protected int createInt(List values) { return NumberUtils.createBitMapInt(values); @@ -355,10 +244,6 @@ protected int createInt(List values) { /** * 获取某位的值,如果是1,返回字符串1,否则返回null - * - * @param num - * @param i - * @return */ protected String getNumBitValue(int num, int i) { boolean exist = NumberUtils.getNumFromBitMapInt(num, i); @@ -386,19 +271,82 @@ public void setBatchSize(int batchSize) { this.batchSize = batchSize; } - public Long getPollingTimeMintue() { - return pollingTimeMintue; + public Long getPollingTimeMinute() { + return pollingTimeMinute; + } + + public void setPollingTimeMinute(Long pollingTimeMinute) { + this.pollingTimeMinute = pollingTimeMinute; } - public void setPollingTimeMintue(Long pollingTimeMintue) { - this.pollingTimeMintue = pollingTimeMintue; + public IDBDriver getOutputDataSource() { + return outputDataSource; } - public String getDatasourceName() { - return datasourceName; + public void setOutputDataSource(IDBDriver outputDataSource) { + this.outputDataSource = outputDataSource; } - public void setDatasourceName(String datasourceName) { - this.datasourceName = datasourceName; + protected class FetchDataTask implements Runnable { + IntValueKV intValueKV; + long startIndex; + long endIndex; + String sql; + CountDownLatch countDownLatch; + int index; + IDBDriver resource; + AtomicInteger finishedCount;//完成了多少条 + AbstractIntelligenceCache cache; + int totalSize;//一共有多少条数据 + + public FetchDataTask(String sql, long startIndex, long endIndex, CountDownLatch countDownLatch, + AtomicInteger finishedCount, IDBDriver resource, int i, IntValueKV intValueKV, + AbstractIntelligenceCache cache, int totalSize) { + this.startIndex = startIndex; + this.endIndex = endIndex; + this.countDownLatch = countDownLatch; + this.sql = sql; + this.finishedCount = finishedCount; + this.resource = resource; + this.index = i; + this.intValueKV = intValueKV; + this.cache = cache; + this.totalSize = totalSize; + } + + @Override + public void run() { + long currentIndex = startIndex; + JSONObject msg = new JSONObject(); + msg.put("endIndex", endIndex); + while (true) { + try { + msg.put("startIndex", currentIndex); + String sql = SQLUtil.parseIbatisSQL(msg, this.sql); + List> rows = resource.queryForList(sql); + if (rows == null || rows.size() == 0) { + break; + } + currentIndex = Long.parseLong(rows.get(rows.size() - 1).get(idFieldName).toString()); + + int size = rows.size(); + int count = finishedCount.addAndGet(size); + double progress = (double) count / (double) totalSize; + progress = progress * 100; + LOGGER.info("[{}] {}, finished count is {} the total count is {}, the progress is {}%", getName(), cache.getClass().getSimpleName(), count, totalSize, String.format("%.2f", progress)); + if (size < batchSize) { + if (size > 0) { + doProccRows(intValueKV, rows, index); + } + break; + } + doProccRows(intValueKV, rows, index); + } catch (Exception e) { + throw new RuntimeException("put data error ", e); + } + } + + countDownLatch.countDown(); + } } } diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/AccountIntelligenceCache.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/AccountIntelligenceCache.java index 1cbda460..3826cf45 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/AccountIntelligenceCache.java +++ b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/AccountIntelligenceCache.java @@ -19,17 +19,12 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.cache.compress.impl.IntValueKV; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; /** * table: ads_yunsec_abnormal_account */ -public class AccountIntelligenceCache extends AbstractIntelligenceCache implements IAfterConfigurableRefreshListener { - - private static final Log LOG = LogFactory.getLog(AccountIntelligenceCache.class); +public class AccountIntelligenceCache extends AbstractIntelligenceCache { /** * 情报域名 @@ -66,7 +61,7 @@ public String getTableName() { @Override protected void doProccRows(IntValueKV intValueKV, List> rows, int index) { rows.forEach(row -> { - String account = (String)row.get(keyName); + String account = (String) row.get(keyName); if (account != null) { intValueKV.put(account, 1); } diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/DomainIntelligenceCache.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/DomainIntelligenceCache.java index 1926a676..25535b08 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/DomainIntelligenceCache.java +++ b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/DomainIntelligenceCache.java @@ -20,13 +20,12 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.cache.compress.impl.IntValueKV; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class DomainIntelligenceCache extends AbstractIntelligenceCache implements IAfterConfigurableRefreshListener { - private static final Log LOG = LogFactory.getLog(DomainIntelligenceCache.class); +public class DomainIntelligenceCache extends AbstractIntelligenceCache { + private static final Logger LOGGER = LoggerFactory.getLogger(DomainIntelligenceCache.class); protected transient String keyName = "domain"; @Override @@ -61,16 +60,16 @@ public String getTableName() { @Override protected void doProccRows(IntValueKV intValueKV, List> rows, int index) { for (Map row : rows) { - String ip = (String)row.get(keyName); + String ip = (String) row.get(keyName); if (ip == null) { - LOG.warn("load Intelligence exception ,the ip is null"); + LOGGER.warn("load Intelligence exception ,the ip is null"); continue; } List values = new ArrayList<>(); - values.add((String)row.get("is_malicious_source")); - values.add((String)row.get("is_phishing")); - values.add((String)row.get("is_c2")); - values.add((String)row.get("is_mining_pool")); + values.add((String) row.get("is_malicious_source")); + values.add((String) row.get("is_phishing")); + values.add((String) row.get("is_c2")); + values.add((String) row.get("is_mining_pool")); int value = createInt(values); synchronized (this) { intValueKV.put(ip, value); diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/IPIntelligenceCache.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/IPIntelligenceCache.java index 61ba2f32..c18030b4 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/IPIntelligenceCache.java +++ b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/IPIntelligenceCache.java @@ -20,18 +20,24 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.cache.compress.impl.IntValueKV; import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; import org.apache.rocketmq.streams.common.dboperator.IDBDriver; import org.apache.rocketmq.streams.db.driver.DriverBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class IPIntelligenceCache extends AbstractIntelligenceCache implements IAfterConfigurableRefreshListener { - private static final Log LOG = LogFactory.getLog(IPIntelligenceCache.class); +public class IPIntelligenceCache extends AbstractIntelligenceCache { + private static final Logger LOGGER = LoggerFactory.getLogger(IPIntelligenceCache.class); protected transient String keyName = "ip"; + public static void main(String[] args) { + ComponentCreator.setProperties( + "siem.properties"); + IPIntelligenceCache ipIntelligenceCache = new IPIntelligenceCache(); + IDBDriver outputDataSource = DriverBuilder.createDriver(); + ipIntelligenceCache.startLoadData(ipIntelligenceCache.getSQL(), outputDataSource); + } @Override protected String getSQL() { @@ -74,34 +80,26 @@ public Map getRow(String ip) { @Override protected void doProccRows(IntValueKV intValueKV, List> rows, int index) { for (Map row : rows) { - String ip = (String)row.get(keyName); + String ip = (String) row.get(keyName); if (ip == null) { - LOG.warn("load Intelligence exception ,the ip is null"); + LOGGER.warn("load Intelligence exception ,the ip is null"); continue; } List values = new ArrayList<>(); - values.add((String)row.get("is_web_attack")); - values.add((String)row.get("is_tor")); - values.add((String)row.get("is_proxy")); - values.add((String)row.get("is_nat")); - values.add((String)row.get("is_mining_pool")); - values.add((String)row.get("is_c2")); - values.add((String)row.get("is_malicious_source")); - values.add((String)row.get("is_3rd")); - values.add((String)row.get("is_idc")); - values.add((String)row.get("is_malicious_login")); + values.add((String) row.get("is_web_attack")); + values.add((String) row.get("is_tor")); + values.add((String) row.get("is_proxy")); + values.add((String) row.get("is_nat")); + values.add((String) row.get("is_mining_pool")); + values.add((String) row.get("is_c2")); + values.add((String) row.get("is_malicious_source")); + values.add((String) row.get("is_3rd")); + values.add((String) row.get("is_idc")); + values.add((String) row.get("is_malicious_login")); int value = createInt(values); synchronized (this) { intValueKV.put(ip, value); } } } - - public static void main(String[] args) { - ComponentCreator.setProperties( - "siem.properties"); - IPIntelligenceCache ipIntelligenceCache = new IPIntelligenceCache(); - IDBDriver outputDataSource = DriverBuilder.createDriver(); - ipIntelligenceCache.startLoadData(ipIntelligenceCache.getSQL(), outputDataSource); - } } diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/URLIntelligenceCache.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/URLIntelligenceCache.java index b1263dae..215a590b 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/URLIntelligenceCache.java +++ b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/URLIntelligenceCache.java @@ -20,14 +20,13 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.cache.compress.impl.IntValueKV; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class URLIntelligenceCache extends AbstractIntelligenceCache implements IAfterConfigurableRefreshListener { +public class URLIntelligenceCache extends AbstractIntelligenceCache { - private static final Log LOG = LogFactory.getLog(URLIntelligenceCache.class); + private static final Logger LOGGER = LoggerFactory.getLogger(URLIntelligenceCache.class); protected transient String keyName = "url"; @@ -61,13 +60,13 @@ public String getTableName() { @Override protected void doProccRows(IntValueKV intValueKV, List> rows, int index) { for (Map row : rows) { - String ip = (String)row.get(keyName); + String ip = (String) row.get(keyName); if (ip == null) { - LOG.warn("load Intelligence exception ,the ip is null"); + LOGGER.warn("load Intelligence exception ,the ip is null"); continue; } List values = new ArrayList<>(); - values.add((String)row.get("is_malicious_source")); + values.add((String) row.get("is_malicious_source")); int value = createInt(values); synchronized (this) { intValueKV.put(ip, value); diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/AbstractDim.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/AbstractDim.java index fb763950..d4858263 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/AbstractDim.java +++ b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/AbstractDim.java @@ -17,30 +17,24 @@ package org.apache.rocketmq.streams.dim.model; import com.alibaba.fastjson.JSONObject; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Date; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.cache.ByteArrayMemoryTable; -import org.apache.rocketmq.streams.common.cache.MappedByteBufferTable; import org.apache.rocketmq.streams.common.cache.compress.AbstractMemoryTable; import org.apache.rocketmq.streams.common.cache.softreference.ICache; import org.apache.rocketmq.streams.common.cache.softreference.impl.SoftReferenceCache; -import org.apache.rocketmq.streams.common.component.ComponentCreator; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.interfaces.IDim; +import org.apache.rocketmq.streams.common.threadpool.ScheduleFactory; import org.apache.rocketmq.streams.common.utils.DataTypeUtil; +import org.apache.rocketmq.streams.common.utils.IdUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.dim.index.DimIndex; @@ -51,175 +45,66 @@ import org.apache.rocketmq.streams.filter.operator.expression.Expression; import org.apache.rocketmq.streams.filter.operator.expression.RelationExpression; import org.apache.rocketmq.streams.script.ScriptComponent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * 这个结构代表一张表 存放表的全部数据和索引 */ -public abstract class AbstractDim extends BasedConfigurable { - - private static final Log LOG = LogFactory.getLog(AbstractDim.class); - - public static final String TYPE = "nameList"; +public abstract class AbstractDim extends BasedConfigurable implements IDim { + private static final Logger LOGGER = LoggerFactory.getLogger(AbstractDim.class); /** - * 同步数据的事件间隔,单位是分钟 + * 软引用缓存,最大可能保存索引执行器,避免频繁创建,带来额外开销 同时会保护内存不被写爆,当内存不足时自动回收内存 */ - protected Long pollingTimeMinute = 60L; - + private static final ICache cache = new SoftReferenceCache<>(); + /** + * 同步数据的事件间隔,单位是秒 + */ + protected Long pollingTimeSeconds = 60L; /** * 支持多组索引,如果一个索引是组合索引,需要拼接成一个string,用;分割 建立索引后,会创建索引的数据结构,类似Map,可以快速定位,无索引会全表扫描,不建议使用 如有两组索引:1.name 2. ip;address */ protected List indexs = new ArrayList<>(); - protected boolean isLarge = false;//if isLarge=true use MapperByteBufferTable 内存结构 - protected String filePath; /** * 把表数据转化成二进制存储在CompressTable中 */ protected transient volatile AbstractMemoryTable dataCache; - /** * 建立名单的时候,可以指定多组索引,索引的值当作key,row在datacache的index当作value,可以快速匹配索引对应的row key:索引的值 value:row在dataCache的index当作value,可以快速匹配索引对应的row */ protected transient DimIndex nameListIndex; protected transient Set columnNames; - //定时加载表数据到内存 - protected transient ScheduledExecutorService executorService; - - public AbstractDim() { - this.setType(TYPE); - } //protected String index;//只是做标记,为了是简化indexs的赋值 - public String addIndex(String... fieldNames) { - return addIndex(this.indexs, fieldNames); - } - - @Override - protected boolean initConfigurable() { - boolean success = super.initConfigurable(); - if (Boolean.TRUE.equals(Boolean.valueOf(ComponentCreator.getProperties().getProperty(ConfigureFileKey.DIPPER_RUNNING_STATUS, ConfigureFileKey.DIPPER_RUNNING_STATUS_DEFAULT)))) { - loadNameList(); - executorService = new ScheduledThreadPoolExecutor(3); - executorService.scheduleWithFixedDelay(new Runnable() { - @Override - public void run() { - loadNameList(); - } - }, pollingTimeMinute, pollingTimeMinute, TimeUnit.MINUTES); - } - - return success; - } - - /** - * 加载维表数据 创建索引 - */ - protected void loadNameList() { - try { - LOG.info(getConfigureName() + " begin polling data"); - //全表数据 - AbstractMemoryTable dataCacheVar = loadData(); - this.dataCache = dataCacheVar; - this.nameListIndex = buildIndex(dataCacheVar); - this.columnNames = this.dataCache.getCloumnName2Index().keySet(); - } catch (Exception e) { - LOG.error("Load configurables error:" + e.getMessage(), e); - } - } - - /** - * 给维表生成索引数据结构 - * - * @param dataCacheVar 维表 - * @return - */ - protected DimIndex buildIndex(AbstractMemoryTable dataCacheVar) { - DimIndex dimIndex = new DimIndex(this.indexs); - dimIndex.buildIndex(dataCacheVar); - return dimIndex; - } - - /** - * 软引用缓存,最大可能保存索引执行器,避免频繁创建,带来额外开销 同时会保护内存不被写爆,当内存不足时自动回收内存 - */ - private static ICache cache = new SoftReferenceCache<>(); - - /** - * 先找索引,如果有索引,通过索引匹配。如果没有,全表扫表. - * - * @param expressionStr 表达式 - * @param msg 消息 - * @return 只返回匹配的第一行 - */ - public Map matchExpression(String expressionStr, JSONObject msg) { - List> rows = matchExpression(expressionStr, msg, true, null); - if (rows != null && rows.size() > 0) { - return rows.get(0); - } - return null; - } - - /** - * 先找索引,如果有索引,通过索引匹配。如果没有,全表扫表 - * - * @param expressionStr 表达式 - * @param msg 消息 - * @return 返回全部匹配的行 - */ - public List> matchExpression(String expressionStr, JSONObject msg, boolean needAll, - String script) { - IndexExecutor indexNamelistExecutor = cache.get(expressionStr); - if (indexNamelistExecutor == null) { - indexNamelistExecutor = new IndexExecutor(expressionStr, getNameSpace(), this.indexs, dataCache.getCloumnName2DatatType().keySet()); - cache.put(expressionStr, indexNamelistExecutor); - } - if (indexNamelistExecutor.isSupport()) { - return indexNamelistExecutor.match(msg, this, needAll, script); - } else { - return matchExpressionByLoop(dataCache.rowIterator(), expressionStr, msg, needAll, script, columnNames); - } - } - - /** - * 全表扫描,做表达式匹配,返回全部匹配结果 - * - * @param expressionStr - * @param msg - * @param needAll - * @return - */ - protected List> matchExpressionByLoop(String expressionStr, JSONObject msg, boolean needAll) { - AbstractMemoryTable dataCache = this.dataCache; - List> rows = matchExpressionByLoop(dataCache.rowIterator(), expressionStr, msg, needAll, null, columnNames); - return rows; + public AbstractDim() { + this.setType(IDim.TYPE); } /** * 全表扫描,做表达式匹配,返回全部匹配结果。join中有使用 * - * @param expressionStr - * @param msg - * @param needAll - * @return + * @param expressionStr expression string + * @param msg msg + * @param needAll need all + * @return list map */ - public static List> matchExpressionByLoop(Iterator> it, - String expressionStr, JSONObject msg, boolean needAll) { + public static List> matchExpressionByLoop(Iterator> it, String expressionStr, JSONObject msg, boolean needAll) { return matchExpressionByLoop(it, expressionStr, msg, needAll, null, new HashSet<>()); } /** * 全表扫描,做表达式匹配,返回全部匹配结果。join中有使用 * - * @param expressionStr - * @param msg - * @param needAll - * @return + * @param expressionStr expression string + * @param msg msg + * @param needAll need all + * @return list map */ - public static List> matchExpressionByLoop(Iterator> it, - String expressionStr, JSONObject msg, boolean needAll, String script, Set colunmNames) { + public static List> matchExpressionByLoop(Iterator> it, String expressionStr, JSONObject msg, boolean needAll, String script, Set colunmNames) { List> rows = new ArrayList<>(); Rule ruleTemplete = ExpressionBuilder.createRule("tmp", "tmpRule", expressionStr); while (it.hasNext()) { @@ -238,26 +123,24 @@ public static List> matchExpressionByLoop(Iterator isMatch(Rule ruleTemplete, Map dimRow, JSONObject msgRow, - String script, Set colunmNames) { + public static Map isMatch(Rule ruleTemplate, Map dimRow, JSONObject msgRow, String script, Set columnNames) { Map oldRow = dimRow; Map newRow = executeScript(oldRow, script); - if (ruleTemplete == null) { + if (ruleTemplate == null) { return newRow; } - Rule rule = ruleTemplete.copy(); + Rule rule = ruleTemplate.copy(); Map expressionMap = new HashMap<>(); String dimAsName = null; ; for (Expression expression : rule.getExpressionMap().values()) { - expressionMap.put(expression.getConfigureName(), expression); + expressionMap.put(expression.getName(), expression); if (expression instanceof RelationExpression) { continue; } @@ -268,7 +151,7 @@ public static Map isMatch(Rule ruleTemplete, Map if (value != null) { Expression e = expression.copy(); e.setValue(value.toString()); - expressionMap.put(e.getConfigureName(), e); + expressionMap.put(e.getName(), e); } } if (expression.getVarName().contains(".")) { @@ -276,7 +159,7 @@ public static Map isMatch(Rule ruleTemplete, Map if (values.length == 2) { String asName = values[0]; String varName = values[1]; - if (colunmNames.contains(varName)) { + if (columnNames.contains(varName)) { dimAsName = asName; } } @@ -299,8 +182,105 @@ public static Map isMatch(Rule ruleTemplete, Map return null; } - public static interface IDimField { - boolean isDimField(Object fieldName); + protected static Map executeScript(Map oldRow, String script) { + if (script == null) { + return oldRow; + } + ScriptComponent scriptComponent = ScriptComponent.getInstance(); + JSONObject msg = new JSONObject(); + msg.putAll(oldRow); + scriptComponent.getService().executeScript(msg, script); + return msg; + } + + @Override public String addIndex(String... fieldNames) { + return addIndex(this.indexs, fieldNames); + } + + @Override protected boolean initConfigurable() { + return super.initConfigurable(); + } + + @Override public void startLoadDimData() { + loadNameList(); + ScheduleFactory.getInstance().execute(getNameSpace() + "-" + getName() + "-dim_schedule", this::loadNameList, pollingTimeSeconds, pollingTimeSeconds, TimeUnit.SECONDS); + } + + /** + * 加载维表数据 创建索引 + */ + protected void loadNameList() { + try { + LOGGER.info("[{}][{}] Dim_Begin_Polling_Data", IdUtil.instanceId(), getName()); + //全表数据 + AbstractMemoryTable dataCacheVar = loadData(); + this.dataCache = dataCacheVar; + this.nameListIndex = buildIndex(dataCacheVar); + this.columnNames = this.dataCache.getCloumnName2Index().keySet(); + } catch (Exception e) { + LOGGER.error("[{}][{}] Dim_Load_Configurables_Error", IdUtil.instanceId(), getName(), e); + } + } + + /** + * 给维表生成索引数据结构 + * + * @param dataCacheVar 维表 + * @return dimIndex + */ + protected DimIndex buildIndex(AbstractMemoryTable dataCacheVar) { + DimIndex dimIndex = new DimIndex(this.indexs); + dimIndex.buildIndex(dataCacheVar); + return dimIndex; + } + + @Override + public List> matchExpression(String msgFieldName, String dimFieldName, JSONObject msg) { + return matchExpression("(" + msgFieldName + "," + dimFieldName + ")", msg, true, null); + } + + /** + * 先找索引,如果有索引,通过索引匹配。如果没有,全表扫表. + * + * @param expressionStr 表达式 + * @param msg 消息 + * @return 只返回匹配的第一行 + */ + @Override public Map matchExpression(String expressionStr, JSONObject msg) { + List> rows = matchExpression(expressionStr, msg, true, null); + if (rows != null && rows.size() > 0) { + return rows.get(0); + } + return null; + } + + /** + * 先找索引,如果有索引,通过索引匹配。如果没有,全表扫表 + * + * @param expressionStr 表达式 + * @param msg 消息 + * @return 返回全部匹配的行 + */ + @Override public List> matchExpression(String expressionStr, JSONObject msg, boolean needAll, String script) { + IndexExecutor indexNamelistExecutor = getOrCreateIndexExecutor(expressionStr); + if (indexNamelistExecutor.isSupportIndex()) { + return matchExpressionByIndex(indexNamelistExecutor, msg, needAll, script); + } else { + return matchExpressionByLoop(dataCache.rowIterator(), expressionStr, msg, needAll, script, columnNames); + } + } + + /** + * 全表扫描,做表达式匹配,返回全部匹配结果 + * + * @param expressionStr expression string + * @param msg msg + * @param needAll need all + * @return list map + */ + protected List> matchExpressionByLoop(String expressionStr, JSONObject msg, boolean needAll) { + AbstractMemoryTable dataCache = this.dataCache; + return matchExpressionByLoop(dataCache.rowIterator(), expressionStr, msg, needAll, null, columnNames); } /** @@ -324,20 +304,20 @@ public void createIndexByJoinCondition(String expressionStr, IDimField dimField) if (relationExpression != null) { Map map = new HashMap<>(); for (Expression tmp : expressions) { - map.put(tmp.getConfigureName(), tmp); + map.put(tmp.getName(), tmp); } for (Expression tmp : relationExpressions) { - map.put(tmp.getConfigureName(), tmp); + map.put(tmp.getName(), tmp); } List expressionNames = relationExpression.getValue(); relationExpression.setValue(new ArrayList<>()); for (String expressionName : expressionNames) { Expression subExpression = map.get(expressionName); - if (subExpression != null && !RelationExpression.class.isInstance(subExpression) && dimField.isDimField(subExpression.getValue())) { + if (subExpression != null && !(subExpression instanceof RelationExpression) && dimField.isDimField(subExpression.getValue())) { indexExpressions.add(subExpression); } else { otherExpressions.add(subExpression); - relationExpression.getValue().add(subExpression.getConfigureName()); + relationExpression.getValue().add(subExpression.getName()); } } @@ -375,44 +355,29 @@ public void createIndexByJoinCondition(String expressionStr, IDimField dimField) } } - protected static Map executeScript(Map oldRow, String script) { - if (script == null) { - return oldRow; - } - ScriptComponent scriptComponent = ScriptComponent.getInstance(); - JSONObject msg = new JSONObject(); - msg.putAll(oldRow); - scriptComponent.getService().executeScript(msg, script); - return msg; - } - protected AbstractMemoryTable loadData() { AbstractMemoryTable memoryTable = null; - if (!isLarge) { - LOG.info(String.format("init ByteArrayMemoryTable.")); - memoryTable = new ByteArrayMemoryTable(); - loadData2Memory(memoryTable); - } else { - LOG.info(String.format("init MappedByteBufferTable.")); -// memoryTable = new MappedByteBufferTable(); -// loadData2Memory(memoryTable); - Date date = new Date(); - try { - memoryTable = MappedByteBufferTable.Creator.newCreator(filePath, date, pollingTimeMinute.intValue()).create(table -> loadData2Memory(table)); - } catch (IOException e) { - e.printStackTrace(); - } - - } +// if (!isLarge) { + memoryTable = new ByteArrayMemoryTable(); + loadData2Memory(memoryTable); + LOGGER.info("[{}][{}] Init_ByteArrayMemoryTable", IdUtil.instanceId(), getName()); +// } else { +// Date date = new Date(); +// try { +// memoryTable = MappedByteBufferTable.Creator.newCreator(filePath, date, pollingTimeSeconds.intValue()).create(table -> loadData2Memory(table)); +// } catch (IOException e) { +// LOGGER.error("[{}][{}] Init_MappedByteBufferTable_Error", IdUtil.instanceId(), getConfigureName(), e); +// } +// LOGGER.info("[{}][{}] Init_MappedByteBufferTable", IdUtil.instanceId(), getConfigureName()); +// } return memoryTable; } protected abstract void loadData2Memory(AbstractMemoryTable table); - @Override - public void destroy() { + @Override public void destroy() { super.destroy(); - executorService.shutdown(); + ScheduleFactory.getInstance().cancel(getNameSpace() + "-" + getName() + "-dim_schedule"); } /** @@ -430,6 +395,25 @@ public void setIndex(String indexs) { this.indexs = tmp; } + protected List> matchExpressionByIndex(IndexExecutor executor, JSONObject msg, boolean needAll, String script) { + return executor.match(msg, this, needAll, script); + } + + /** + * 每个表达式一个执行器,主要是执行非索引的过滤能力 + * + * @param expressionStr + * @return + */ + protected IndexExecutor getOrCreateIndexExecutor(String expressionStr) { + IndexExecutor indexNamelistExecutor = cache.get(expressionStr); + if (indexNamelistExecutor == null) { + indexNamelistExecutor = new IndexExecutor(expressionStr, getNameSpace(), this.indexs, dataCache.getCloumnName2DatatType().keySet()); + cache.put(expressionStr, indexNamelistExecutor); + } + return indexNamelistExecutor; + } + /** * 建议指定索引,会基于索引建立map,对于等值的判断,可以快速匹配 * @@ -447,15 +431,15 @@ private String addIndex(List indexs, String... fieldNames) { return index; } - public Long getPollingTimeMinute() { - return pollingTimeMinute; + public Long getPollingTimeSeconds() { + return pollingTimeSeconds; } - public void setPollingTimeMinute(Long pollingTimeMinute) { - this.pollingTimeMinute = pollingTimeMinute; + public void setPollingTimeSeconds(Long pollingTimeSeconds) { + this.pollingTimeSeconds = pollingTimeSeconds; } - public List getIndexs() { + @Override public List getIndexs() { return indexs; } @@ -486,4 +470,8 @@ public String getFilePath() { public void setFilePath(String filePath) { this.filePath = filePath; } + + public static interface IDimField { + boolean isDimField(Object fieldName); + } } diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/AbstractProcShareDim.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/AbstractProcShareDim.java index 41d56f10..d8c4a448 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/AbstractProcShareDim.java +++ b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/AbstractProcShareDim.java @@ -20,21 +20,25 @@ import java.nio.channels.FileChannel; import java.text.SimpleDateFormat; import java.util.Date; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.cache.compress.AbstractMemoryTable; +import org.apache.rocketmq.streams.common.threadpool.ScheduleFactory; import org.apache.rocketmq.streams.dim.index.DimIndex; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * @description 基于本地文件存储的多进程共享的dim */ public abstract class AbstractProcShareDim extends AbstractDim { - static final Log logger = LogFactory.getLog(AbstractProcShareDim.class); - + static final Logger LOGGER = LoggerFactory.getLogger(AbstractProcShareDim.class); + static final Object lock = new Object(); + static volatile boolean isFinishCreate = false; + static AbstractMemoryTable table; + static DimIndex dimIndex; + static String cycleStr; + String filePath; /** * 多进程文件锁 */ @@ -44,14 +48,6 @@ public abstract class AbstractProcShareDim extends AbstractDim { */ private transient File doneFile; private transient FileChannel lockChannel; - String filePath; - - static final Object lock = new Object(); - static volatile boolean isFinishCreate = false; - static AbstractMemoryTable table; - static ScheduledExecutorService executorService; - static DimIndex dimIndex; - static String cycleStr; public AbstractProcShareDim(String filePath) { super(); @@ -60,50 +56,47 @@ public AbstractProcShareDim(String filePath) { @Override protected boolean initConfigurable() { - if (executorService == null) { - synchronized (lock) { - if (executorService == null) { - executorService = new ScheduledThreadPoolExecutor(1); - executorService.scheduleAtFixedRate(new Runnable() { - @Override - public void run() { - loadNameList(); - } - }, pollingTimeMinute, pollingTimeMinute, TimeUnit.MINUTES); + synchronized (lock) { + ScheduleFactory.getInstance().execute(getNameSpace() + "-" + getName() + "-file_dim_schedule", new Runnable() { + @Override + public void run() { + loadNameList(); } - } + }, pollingTimeSeconds, pollingTimeSeconds, TimeUnit.MINUTES); } -// boolean old = Boolean.parseBoolean(ComponentCreator.getProperties().getProperty(ConfigureFileKey.DIPPER_RUNNING_STATUS)); -// ComponentCreator.getProperties().put(ConfigureFileKey.DIPPER_RUNNING_STATUS, false); -// boolean success = super.initConfigurable(); -// ComponentCreator.getProperties().put(ConfigureFileKey.DIPPER_RUNNING_STATUS, old); return true; } + @Override protected void loadNameList() { try { - logger.info(getConfigureName() + " begin polling data"); + LOGGER.info(getName() + " begin polling data"); //全表数据 AbstractMemoryTable dataCacheVar = loadData(); table = dataCacheVar; this.nameListIndex = buildIndex(dataCacheVar); this.columnNames = this.dataCache.getCloumnName2Index().keySet(); } catch (Exception e) { - logger.error("Load configurables error:" + e.getMessage(), e); + LOGGER.error("Load configurables error:" + e.getMessage(), e); } } private void getCycleStr(Date date) { - if (pollingTimeMinute >= 1 && pollingTimeMinute < 60) { + String cycleStr = ""; + if (pollingTimeSeconds >= 1 && pollingTimeSeconds < 60) { + SimpleDateFormat format = new SimpleDateFormat("yyyyMMddHHmmss"); + cycleStr = format.format(date); + } else if (pollingTimeSeconds >= 60 && pollingTimeSeconds < (60 * 60)) { SimpleDateFormat format = new SimpleDateFormat("yyyyMMddHHmm"); cycleStr = format.format(date); - } else if (pollingTimeMinute >= 60 && pollingTimeMinute < (60 * 24)) { + } else if (pollingTimeSeconds >= 60 * 60 && pollingTimeSeconds < (60 * 60 * 24)) { SimpleDateFormat format = new SimpleDateFormat("yyyyMMddHH"); cycleStr = format.format(date); - } else if (pollingTimeMinute >= (60 * 24)) { + } else if (pollingTimeSeconds >= (60 * 60 * 24)) { SimpleDateFormat format = new SimpleDateFormat("yyyyMMdd"); cycleStr = format.format(date); } + } } diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/DBDim.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/DBDim.java index 48b2bd9e..e84d5701 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/DBDim.java +++ b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/DBDim.java @@ -18,48 +18,32 @@ import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.cache.compress.AbstractMemoryTable; import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; -import org.apache.rocketmq.streams.common.utils.IPUtil; -import org.apache.rocketmq.streams.common.utils.MapKeyUtil; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.db.driver.DriverBuilder; import org.apache.rocketmq.streams.db.driver.JDBCDriver; import org.apache.rocketmq.streams.db.driver.batchloader.BatchRowLoader; import org.apache.rocketmq.streams.db.driver.batchloader.IRowOperator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class DBDim extends AbstractDim { - private static final Log LOG = LogFactory.getLog(DBDim.class); - - private String jdbcdriver = "com.mysql.jdbc.Driver"; - - @ENVDependence - private String url; - - @ENVDependence - private String userName; - - @ENVDependence - private String password; - - private String sql;//sql 会被定时执行 - + protected static final Logger LOGGER = LoggerFactory.getLogger(DBDim.class); protected String idFieldName; - - private static transient AtomicInteger nameCreator = new AtomicInteger(0); - /** * 是否支持批量查找 */ protected transient Boolean supportBatch = false; + @ENVDependence private String jdbcDriver = ConfigurationKey.DEFAULT_JDBC_DRIVER; + @ENVDependence private String url; + @ENVDependence private String userName; + @ENVDependence private String password; + private String sql;//sql 会被定时执行 public DBDim() { - this.setConfigureName(MapKeyUtil.createKey(IPUtil.getLocalIdentification(), System.currentTimeMillis() + "", - nameCreator.incrementAndGet() + "")); this.setType(TYPE); } @@ -69,25 +53,32 @@ protected void loadData2Memory(AbstractMemoryTable tableCompress) { BatchRowLoader batchRowLoader = new BatchRowLoader(idFieldName, sql, new IRowOperator() { @Override public synchronized void doProcess(Map row) { - tableCompress.addRow(row); + doProcessRow(tableCompress, row); } - }); - batchRowLoader.startLoadData(); + }, jdbcDriver, url, userName, password); + startBatchRowLoader(batchRowLoader); return; } List> rows = executeQuery(); for (Map row : rows) { - tableCompress.addRow(row); + doProcessRow(tableCompress, row); } } + protected void startBatchRowLoader(BatchRowLoader batchRowLoader) { + batchRowLoader.startLoadData(); + } + + protected void doProcessRow(AbstractMemoryTable tableCompress, Map row) { + tableCompress.addRow(row); + } + protected List> executeQuery() { - JDBCDriver resource = createResouce(); + JDBCDriver resource = createResource(); try { List> result = resource.queryForList(sql); - ; - LOG.info("load configurable's count is " + result.size()); + LOGGER.info("load configurable's count is " + result.size()); return result; } finally { if (resource != null) { @@ -97,54 +88,58 @@ protected List> executeQuery() { } - protected JDBCDriver createResouce() { - return DriverBuilder.createDriver(jdbcdriver, url, userName, password); - } - - public void setJdbcdriver(String jdbcdriver) { - this.jdbcdriver = jdbcdriver; - } - - public void setUrl(String url) { - this.url = url; - } - - public void setUserName(String userName) { - this.userName = userName; - } - - public void setPassword(String password) { - this.password = password; + protected JDBCDriver createResource() { + return DriverBuilder.createDriver(jdbcDriver, url, userName, password); } - public void setSql(String sql) { - this.sql = sql; + public String getJdbcDriver() { + return jdbcDriver; } - public String getJdbcdriver() { - return jdbcdriver; + public void setJdbcDriver(String jdbcDriver) { + this.jdbcDriver = jdbcDriver; } public String getUrl() { return url; } + public void setUrl(String url) { + this.url = url; + } + public String getUserName() { return userName; } + public void setUserName(String userName) { + this.userName = userName; + } + public String getPassword() { return password; } + public void setPassword(String password) { + this.password = password; + } + public String getSql() { return sql; } + public void setSql(String sql) { + this.sql = sql; + } + public Boolean getSupportBatch() { return supportBatch; } + public void setSupportBatch(Boolean supportBatch) { + this.supportBatch = supportBatch; + } + public String getIdFieldName() { return idFieldName; } @@ -153,8 +148,4 @@ public void setIdFieldName(String idFieldName) { this.idFieldName = idFieldName; } - public void setSupportBatch(Boolean supportBatch) { - this.supportBatch = supportBatch; - } - } diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/FileDim.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/FileDim.java index 325ba366..6eaddd84 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/FileDim.java +++ b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/FileDim.java @@ -27,6 +27,14 @@ public class FileDim extends AbstractDim { protected String filePath; + public static void main(String[] args) { + List lines = FileUtil.loadFileLine("/tmp/data_model_extractor_config.txt"); + for (String row : lines) { + JSONObject jsonObject = JSON.parseObject(row); + System.out.println(jsonObject); + } + } + @Override protected void loadData2Memory(AbstractMemoryTable tableCompress) { List rows = FileUtil.loadFileLine(filePath); @@ -40,18 +48,12 @@ protected void loadData2Memory(AbstractMemoryTable tableCompress) { } } - public static void main(String[] args) { - List lines = FileUtil.loadFileLine("/tmp/data_model_extractor_config.txt"); - for (String row : lines) { - JSONObject jsonObject = JSON.parseObject(row); - System.out.println(jsonObject); - } - } - + @Override public String getFilePath() { return filePath; } + @Override public void setFilePath(String filePath) { this.filePath = filePath; } diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/IDataCache.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/IDataCache.java index ba3a71e2..9edda3bd 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/IDataCache.java +++ b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/IDataCache.java @@ -16,11 +16,6 @@ */ package org.apache.rocketmq.streams.dim.model; -/** - * @author zengyu.cw - * @program rocketmq-streams-apache - * @create 2021-11-17 09:43 - * @description - */ + public interface IDataCache { } diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/IDimSource.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/IDimSource.java index 5b1d9494..485d8053 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/IDimSource.java +++ b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/IDimSource.java @@ -18,12 +18,7 @@ import java.util.List; -/** - * @author zengyu.cw - * @program rocketmq-streams-apache - * @create 2021-11-17 09:42:20 - * @description - */ + public interface IDimSource { public boolean put(InputRecord record) throws Exception; diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/service/impl/DimServiceImpl.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/service/impl/DimServiceImpl.java index 30201d27..e02a329a 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/service/impl/DimServiceImpl.java +++ b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/service/impl/DimServiceImpl.java @@ -19,16 +19,12 @@ import com.alibaba.fastjson.JSONObject; import java.util.List; import java.util.Map; -import org.apache.rocketmq.streams.configurable.ConfigurableComponent; -import org.apache.rocketmq.streams.dim.model.AbstractDim; +import org.apache.rocketmq.streams.common.interfaces.IDim; import org.apache.rocketmq.streams.dim.service.IDimService; public class DimServiceImpl implements IDimService { - protected ConfigurableComponent configurableComponent; - public DimServiceImpl(ConfigurableComponent configurableComponent) { - this.configurableComponent = configurableComponent; - } + private IDim nameList; /** * 传入要比对的字段,进行规则匹配。字段和名单的比对逻辑,写在规则中 @@ -55,10 +51,10 @@ public List> matchSupportMultiRow(String dimName, String exp return matchSupportMultiRow(dimName, expressionStr, msgs, null); } + //ewewwew @Override public List> matchSupportMultiRow(String dimName, String expressionStr, Map msgs, String script) { JSONObject jsonObject = createJsonable(msgs); - AbstractDim nameList = configurableComponent.queryConfigurable(AbstractDim.TYPE, dimName); if (nameList != null) { return nameList.matchExpression(expressionStr, jsonObject, true, script); } else { @@ -66,10 +62,10 @@ public List> matchSupportMultiRow(String dimName, String exp } } + ///sdsasdasds @Override public Map match(String nameListName, String expressionStr, Map parameters) { JSONObject jsonObject = createJsonable(parameters); - AbstractDim nameList = configurableComponent.queryConfigurable(AbstractDim.TYPE, nameListName); if (nameList != null) { return nameList.matchExpression(expressionStr, jsonObject); } else { @@ -80,11 +76,18 @@ public Map match(String nameListName, String expressionStr, Map< private JSONObject createJsonable(Map parameters) { JSONObject jsonObject = null; if (parameters instanceof JSONObject) { - jsonObject = (JSONObject)parameters; + jsonObject = (JSONObject) parameters; } else { jsonObject.putAll(parameters); } return jsonObject; } + public IDim getNameList() { + return nameList; + } + + public void setNameList(IDim nameList) { + this.nameList = nameList; + } } diff --git a/rocketmq-streams-dim/src/test/java/com/aliyun/service/DBDimTest.java b/rocketmq-streams-dim/src/test/java/com/aliyun/service/DBDimTest.java new file mode 100644 index 00000000..8837bc5f --- /dev/null +++ b/rocketmq-streams-dim/src/test/java/com/aliyun/service/DBDimTest.java @@ -0,0 +1,68 @@ +/* + * 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 com.aliyun.service; + +import com.alibaba.fastjson.JSONObject; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.rocketmq.streams.common.cache.compress.AbstractMemoryTable; +import org.apache.rocketmq.streams.dim.model.AbstractDim; +import org.apache.rocketmq.streams.dim.model.DBDim; +import org.junit.Test; + +public class DBDimTest { + @Test + public void testDBDim() { + + /** + * udf open执行 + * dbDim可以声明成静态的(相同的url,username,password,sql),同一个进程可以共享 + * 可以通过job参数获取url,username,password + */ + DBDim dbDim = new DBDim(); + dbDim.setUrl(""); + dbDim.setUserName(""); + dbDim.setPassword(""); + dbDim.setSql(""); + dbDim.setIdFieldName("id"); + dbDim.setPollingTimeSeconds(30L); + dbDim.addIndex("ip"); + dbDim.init(); + dbDim.startLoadDimData(); + System.out.println("cost memory is " + dbDim.getDataCache().getByteCount() / 1024 / 1024 + "M"); + + /** + * udf eval 执行,可多条匹配 + */ + JSONObject msg = null; + String msgFieldName = null; + String dimFieldName = null; + + List> resultList = dbDim.matchExpression(msgFieldName, dimFieldName, msg); + } + + public void testSelf() { + new AbstractDim() { + + @Override protected void loadData2Memory(AbstractMemoryTable table) { + table.addRow(new HashMap<>()); + + } + }; + } +} diff --git a/rocketmq-streams-dim/src/test/java/com/aliyun/service/NameListFunctionTest.java b/rocketmq-streams-dim/src/test/java/com/aliyun/service/NameListFunctionTest.java index f08aa453..dd7138be 100644 --- a/rocketmq-streams-dim/src/test/java/com/aliyun/service/NameListFunctionTest.java +++ b/rocketmq-streams-dim/src/test/java/com/aliyun/service/NameListFunctionTest.java @@ -59,7 +59,7 @@ public void testNameList2() { private AbstractDim create() { DBDim dbNameList = new DBDim(); dbNameList.setNameSpace("soc"); - dbNameList.setConfigureName("isoc_field_mappings"); + dbNameList.setName("isoc_field_mappings"); dbNameList.setUrl(""); dbNameList.setUserName(""); dbNameList.setPassword(""); @@ -87,7 +87,7 @@ public void testNameListAllRow() { private AbstractDim createMapping() { DBDim dbNameList = new DBDim(); dbNameList.setNameSpace("soc"); - dbNameList.setConfigureName("isoc_field_mappings"); + dbNameList.setName("isoc_field_mappings"); dbNameList.setUrl(""); dbNameList.setUserName(""); dbNameList.setPassword(""); diff --git a/rocketmq-streams-dispatcher/pom.xml b/rocketmq-streams-dispatcher/pom.xml new file mode 100644 index 00000000..db5dd75c --- /dev/null +++ b/rocketmq-streams-dispatcher/pom.xml @@ -0,0 +1,25 @@ + + + + rocketmq-streams + org.apache.rocketmq + 2.0.1-SNAPSHOT + + 4.0.0 + + rocketmq-streams-dispatcher + ROCKETMQ STREAMS :: dispatcher + + + + org.apache.rocketmq + rocketmq-streams-commons + + + org.apache.rocketmq + rocketmq-streams-lease + + + \ No newline at end of file diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IBalanceCallback.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IBalanceCallback.java new file mode 100644 index 00000000..af25c1d4 --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IBalanceCallback.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.dispatcher; + +import java.util.List; +import java.util.Set; +import org.apache.rocketmq.streams.dispatcher.enums.DispatchMode; + +public interface IBalanceCallback { + + void doBalance(Set currentInstanceTasks); + + List getTasks(); + + DispatchMode getDisPatchMode(); + +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/ICache.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/ICache.java new file mode 100644 index 00000000..eaec4a1f --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/ICache.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.dispatcher; + +import java.util.HashMap; + +public interface ICache { + + void putKeyConfig(String namespace, String key, String value); + + String getKeyConfig(String namespace, String key); + + void deleteKeyConfig(String namespace, String key); + + HashMap getKVListByNameSpace(String namespace); + +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IDispatcher.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IDispatcher.java new file mode 100644 index 00000000..76c1c78a --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IDispatcher.java @@ -0,0 +1,26 @@ +package org.apache.rocketmq.streams.dispatcher; + +public interface IDispatcher { + + /** + * Start the dispatcher + * + * @throws Exception exception + */ + void start() throws Exception; + + /** + * wake up the dispatch + * + * @throws Exception exception + */ + void wakeUp() throws Exception; + + /** + * close dispatch + * + * @throws Exception exception + */ + void close() throws Exception; + +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IDispatcherCallback.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IDispatcherCallback.java new file mode 100644 index 00000000..31efcc85 --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IDispatcherCallback.java @@ -0,0 +1,51 @@ +/* + * 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.dispatcher; + +import java.util.List; + +public interface IDispatcherCallback { + + /** + * 启动任务 + * + * @param jobNames 需要启动的任务名称列表 + * @return 启动成功的任务名称列表 + */ + List start(List jobNames); + + /** + * 停止任务 + * + * @param jobNames 需要停止的任务名称列表 + * @return 停止成功的任务名称列表 + */ + List stop(List jobNames); + + /** + * 获取需要调度的任务名称列表 + * + * @return 需要调度的任务名称列表 + */ + List list(); + + /** + * callback 销毁 + */ + void destroy(); + +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IMapper.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IMapper.java new file mode 100644 index 00000000..f2b512bb --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IMapper.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.dispatcher; + +import java.util.Set; + +public interface IMapper { + + /** + * get the instances + * + * @return instance set + * @throws Exception exception + */ + Set getInstances() throws Exception; + + /** + * get the task list on the instance + * + * @param instance instance + * @return task list + * @throws Exception exception + */ + Set getTasks(String instance) throws Exception; + + /** + * get the Task lie + * + * @return task list + * @throws Exception exception + */ + Set getTasks() throws Exception; + + /** + * put the task into the instance task list + * + * @param instance instance + * @param task task + * @throws Exception exception + */ + void putTask(String instance, T task) throws Exception; + + /** + * remove the task + * + * @param instance instance + * @param task task + * @throws Exception exception + */ + void removeTask(String instance, T task) throws Exception; + + /** + * remove the instance + * + * @param instance instance + * @throws Exception exception + */ + void remove(String instance) throws Exception; + + /** + * check wether the task has bean dispatched + * + * @param task task + * @return boolean + * @throws Exception exception + */ + boolean containTask(T task) throws Exception; + +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IStrategy.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IStrategy.java new file mode 100644 index 00000000..6e2edec3 --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IStrategy.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.dispatcher; + +import java.util.List; +import org.apache.rocketmq.streams.dispatcher.entity.DispatcherMapper; + +public interface IStrategy { + + DispatcherMapper dispatch(List tasks, List instances) throws Exception; + +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IStrategyCallBack.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IStrategyCallBack.java new file mode 100644 index 00000000..8ef01ebc --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/IStrategyCallBack.java @@ -0,0 +1,45 @@ +/* + * 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.dispatcher; + +import java.util.List; +import java.util.Map; + +public interface IStrategyCallBack { + + /** + * 获取需要调度的任务名称列表 + * + * @return 需要调度的任务名称列表 + */ + List list(); + + /** + * 相同groupname的task,允许共享实例,非相同groupname的task 不允许共享资源 + * + * @return + */ + Map getTaskGroupName(); + + /** + * 一个实例最大的任务数,如果返回-1,表示不限制 + * + * @return + */ + int getInstanceMaxTaskCount(); + +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/cache/DBCache.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/cache/DBCache.java new file mode 100644 index 00000000..08ef91f1 --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/cache/DBCache.java @@ -0,0 +1,175 @@ +/* + * 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.dispatcher.cache; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.configuration.SystemContext; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.dispatcher.ICache; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DBCache implements ICache { + + private static final Logger LOGGER = LoggerFactory.getLogger(DBCache.class); + + private final String cacheTable = "dipper_dispatcher_kv_config"; + private final String url; + private final String userName; + private final String password; + + public DBCache() { + try { + String driver = SystemContext.getProperty(ConfigurationKey.JDBC_DRIVER); + if (StringUtil.isEmpty(driver)) { + driver = ConfigurationKey.DEFAULT_JDBC_DRIVER; + } + this.url = SystemContext.getProperty(ConfigurationKey.JDBC_URL); + this.userName = SystemContext.getProperty(ConfigurationKey.JDBC_USERNAME); + this.password = SystemContext.getProperty(ConfigurationKey.JDBC_PASSWORD); + Class.forName(driver); + } catch (ClassNotFoundException e) { + throw new RuntimeException(e); + } + } + + public DBCache(Properties properties) { + try { + String driver = properties.getProperty(ConfigurationKey.JDBC_DRIVER); + this.url = properties.getProperty(ConfigurationKey.JDBC_URL); + this.userName = properties.getProperty(ConfigurationKey.JDBC_USERNAME); + this.password = properties.getProperty(ConfigurationKey.JDBC_PASSWORD); + Class.forName(driver); + } catch (ClassNotFoundException e) { + throw new RuntimeException(e); + } + } + + @Override + public void putKeyConfig(String namespace, String key, String value) { + try { + String sql = "insert into " + cacheTable + " (namespace, config_key, config_value) values (?, ?, ?) on duplicate key update namespace=?, config_key=?, config_value=?"; + execute(sql, new String[] {namespace, key, value, namespace, key, value}); + } catch (Exception e) { + LOGGER.error("Put_Key_Config_Error", e); + } + } + + @Override + public String getKeyConfig(String namespace, String key) { + String configValue = null; + try { + String sql = "select config_value from " + cacheTable + " where namespace=? and config_key=?"; + List> result = executeQuery(sql, new String[] {namespace, key}, new String[] {"config_value"}); + if (!result.isEmpty()) { + configValue = result.get(0).get("config_value"); + } + } catch (Exception e) { + LOGGER.error("Get_Key_Config_Error", e); + } + return configValue; + } + + @Override + public void deleteKeyConfig(String namespace, String key) { + try { + String sql = "delete from " + cacheTable + " where namespace=? and config_key=?"; + execute(sql, new String[] {namespace, key}); + } catch (Exception e) { + LOGGER.error("Delete_Key_Config_Error", e); + } + } + + @Override + public HashMap getKVListByNameSpace(String namespace) { + HashMap kv = Maps.newHashMap(); + try { + String sql = "select config_key, config_value from " + cacheTable + " where namespace=?"; + List> result = executeQuery(sql, new String[] {namespace}, new String[] {"config_key", "config_value"}); + for (Map data : result) { + kv.put(data.get("config_key"), data.get("config_value")); + } + } catch (Exception e) { + LOGGER.error("Get_KVList_Config_Error", e); + } + return kv; + } + + private void execute(String sql, String[] params) throws Exception { + Connection connection = null; + PreparedStatement preparedStatement = null; + try { + connection = DriverManager.getConnection(this.url, this.userName, this.password); + preparedStatement = connection.prepareStatement(sql); + for (int i = 1; i <= params.length; i++) { + preparedStatement.setString(i, params[i - 1]); + } + preparedStatement.execute(); + } finally { + if (preparedStatement != null) { + preparedStatement.close(); + } + if (connection != null) { + connection.close(); + } + } + } + + private List> executeQuery(String sql, String[] params, String[] columns) throws Exception { + Connection connection = null; + PreparedStatement preparedStatement = null; + ResultSet resultSet = null; + List> result = Lists.newArrayList(); + try { + connection = DriverManager.getConnection(this.url, this.userName, this.password); + preparedStatement = connection.prepareStatement(sql); + for (int i = 1; i <= params.length; i++) { + preparedStatement.setString(i, params[i - 1]); + } + resultSet = preparedStatement.executeQuery(); + while (resultSet.next()) { + Map data = Maps.newHashMap(); + for (String column : columns) { + data.put(column, resultSet.getString(column)); + } + result.add(data); + } + } finally { + if (preparedStatement != null) { + preparedStatement.close(); + } + if (connection != null) { + connection.close(); + } + if (resultSet != null) { + resultSet.close(); + } + } + return result; + } + +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/cache/RocketmqCache.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/cache/RocketmqCache.java new file mode 100644 index 00000000..ad17733f --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/cache/RocketmqCache.java @@ -0,0 +1,57 @@ +/* + * 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.dispatcher.cache; + +import java.util.HashMap; +import org.apache.rocketmq.common.protocol.body.KVTable; +import org.apache.rocketmq.streams.common.utils.RocketmqUtil; +import org.apache.rocketmq.streams.dispatcher.ICache; + +public class RocketmqCache implements ICache { + + private String nameServAddr; + + public RocketmqCache(String nameServAddr) { + this.nameServAddr = nameServAddr; + } + + @Override public void putKeyConfig(String namespace, String key, String value) { + RocketmqUtil.putKeyConfig(namespace, key, value, this.nameServAddr); + } + + @Override public String getKeyConfig(String namespace, String key) { + return RocketmqUtil.getKeyConfig(namespace, key, this.nameServAddr); + } + + @Override public void deleteKeyConfig(String namespace, String key) { + RocketmqUtil.deleteKeyConfig(namespace, key, this.nameServAddr); + } + + @Override public HashMap getKVListByNameSpace(String namespace) { + KVTable kvTable = RocketmqUtil.getKVListByNameSpace(namespace, this.nameServAddr); + assert kvTable != null; + return kvTable.getTable(); + } + + public String getNameServAddr() { + return nameServAddr; + } + + public void setNameServAddr(String nameServAddr) { + this.nameServAddr = nameServAddr; + } +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/callback/DefaultDispatcherCallback.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/callback/DefaultDispatcherCallback.java new file mode 100644 index 00000000..c411b453 --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/callback/DefaultDispatcherCallback.java @@ -0,0 +1,52 @@ +package org.apache.rocketmq.streams.dispatcher.callback; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import java.util.List; +import java.util.Map; +import org.apache.rocketmq.streams.common.topology.graph.JobGraph; +import org.apache.rocketmq.streams.dispatcher.IDispatcherCallback; + +public class DefaultDispatcherCallback implements IDispatcherCallback { + + private final Map jobGraphMap = Maps.newHashMap(); + + @Override public List start(List jobNames) { + List success = Lists.newArrayList(); + for (String jobName : jobNames) { + if (jobGraphMap.containsKey(jobName)) { + jobGraphMap.get(jobName).start(); + success.add(jobName); + } + } + return success; + } + + @Override public List stop(List jobNames) { + List success = Lists.newArrayList(); + for (String jobName : jobNames) { + if (jobGraphMap.containsKey(jobName)) { + jobGraphMap.get(jobName).stop(); + success.add(jobName); + } + } + return success; + } + + @Override public List list() { + return Lists.newArrayList(jobGraphMap.keySet()); + } + + public void add(String jobName, JobGraph job) { + this.jobGraphMap.put(jobName, job); + } + + public void delete(String jobName) { + this.jobGraphMap.remove(jobName); + } + + @Override + public void destroy() { + this.jobGraphMap.clear(); + } +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/constant/DispatcherConstant.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/constant/DispatcherConstant.java new file mode 100644 index 00000000..52f57eff --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/constant/DispatcherConstant.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.dispatcher.constant; + +public class DispatcherConstant { + public static String TASK_SERIAL_NUM_SEPARATOR = "$#"; +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/entity/DispatcherMapper.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/entity/DispatcherMapper.java new file mode 100644 index 00000000..e2d5cc60 --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/entity/DispatcherMapper.java @@ -0,0 +1,180 @@ +/* + * 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.dispatcher.entity; + +import com.alibaba.fastjson.JSONObject; +import com.alibaba.fastjson.TypeReference; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; +import org.apache.rocketmq.streams.dispatcher.IMapper; + +public class DispatcherMapper implements IMapper { + + private Map> mapping; + + public DispatcherMapper() { + this.mapping = Maps.newHashMap(); + } + + public DispatcherMapper(Map> mapping) { + this.mapping = mapping; + } + + public static DispatcherMapper parse(String mapperString) { + if (mapperString == null || mapperString.isEmpty()) { + return null; + } + return new DispatcherMapper(JSONObject.parseObject(mapperString, new TypeReference>>() { + })); + } + + public static void main(String[] args) { + Map map1 = Maps.newHashMap(); + map1.put("key1", "value1"); + map1.put("key2", "value2"); + map1.put("key3", "value3"); + + Map map2 = Maps.newHashMap(); + map2.put("key1", "value1"); + map2.put("key3", "value3"); + map2.put("key2", "value2"); + + System.out.println(map1.equals(map2)); + + TreeSet list1 = Sets.newTreeSet(); + list1.add("value3"); + list1.add("value2"); + list1.add("value1"); + + System.out.println(list1.iterator().next()); + for (String s : list1) { + System.out.println(s); + } + + TreeSet list2 = Sets.newTreeSet(); + list2.add("value3"); + list2.add("value1"); + list2.add("value2"); + + System.out.println(list1.equals(list2)); + + TreeSet list3 = Sets.newTreeSet(); + list3.add("value2"); + list3.add("value1"); + list3.add("value3"); + + TreeSet list4 = Sets.newTreeSet(); + list4.add("value1"); + list4.add("value3"); + list4.add("value2"); + + Map> map3 = Maps.newHashMap(); + map3.put("key2", list2); + map3.put("key1", list1); + + Map> map4 = Maps.newHashMap(); + map4.put("key2", list4); + map4.put("key1", list3); + + System.out.println(map3.equals(map4)); + + DispatcherMapper mapper = new DispatcherMapper(map3); + String json = mapper.toString(); + + System.out.println(json); + + DispatcherMapper mapper1 = DispatcherMapper.parse(json); + try { + Set tasks = mapper1.getTasks("key2"); + for (String task : tasks) { + System.out.println(task); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + + } + + public Map> getMapping() { + return mapping; + } + + public void setMapping(Map> mapping) { + this.mapping = mapping; + } + + @Override public Set getInstances() throws Exception { + return this.mapping.keySet(); + } + + @Override public Set getTasks(String instance) throws Exception { + return mapping.get(instance) == null ? Sets.newTreeSet() : mapping.get(instance); + } + + @Override public Set getTasks() throws Exception { + Set tasks = Sets.newTreeSet(); + for (Set sets : mapping.values()) { + tasks.addAll(sets); + } + return tasks; + } + + @Override public void putTask(String instance, String task) throws Exception { + TreeSet tasks = mapping.get(instance); + if (tasks == null) { + tasks = Sets.newTreeSet(); + } + if (task != null) { + tasks.add(task); + } + mapping.put(instance, tasks); + } + + @Override public void removeTask(String instance, String task) throws Exception { + TreeSet tasks = mapping.get(instance); + tasks.remove(task); + } + + @Override public void remove(String instance) throws Exception { + mapping.remove(instance); + } + + @Override public boolean containTask(String task) throws Exception { + for (Map.Entry> entry : this.mapping.entrySet()) { + if (entry.getValue().contains(task)) { + return true; + } + } + return false; + } + + @Override public boolean equals(Object obj) { + if (obj instanceof DispatcherMapper) { + DispatcherMapper mapper = (DispatcherMapper) obj; + return this.mapping.equals(mapper.mapping); + } + return super.equals(obj); + } + + @Override public String toString() { + return JSONObject.toJSONString(this.mapping); + } + +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/enums/DispatchMode.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/enums/DispatchMode.java new file mode 100644 index 00000000..675f1aa4 --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/enums/DispatchMode.java @@ -0,0 +1,43 @@ +/* + * 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.dispatcher.enums; + +public enum DispatchMode { + + ALL("All"), + + LEAST("Least"), + + HASH("Hash"), + AVERAGELY("Averagely"), + + CONSISTENCY("Consistency"); + + private String name; + + DispatchMode(String name) { + this.name = name; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/impl/LeaseDispatcher.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/impl/LeaseDispatcher.java new file mode 100644 index 00000000..3ecc173e --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/impl/LeaseDispatcher.java @@ -0,0 +1,238 @@ +package org.apache.rocketmq.streams.dispatcher.impl; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.commons.collections.CollectionUtils; +import org.apache.rocketmq.streams.common.threadpool.ScheduleFactory; +import org.apache.rocketmq.streams.common.utils.IdUtil; +import org.apache.rocketmq.streams.dispatcher.ICache; +import org.apache.rocketmq.streams.dispatcher.IDispatcher; +import org.apache.rocketmq.streams.dispatcher.IDispatcherCallback; +import org.apache.rocketmq.streams.dispatcher.IStrategy; +import org.apache.rocketmq.streams.dispatcher.entity.DispatcherMapper; +import org.apache.rocketmq.streams.dispatcher.enums.DispatchMode; +import org.apache.rocketmq.streams.dispatcher.strategy.LeastStrategy; +import org.apache.rocketmq.streams.dispatcher.strategy.StrategyFactory; +import org.apache.rocketmq.streams.lease.model.LeaseInfo; +import org.apache.rocketmq.streams.lease.service.ILeaseStorage; +import org.apache.rocketmq.streams.lease.service.impl.LeaseServiceImpl; +import org.apache.rocketmq.streams.lease.service.storages.DBLeaseStorage; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class LeaseDispatcher implements IDispatcher { + + public static final String MAPPER_KEY = "mapper_key"; + public static final String NAMESPACE_CONFIG_SUFFIX = "_config"; + public static final String NAMESPACE_STATUS_SUFFIX = "_status"; + public static final Map> local = Maps.newHashMap(); + private static final Logger LOGGER = LoggerFactory.getLogger(LeaseDispatcher.class); + private final String instanceName; + private final String dispatchGroup; + private final DispatchMode dispatchMode; + private final IDispatcherCallback dispatcherCallback; + private final ICache cache; + private final int scheduleTime; + + private final String type; + + protected transient AtomicBoolean isStart = new AtomicBoolean(false); + + protected transient LeaseServiceImpl leaseService; + + public LeaseDispatcher(String jdbcDriver, String url, String userName, String password, String instanceName, String dispatchGroup, DispatchMode dispatchMode, int scheduleTime, IDispatcherCallback dispatcherCallback, ICache cache) { + this(jdbcDriver, url, userName, password, instanceName, dispatchGroup, dispatchMode, scheduleTime, dispatcherCallback, cache, "job"); + } + + public LeaseDispatcher(String jdbcDriver, String url, String userName, String password, String instanceName, String dispatchGroup, DispatchMode dispatchMode, int scheduleTime, IDispatcherCallback dispatcherCallback, ICache cache, String type) { + this.dispatchGroup = dispatchGroup; + this.instanceName = instanceName; + this.dispatchMode = dispatchMode; + this.dispatcherCallback = dispatcherCallback; + this.cache = cache; + this.scheduleTime = scheduleTime; + + //构建leaseService + this.leaseService = new LeaseServiceImpl(); + ILeaseStorage storage = new DBLeaseStorage(jdbcDriver, url, userName, password); + this.leaseService.setLeaseStorage(storage); + this.type = type; + } + + @Override + public void start() throws Exception { + try { + if (this.isStart.compareAndSet(false, true)) { + + //启动Master选举 + this.leaseService.startLeaseTask("Master_" + this.dispatchGroup, scheduleTime, nextLeaseDate -> { + LOGGER.info("[{}][{}][{}] Select_Master_Success", this.instanceName, this.dispatchGroup, IdUtil.objectId(this)); + }); + + ScheduleFactory.getInstance().execute("lease_dispatcher_" + this.instanceName + "_" + this.dispatchGroup, () -> { + try { + wakeUp(); + doRunning(); + LOGGER.info("[{}][{}][{}] Schedule_Execute_Success", this.instanceName, this.dispatchGroup, IdUtil.objectId(this)); + } catch (Exception e) { + LOGGER.error("[{}][{}][{}] Schedule_Execute_Error", this.instanceName, this.dispatchGroup, IdUtil.objectId(this), e); + } + }, 0, scheduleTime / 2, TimeUnit.SECONDS); + //启动租约的调度任务上报心跳 + this.leaseService.holdLock(dispatchGroup, instanceName, scheduleTime); + } + } catch (Exception e) { + this.isStart.set(false); + throw new RuntimeException("Dispatcher_Start_Error", e); + } + } + + public synchronized void doDispatch() throws Exception { + TreeSet tmpConsumerIdList = Sets.newTreeSet(); + List consumerList = this.leaseService.queryLockedInstanceByNamePrefix(this.dispatchGroup, ""); + if (consumerList != null && !consumerList.isEmpty()) { + for (LeaseInfo leaseInfo : consumerList) { + LOGGER.info("[{}][{}][{}] Instance_List_[{}]", this.instanceName, this.dispatchGroup, IdUtil.objectId(this), leaseInfo.getLeaseUserIp()); + String workId = leaseInfo.getLeaseUserIp().substring(0, leaseInfo.getLeaseUserIp().indexOf(":")).replaceAll("\\.", "_"); + tmpConsumerIdList.add(workId); + } + if (type.equalsIgnoreCase("job")) { + tmpConsumerIdList.remove(IdUtil.workerId()); + } + } + + TreeSet tasks = Sets.newTreeSet(this.dispatcherCallback.list());//此逻辑必须放在选主之前,因为所有实例都需要加载任务 + if (!tasks.isEmpty()) { + for (String task : tasks) { + LOGGER.info("[{}][{}][{}] Task_List_[{}]", this.instanceName, this.dispatchGroup, IdUtil.objectId(this), task); + } + } + + if (!tmpConsumerIdList.isEmpty()) { + TreeSet consumerIdList = local.getOrDefault(this.dispatchGroup + "_" + this.instanceName + "_consumerList", Sets.newTreeSet()); + boolean isConsumerChange = !consumerIdList.equals(tmpConsumerIdList); + if (isConsumerChange) { + LOGGER.info("[{}][{}][{}] Cluster_CurrentInstanceList({})_NewInstanceList({})_IsChanged", this.instanceName, this.dispatchGroup, IdUtil.objectId(this), String.join(",", consumerIdList), String.join(",", tmpConsumerIdList)); + } + + if (!tmpConsumerIdList.isEmpty()) { + TreeSet taskList = local.getOrDefault(this.dispatchGroup + "_" + this.instanceName + "_taskList", Sets.newTreeSet()); + boolean isTaskChange = !taskList.equals(tasks); + if (isTaskChange) { + LOGGER.info("[{}][{}][{}] Task_CurrentTask({})_NewTask({})_IsChanged", this.instanceName, this.dispatchGroup, IdUtil.objectId(this), String.join(", ", taskList), String.join(",", tasks)); + } + if (isTaskChange || isConsumerChange) { + //计算新的调度公式 + IStrategy iStrategy = StrategyFactory.getStrategy(this.dispatchMode); + if (iStrategy instanceof LeastStrategy) {//如果是Least策略,则需要通过cache获取到当前的调度状态 + String currentMessage = this.cache.getKeyConfig(this.dispatchGroup + NAMESPACE_CONFIG_SUFFIX, MAPPER_KEY); + if (currentMessage != null && !currentMessage.isEmpty()) { + DispatcherMapper currentDispatcherMapper = DispatcherMapper.parse(currentMessage); + ((LeastStrategy)iStrategy).setCurrentDispatcherMapper(currentDispatcherMapper); + } + } + DispatcherMapper dispatcherMapper = iStrategy.dispatch(Lists.newArrayList(tasks), Lists.newArrayList(tmpConsumerIdList)); + String currentMessage = this.cache.getKeyConfig(this.dispatchGroup + NAMESPACE_CONFIG_SUFFIX, MAPPER_KEY); + DispatcherMapper currentDispatcherMapper = DispatcherMapper.parse(currentMessage); + if (currentMessage != null && dispatcherMapper.equals(currentDispatcherMapper)) { + LOGGER.info("[{}][{}][{}] Dispatcher_Result_Not_Changed", this.instanceName, this.dispatchGroup, IdUtil.objectId(this)); + } else { + LOGGER.info("[{}][{}][{}] Dispatcher_Result_Changed_CurrentResult({})_NewResult({})", this.dispatchGroup, this.instanceName, IdUtil.objectId(this), currentMessage == null ? "{}" : currentMessage, dispatcherMapper); + //确认需要推送新的调度列表 + this.cache.putKeyConfig(this.dispatchGroup + NAMESPACE_CONFIG_SUFFIX, MAPPER_KEY, dispatcherMapper.toString()); + } + local.put(this.dispatchGroup + "_" + this.instanceName + "_consumerList", tmpConsumerIdList); + local.put(this.dispatchGroup + "_" + this.instanceName + "_taskList", tasks); + } + } + } else { + LOGGER.info("[{}][{}][{}] ConsumerList_Empty", this.instanceName, this.dispatchGroup, IdUtil.objectId(this)); + } + } + + public void doRunning() { + try { + //启动前,重新加载一下内存中的任务实例 + this.dispatcherCallback.list(); + + String currentMessage = this.cache.getKeyConfig(this.dispatchGroup + NAMESPACE_CONFIG_SUFFIX, MAPPER_KEY); + DispatcherMapper currentDispatcherMapper = DispatcherMapper.parse(currentMessage); + + Set taskSet = (currentDispatcherMapper == null || currentDispatcherMapper.getTasks(this.instanceName) == null) ? Sets.newHashSet() : currentDispatcherMapper.getTasks(this.instanceName); + TreeSet localCache = local.getOrDefault(this.dispatchGroup + "_" + this.instanceName + "_local_cache", Sets.newTreeSet()); + LOGGER.info("[{}][{}][{}] Dispatcher({})_Local({})", this.instanceName, this.dispatchGroup, IdUtil.objectId(this), String.join(",", taskSet), String.join(",", localCache)); + + TreeSet tmpCache = Sets.newTreeSet(); + tmpCache.addAll(localCache); + + //执行停止指令 + List needStop = (List)CollectionUtils.subtract(tmpCache, taskSet); + if (!needStop.isEmpty()) { + List stopSuccess = this.dispatcherCallback.stop(Lists.newArrayList(needStop)); + if (stopSuccess != null && !stopSuccess.isEmpty()) { + stopSuccess.forEach(tmpCache::remove); + } + LOGGER.info("[{}][{}][{}] Stop_Tasks_NeedStop({})_Stopped({})", this.instanceName, this.dispatchGroup, IdUtil.objectId(this), String.join(",", needStop), stopSuccess == null ? "" : String.join(",", stopSuccess)); + } else { + LOGGER.info("[{}][{}][{}] No_NeedStop", this.instanceName, this.dispatchGroup, IdUtil.objectId(this)); + } + + //执行启动 + List needStart = (List)CollectionUtils.subtract(taskSet, tmpCache); + if (!needStart.isEmpty()) { + List startSuccess = this.dispatcherCallback.start(Lists.newArrayList(needStart)); + if (startSuccess != null && !startSuccess.isEmpty()) { + tmpCache.addAll(startSuccess); + } + LOGGER.info("[{}][{}][{}] Start_Tasks_NeedStart({})_Started({})", this.instanceName, this.dispatchGroup, IdUtil.objectId(this), String.join(",", needStart), startSuccess == null ? "" : String.join(",", startSuccess)); + } else { + LOGGER.info("[{}][{}][{}] No_NeedStart", this.instanceName, this.dispatchGroup, IdUtil.objectId(this)); + } + if (!localCache.equals(tmpCache)) { + local.put(this.dispatchGroup + "_" + this.instanceName + "_local_cache", tmpCache); + this.cache.putKeyConfig(this.dispatchGroup + NAMESPACE_STATUS_SUFFIX, this.instanceName, String.join(",", tmpCache)); + LOGGER.info("[{}][{}][{}] Local_Cache_Changed_From({})_To({})", this.instanceName, this.dispatchGroup, IdUtil.objectId(this), String.join(",", localCache), String.join(",", tmpCache)); + } + } catch (Exception e) { + LOGGER.error("[{}][{}] Dispatcher_Error", this.instanceName, this.dispatchGroup, e); + } + } + + @Override + public void wakeUp() throws Exception { + if (this.leaseService.hasLease("Master_" + this.dispatchGroup)) { + LOGGER.info("[{}][{}][{}] Master_Running", this.instanceName, this.dispatchGroup, IdUtil.objectId(this)); + doDispatch(); + } else { + LOGGER.info("[{}][{}][{}] Slave_Running", this.instanceName, this.dispatchGroup, IdUtil.objectId(this)); + } + } + + @Override + public void close() { + try { + if (this.isStart.compareAndSet(true, false)) { + if (this.leaseService != null) { + this.leaseService.unlock(this.dispatchGroup, this.instanceName); + this.leaseService.stopLeaseTask("Master_" + this.dispatchGroup); + } + ScheduleFactory.getInstance().cancel("lease_dispatcher_" + this.instanceName + "_" + this.dispatchGroup); + local.clear(); + LOGGER.info("[{}][{}] Dispatcher_Close_Success", this.instanceName, this.dispatchGroup); + } + } catch (Exception e) { + this.isStart.set(true); + LOGGER.error("[{}][{}] Dispatcher_Close_Error", this.instanceName, this.dispatchGroup, e); + } + } + +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/impl/RocketmqDispatcher.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/impl/RocketmqDispatcher.java new file mode 100644 index 00000000..4eb1ed46 --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/impl/RocketmqDispatcher.java @@ -0,0 +1,362 @@ +/* + * 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.dispatcher.impl; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.commons.collections.CollectionUtils; +import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; +import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus; +import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently; +import org.apache.rocketmq.client.impl.MQClientManager; +import org.apache.rocketmq.client.impl.consumer.DefaultMQPushConsumerImpl; +import org.apache.rocketmq.client.impl.consumer.RebalancePushImpl; +import org.apache.rocketmq.client.impl.factory.MQClientInstance; +import org.apache.rocketmq.client.producer.DefaultMQProducer; +import org.apache.rocketmq.client.producer.SendCallback; +import org.apache.rocketmq.client.producer.SendResult; +import org.apache.rocketmq.common.message.Message; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.streams.common.utils.ReflectUtil; +import org.apache.rocketmq.streams.common.utils.RocketmqUtil; +import org.apache.rocketmq.streams.dispatcher.ICache; +import org.apache.rocketmq.streams.dispatcher.IDispatcher; +import org.apache.rocketmq.streams.dispatcher.IDispatcherCallback; +import org.apache.rocketmq.streams.dispatcher.IStrategy; +import org.apache.rocketmq.streams.dispatcher.cache.RocketmqCache; +import org.apache.rocketmq.streams.dispatcher.entity.DispatcherMapper; +import org.apache.rocketmq.streams.dispatcher.enums.DispatchMode; +import org.apache.rocketmq.streams.dispatcher.strategy.LeastStrategy; +import org.apache.rocketmq.streams.dispatcher.strategy.StrategyFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RocketmqDispatcher implements IDispatcher { + + public static final String MAPPER_KEY = "mapper_key"; + public static final String NAMESPACE_CONFIG_SUFFIX = "_config"; + public static final String NAMESPACE_STATUS_SUFFIX = "_status"; + public static final String CLUSTER_NAME = "DefaultCluster"; + private static final Logger LOGGER = LoggerFactory.getLogger(RocketmqDispatcher.class); + private final String instanceName; + private final String dispatchGroup; + private final String voteTopic; + private final DispatchMode dispatchMode; + private final IDispatcherCallback dispatcherCallback; + private final ICache cache; + private final String nameServer; + protected transient AtomicBoolean isStart = new AtomicBoolean(false); + protected transient TreeSet consumerIdList = Sets.newTreeSet(); + protected transient TreeSet taskList = Sets.newTreeSet(); + protected transient Set localCache = Sets.newHashSet(); + protected transient String className; + protected transient String methodName; + protected transient boolean isMaster = false; + protected transient Long lastTimestamp = System.currentTimeMillis(); + private DefaultMQPushConsumer voteListener; + private DefaultMQPushConsumer messageListener; + private DefaultMQProducer messageSender; + + public RocketmqDispatcher(String nameServer, String voteTopic, String instanceName, String dispatchGroup, DispatchMode dispatchMode, IDispatcherCallback dispatcherCallback) { + this(nameServer, voteTopic, instanceName, dispatchGroup, dispatchMode, dispatcherCallback, new RocketmqCache(nameServer)); + + } + + public RocketmqDispatcher(String nameServer, String voteTopic, String instanceName, String dispatchGroup, DispatchMode dispatchMode, IDispatcherCallback dispatcherCallback, ICache cache) { + this.nameServer = nameServer; + this.voteTopic = voteTopic; + this.dispatchGroup = dispatchGroup; + this.instanceName = instanceName; + this.dispatchMode = dispatchMode; + this.dispatcherCallback = dispatcherCallback; + this.cache = cache; + + //一个分片可以确保我的消息都是顺序的 + RocketmqUtil.createTopic(nameServer, this.voteTopic, 1, CLUSTER_NAME, false); + RocketmqUtil.createTopic(nameServer, this.dispatchGroup, 1, CLUSTER_NAME, false); + + this.className = Thread.currentThread().getStackTrace()[2].getClassName(); + this.methodName = Thread.currentThread().getStackTrace()[2].getMethodName(); + } + + @Override + public void start() throws Exception { + try { + if (this.isStart.compareAndSet(false, true)) { + if (this.messageSender == null) { + //用于向message topic 发送消息 + this.messageSender = RocketmqUtil.initDefaultMQProducer(this.nameServer, this.instanceName, this.dispatchGroup); + LOGGER.info("[{}] MessageSender_Init_Success", this.instanceName); + } + if (this.messageListener == null) { + //监听message, 如果有消息过来, 判断过来的消息是启动还是停止, 根据消息的种类去调用不同的任务 + this.messageListener = RocketmqUtil.initDefaultMQPushConsumer(this.nameServer, this.instanceName, this.dispatchGroup + "_" + this.instanceName, this.dispatchGroup); + this.messageListener.registerMessageListener((MessageListenerConcurrently) (msgs, context) -> { + try { + for (MessageExt ext : msgs) { + String[] messageTags = ext.getTags().split("_"); + + String tags = messageTags[0]; + String requestId = messageTags[1]; + long timestamp = messageTags.length > 2 ? Long.parseLong(messageTags[2]) : 0; + if (timestamp < lastTimestamp) { + LOGGER.info("[{}] Dispatcher_Mapper_Newer_Than_Last_({}<{})", this.instanceName, timestamp, lastTimestamp); + continue; + } + lastTimestamp = timestamp; + + String receivedMessage = new String(ext.getBody(), StandardCharsets.UTF_8); + DispatcherMapper receivedDispatcherMapper = DispatcherMapper.parse(receivedMessage); + String currentMessage = this.cache.getKeyConfig(this.dispatchGroup + NAMESPACE_CONFIG_SUFFIX, MAPPER_KEY); + DispatcherMapper currentDispatcherMapper = DispatcherMapper.parse(currentMessage); + + if (currentMessage != null && !receivedDispatcherMapper.equals(currentDispatcherMapper)) { + //接收到的信息和中心存储的不一样,说明目前接受到的不是最新的,不做处理,继续等待 + continue; + } + LOGGER.info("[{}] Received_Message({})_Local({})_Tags({})", this.instanceName, receivedMessage, String.join(",", localCache), tags); + Set taskSet = receivedDispatcherMapper.getTasks(this.instanceName); + if ("stop".equals(tags)) { + //执行停止指令 + List needStop = (List) CollectionUtils.subtract(localCache, taskSet); + if (!needStop.isEmpty()) { + List stopSuccess = this.dispatcherCallback.stop(Lists.newArrayList(needStop)); + if (stopSuccess != null && !stopSuccess.isEmpty()) { + stopSuccess.forEach(localCache::remove); + } + LOGGER.info("[{}] Received_Stop_Message_NeedStop({})_Stopped({})_From({}.{})_({})", this.instanceName, String.join(",", needStop), stopSuccess == null ? "" : String.join(",", stopSuccess), className, methodName, requestId); + } + this.cache.putKeyConfig(this.dispatchGroup + NAMESPACE_STATUS_SUFFIX, this.instanceName, "stop"); + } else if ("start".equals(tags)) { + //执行启动 + List needStart = (List) CollectionUtils.subtract(taskSet, localCache); + if (!needStart.isEmpty()) { + //启动前,重新加载一下内存中的任务实例 + this.dispatcherCallback.list(); + List startSuccess = this.dispatcherCallback.start(Lists.newArrayList(needStart)); + if (startSuccess != null && !startSuccess.isEmpty()) { + localCache.addAll(startSuccess); + } + LOGGER.info("[{}] Received_Start_Message_NeedStart({})_Started({})_FROM({}.{})_({})", this.instanceName, String.join(",", needStart), startSuccess == null ? "" : String.join(",", startSuccess), className, methodName, requestId); + } + this.cache.putKeyConfig(this.dispatchGroup + NAMESPACE_STATUS_SUFFIX, this.instanceName, "start"); + } + } + } catch (Exception e) { + LOGGER.error("[{}] Received_Message_Error", this.instanceName, e); + } + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; + }); + + LOGGER.info("[{}] MessageListener_Init_Success", this.instanceName); + } + + if (this.voteListener == null) { + this.voteListener = RocketmqUtil.initDefaultMQPushConsumer(this.nameServer, this.instanceName, this.dispatchGroup, this.voteTopic); + DefaultMQPushConsumerImpl impl = ReflectUtil.getDeclaredField(this.voteListener, "defaultMQPushConsumerImpl"); + ReflectUtil.setBeanFieldValue(impl, "rebalanceImpl", new RebalancePushImpl(impl) { + @Override + public boolean doRebalance(boolean isOrder) { + super.doRebalance(isOrder); + String requestId = UUID.randomUUID().toString(); + try { + doDispatch(requestId, mQClientFactory, 0); + return true; + } catch (Exception e) { + LOGGER.error("[{}] Dispatcher_Execute_Error_({})", instanceName, requestId, e); + return false; + } + } + }); + LOGGER.info("[{}] VoteListener_Init_Success", this.instanceName); + } + + //重置offset, 从当前时间开始消费 + RocketmqUtil.resetOffsetNew(this.nameServer, this.dispatchGroup + "_" + this.instanceName, this.dispatchGroup, System.currentTimeMillis()); + this.messageSender.start(); + this.messageListener.start(); + this.voteListener.start(); + } + } catch (Exception e) { + this.isStart.set(false); + throw new RuntimeException("Dispatcher_Start_Error", e); + } + } + + public synchronized void doDispatch(String requestId, MQClientInstance mqClientInstance, int depth) throws Exception { + if (depth > 2) { + return; + } + + TreeSet tmpConsumerIdList = Sets.newTreeSet(); + List consumerList = mqClientInstance.findConsumerIdList(this.voteTopic, this.dispatchGroup); + if (consumerList != null && !consumerList.isEmpty()) { + for (String consumerId : consumerList) { + tmpConsumerIdList.add(consumerId.substring(consumerId.indexOf("@") + 1)); + } + boolean isConsumerChange = !consumerIdList.equals(tmpConsumerIdList); + LOGGER.info("[{}] Cluster_CurrentInstanceList({})_NewInstanceList({})_IsChanged({})_({})", instanceName, String.join(",", consumerIdList), String.join(",", tmpConsumerIdList), isConsumerChange, requestId); + if (isConsumerChange) { + //当consumer发生变化时, 触发选主操作 + consumerIdList = tmpConsumerIdList; + if (!tmpConsumerIdList.isEmpty()) { + String masterInstance = tmpConsumerIdList.iterator().next(); + //选主成功 + if (instanceName.equals(masterInstance)) { + isMaster = true; + LOGGER.info("[{}] Master", instanceName); + } else { + LOGGER.info("[{}] Slave", instanceName); + } + } + } + if (isMaster) { + if (!consumerIdList.isEmpty()) { + TreeSet tasks = Sets.newTreeSet(this.dispatcherCallback.list());//此逻辑必须放在选主之前,因为所有实例都需要加载任务 + boolean isTaskChange = !taskList.equals(tasks); + LOGGER.info("[{}] Task_CurrentTask({})_NewTask({})_IsChanged({})_({})", instanceName, String.join(", ", taskList), String.join(",", tasks), isTaskChange, requestId); + if (isTaskChange || isConsumerChange) { + //计算新的调度公式 + IStrategy iStrategy = StrategyFactory.getStrategy(this.dispatchMode); + if (iStrategy instanceof LeastStrategy) {//如果是Least策略,则需要通过cache获取到当前的调度状态 + String currentMessage = this.cache.getKeyConfig(this.dispatchGroup + NAMESPACE_CONFIG_SUFFIX, MAPPER_KEY); + if (currentMessage != null && !currentMessage.isEmpty()) { + DispatcherMapper currentDispatcherMapper = DispatcherMapper.parse(currentMessage); + ((LeastStrategy) iStrategy).setCurrentDispatcherMapper(currentDispatcherMapper); + } + } + DispatcherMapper dispatcherMapper = iStrategy.dispatch(Lists.newArrayList(tasks), Lists.newArrayList(consumerIdList)); + String currentMessage = this.cache.getKeyConfig(this.dispatchGroup + NAMESPACE_CONFIG_SUFFIX, MAPPER_KEY); + DispatcherMapper currentDispatcherMapper = DispatcherMapper.parse(currentMessage); + if (currentMessage != null && dispatcherMapper.equals(currentDispatcherMapper)) { + LOGGER.info("[{}] Dispatcher_Result_Not_Changed_({})", this.instanceName, requestId); + } else { + LOGGER.info("[{}] Dispatcher_Result_Changed_CurrentResult({})_NewResult({})_({})", this.instanceName, currentMessage == null ? "{}" : currentMessage, dispatcherMapper, requestId); + //确认需要推送新的调度列表 + this.cache.putKeyConfig(this.dispatchGroup + NAMESPACE_CONFIG_SUFFIX, MAPPER_KEY, dispatcherMapper.toString()); + } + + long currentTime = System.currentTimeMillis(); + //推送stop指令 + this.messageSender.send(new Message(this.dispatchGroup, "stop_" + requestId + "_" + currentTime, dispatcherMapper.toString().getBytes(StandardCharsets.UTF_8)), new SendCallback() { + @Override + public void onSuccess(SendResult sendResult) { + LOGGER.info("[{}] Send_Stop_Message_({}.{}.{}_{})", instanceName, className, methodName, depth, requestId); + } + + @Override + public void onException(Throwable e) { + LOGGER.error("[{}] Send_Stop_Message_Error_({}.{}.{}_{})", instanceName, className, methodName, depth, requestId, e); + } + }); + + //当检查到所有节点都已经完成上一轮的同步, 推送start停止指令 + boolean ifWait = pushWait(requestId, mqClientInstance, Lists.newArrayList(consumerIdList), depth); + if (ifWait) { + Message message = new Message(this.dispatchGroup, "start_" + requestId + "_" + currentTime, dispatcherMapper.toString().getBytes(StandardCharsets.UTF_8)); + this.messageSender.send(message, new SendCallback() { + @Override + public void onSuccess(SendResult sendResult) { + LOGGER.info("[{}] Send_Start_Message_({}.{}.{}_{})", instanceName, className, methodName, depth, requestId); + } + + @Override + public void onException(Throwable e) { + LOGGER.error("[{}] Send_Start_Message_Error_({}.{}.{}_{})", instanceName, className, methodName, depth, requestId, e); + } + }); + //调度完成后重置taskList + taskList = tasks; + } + } + } + } + } else { + LOGGER.warn("[{}] ConsumerList_Empty", instanceName); + } + + } + + private boolean pushWait(String requestId, MQClientInstance mQClientFactory, List consumerIdList, int depth) throws Exception { + boolean pushTag = false; + //发送之前需要先查看各个客户端是否已经结束了上一轮的调度 + for (int i = 0; i < 3; i++) { + boolean stateTag = true; + for (String consumeId : consumerIdList) { + String instanceId = consumeId.substring(consumeId.indexOf("@") + 1); + String localStatus = this.cache.getKeyConfig(this.dispatchGroup + NAMESPACE_STATUS_SUFFIX, instanceId); + if (localStatus == null || !localStatus.equals("stop")) { + stateTag = false; + LOGGER.info("[{}] Instance_Status_({}-{})", instanceName, instanceId, localStatus); + break; + } + } + if (stateTag) { + pushTag = true; + break; + } else { + Thread.sleep(5000); + } + } + if (!pushTag) { + doDispatch(requestId, mQClientFactory, depth + 1); + } + return pushTag; + } + + @Override + public void wakeUp() throws Exception { + MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(this.voteListener); + mqClientInstance.rebalanceImmediately(); + } + + @Override + public void close() { + try { + if (this.isStart.compareAndSet(true, false)) { + if (this.voteListener != null) { + this.voteListener.shutdown(); + this.voteListener = null; + } + if (this.messageSender != null) { + this.messageSender.shutdown(); + this.messageSender = null; + } + if (this.messageListener != null) { + this.messageListener.shutdown(); + this.messageListener = null; + } + if (this.consumerIdList != null) { + this.consumerIdList.clear(); + } + if (this.localCache != null) { + this.localCache.clear(); + } + } + } catch (Exception e) { + this.isStart.set(true); + throw new RuntimeException("Dispatcher_Close_Error", e); + } + } + +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/AllStrategy.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/AllStrategy.java new file mode 100644 index 00000000..f74c14bd --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/AllStrategy.java @@ -0,0 +1,50 @@ +/* + * 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.dispatcher.strategy; + +import com.google.common.collect.Lists; +import java.util.List; +import org.apache.rocketmq.streams.dispatcher.IStrategy; +import org.apache.rocketmq.streams.dispatcher.entity.DispatcherMapper; + +public class AllStrategy implements IStrategy { + public static void main(String[] args) { + List tasks = Lists.newArrayList("t1", "t2", "t3"); + List instances = Lists.newArrayList("ins1", "ins2", "ins3", "ins4", "ins5"); + AllStrategy allStrategy = new AllStrategy(); + + try { + DispatcherMapper dispatcherMapper = allStrategy.dispatch(tasks, instances); + for (String instance : instances) { + System.out.println(instance + ":" + dispatcherMapper.getTasks(instance)); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + + } + + @Override public DispatcherMapper dispatch(List tasks, List instances) throws Exception { + DispatcherMapper dispatcherMapper = new DispatcherMapper(); + for (String instance : instances) { + for (String task : tasks) { + dispatcherMapper.putTask(instance, task); + } + } + return dispatcherMapper; + } +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/AveragelyStrategy.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/AveragelyStrategy.java new file mode 100644 index 00000000..75765a09 --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/AveragelyStrategy.java @@ -0,0 +1,68 @@ +/* + * 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.dispatcher.strategy; + +import com.google.common.collect.Lists; +import java.util.Collections; +import java.util.List; +import org.apache.rocketmq.streams.dispatcher.IStrategy; +import org.apache.rocketmq.streams.dispatcher.entity.DispatcherMapper; + +public class AveragelyStrategy implements IStrategy { + + public static void main(String[] args) { + List tasks = Lists.newArrayList("sas_proc->0", "sas_proc->1"); + List instances = Lists.newArrayList("ins1", "ins2"); + AveragelyStrategy averagelyStrategy = new AveragelyStrategy(); + + try { + DispatcherMapper dispatcherMapper = averagelyStrategy.dispatch(tasks, instances); + System.out.println(dispatcherMapper.getTasks("ins1")); + System.out.println(dispatcherMapper.getTasks("ins2")); + } catch (Exception e) { + throw new RuntimeException(e); + } + + } + + @Override public DispatcherMapper dispatch(List tasks, List instances) throws Exception { + DispatcherMapper dispatcherMapper = new DispatcherMapper(); + Collections.sort(tasks); + Collections.sort(instances); + for (String instance : instances) { + List task4Instance = average(instance, tasks, instances); + for (String task : task4Instance) { + dispatcherMapper.putTask(instance, task); + } + } + return dispatcherMapper; + } + + private List average(String instanceId, List tasks, List instances) throws Exception { + List result = Lists.newArrayList(); + int index = instances.indexOf(instanceId); + int mod = tasks.size() % instances.size(); + int averageSize = tasks.size() <= instances.size() ? 1 : (mod > 0 && index < mod ? tasks.size() / instances.size() + 1 : tasks.size() / instances.size()); + int startIndex = (mod > 0 && index < mod) ? index * averageSize : index * averageSize + mod; + int range = Math.min(averageSize, tasks.size() - startIndex); + for (int i = 0; i < range; i++) { + result.add(tasks.get((startIndex + i) % tasks.size())); + } + return result; + } + +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/ConsistencyStrategy.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/ConsistencyStrategy.java new file mode 100644 index 00000000..dd302ea8 --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/ConsistencyStrategy.java @@ -0,0 +1,81 @@ +/* + * 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.dispatcher.strategy; + +import java.util.List; +import java.util.SortedMap; +import java.util.TreeMap; +import org.apache.rocketmq.streams.dispatcher.IStrategy; +import org.apache.rocketmq.streams.dispatcher.entity.DispatcherMapper; + +public class ConsistencyStrategy implements IStrategy { + + private static String getWorker(SortedMap sortedMap, String key) { + //得到该key的hash值 + int hash = getHash(key); + //得到大于该Hash值的所有Map + String worker; + SortedMap subMap = sortedMap.tailMap(hash); + if (subMap.isEmpty()) { + //如果没有比该key的hash值大的,则从第一个node开始 + Integer i = sortedMap.firstKey(); + //返回对应的服务器 + worker = sortedMap.get(i); + } else { + //第一个Key就是顺时针过去离node最近的那个结点 + Integer i = subMap.firstKey(); + //返回对应的服务器 + worker = subMap.get(i); + } + return worker; + } + + //使用FNV1_32_HASH算法计算服务器的Hash值 + private static int getHash(String str) { + final int p = 16777619; + int hash = (int) 2166136261L; + for (int i = 0; i < str.length(); i++) { + hash = (hash ^ str.charAt(i)) * p; + } + hash += hash << 13; + hash ^= hash >> 7; + hash += hash << 3; + hash ^= hash >> 17; + hash += hash << 5; + + // 如果算出来的值为负数则取其绝对值 + if (hash < 0) { + hash = Math.abs(hash); + } + return hash; + } + + @Override public DispatcherMapper dispatch(List tasks, List instances) throws Exception { + DispatcherMapper dispatcherMapper = new DispatcherMapper(); + SortedMap sortedMap = new TreeMap<>(); + for (String instanceId : instances) { + for (int i = 1; i <= 1000; i++) { + int hash = getHash(instanceId);//nodeName.hashCode(); + sortedMap.put(hash, instanceId); + } + } + for (int i = 0; i < tasks.size(); i++) { + dispatcherMapper.putTask(getWorker(sortedMap, i + ""), tasks.get(i)); + } + return dispatcherMapper; + } +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/HashStrategy.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/HashStrategy.java new file mode 100644 index 00000000..3c019c72 --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/HashStrategy.java @@ -0,0 +1,50 @@ +/* + * 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.dispatcher.strategy; + +import com.google.common.collect.Lists; +import java.util.List; +import org.apache.rocketmq.streams.dispatcher.IStrategy; +import org.apache.rocketmq.streams.dispatcher.entity.DispatcherMapper; + +public class HashStrategy implements IStrategy { + + public static void main(String[] args) { + List tasks = Lists.newArrayList("test1", "test2"); + List instances = Lists.newArrayList("ins1", "ins2", "ins3", "ins4", "ins5"); + HashStrategy averagelyStrategy = new HashStrategy(); + + try { + DispatcherMapper dispatcherMapper = averagelyStrategy.dispatch(tasks, instances); + + System.out.println(dispatcherMapper); + } catch (Exception e) { + throw new RuntimeException(e); + } + + } + + @Override public DispatcherMapper dispatch(List tasks, List instances) throws Exception { + DispatcherMapper dispatcherMapper = new DispatcherMapper(); + int instanceCnt = instances.size(); + for (int i = 0; i < tasks.size(); i++) { + String instanceId = instances.get(i % instanceCnt); + dispatcherMapper.putTask(instanceId, tasks.get(i)); + } + return dispatcherMapper; + } +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/LeastStrategy.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/LeastStrategy.java new file mode 100644 index 00000000..6a13e35f --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/LeastStrategy.java @@ -0,0 +1,204 @@ +/* + * 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.dispatcher.strategy; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.rocketmq.streams.dispatcher.IStrategy; +import org.apache.rocketmq.streams.dispatcher.constant.DispatcherConstant; +import org.apache.rocketmq.streams.dispatcher.entity.DispatcherMapper; + +public class LeastStrategy implements IStrategy { + + private DispatcherMapper currentDispatcherMapper = new DispatcherMapper(); + + public static void main(String[] args) { + List tasks = Lists.newArrayList("cloud_siem_vpc_slb7l_log_1335759343513432_0$#0", + "cloud_siem_aegis_vul_log_0$#0", "cloud_siem_aegis_crack_from_beaver_0$#0", "cloud_siem_aegis_proc_snapshot_0$#0", + "cloud_siem_polardb1x_audit_1154554141560953_0$#0", "cloud_siem_aegis_proc_0$#0", + "cloud_siem_aegis_dns_0$#0", + "cloud_siem_cfw_flow_1335759343513432_0$#0", + "cloud_siem_polardb2x_audit_1335759343513432_0$#0", + "cloud_siem_aegis_health_check_0$#0", + "cloud_siem_vpc_flow_log_1335759343513432_0$#0", + "cloud_siem_aegis_netstat_0$#0", + "cloud_siem_polardb2x_audit_1154554141560953_0$#0", + "cloud_siem_aegis_sas_alert_0$#0", + "cloud_siem_waf_flow_1335759343513432_0$#0", + "cloud_siem_oss_access_log_1335759343513432_0$#0", + "cloud_siem_vpc_eni_log_1335759343513432_0$#0", + "cloud_siem_aegis_login_event_0$#0", + "cloud_siem_cfw_alert_1335759343513432$#0", + "cloud_siem_polardb1x_audit_1335759343513432_0$#0", + "cloud_siem_aegis_netinfo_snapshot_0$#0", + "cloud_siem_cfw_alert_1335759343513432_0$#0"); + List instances = Lists.newArrayList("10_219_220_145_4129", "10_219_223_109_6190", "10_219_220_103_5984", "10_219_220_106_6049"); + LeastStrategy leastStrategy = new LeastStrategy(); + DispatcherMapper dispatcherMapper = DispatcherMapper.parse("{\n" + + " \"10_219_220_145_4129\": [\n" + + " \"cloud_siem_polardb2x_audit_1335759343513432_0$#0\",\n" + + " \"cloud_siem_vpc_eni_log_1335759343513432_0$#0\",\n" + + " \"cloud_siem_vpc_flow_log_1335759343513432_0$#0\",\n" + + " \"cloud_siem_vpc_slb7l_log_1335759343513432_0$#0\",\n" + + " \"cloud_siem_waf_flow_1335759343513432_0$#0\"\n" + + " ],\n" + + " \"10_219_220_106_6050\": [\n" + + " \"cloud_siem_aegis_login_event_0$#0\",\n" + + " \"cloud_siem_aegis_netstat_0$#0\",\n" + + " \"cloud_siem_aegis_proc_0$#0\",\n" + + " \"cloud_siem_aegis_vul_log_0$#0\",\n" + + " \"cloud_siem_cfw_alert_1335759343513432_0$#0\"\n" + + " ],\n" + + " \"10_219_223_109_6190\": [\n" + + " \"cloud_siem_aegis_crack_from_beaver_0$#0\",\n" + + " \"cloud_siem_aegis_dns_0$#0\",\n" + + " \"cloud_siem_cfw_alert_1335759343513432$#0\",\n" + + " \"cloud_siem_polardb2x_audit_1154554141560953_0$#0\"\n" + + " ],\n" + + " \"10_219_220_103_5984\": [\n" + + " \"cloud_siem_aegis_proc_snapshot_0$#0\",\n" + + " \"cloud_siem_aegis_sas_alert_0$#0\",\n" + + " \"cloud_siem_cfw_flow_1335759343513432_0$#0\",\n" + + " \"cloud_siem_polardb1x_audit_1335759343513432_0$#0\"\n" + + " ],\n" + + " \"10_219_220_106_6049\": [\n" + + " \"cloud_siem_aegis_health_check_0$#0\",\n" + + " \"cloud_siem_aegis_netinfo_snapshot_0$#0\",\n" + + " \"cloud_siem_oss_access_log_1335759343513432_0$#0\",\n" + + " \"cloud_siem_polardb1x_audit_1154554141560953_0$#0\"\n" + + " ]\n" + + "}"); + leastStrategy.setCurrentDispatcherMapper(dispatcherMapper); + + try { + DispatcherMapper newDispatcher = leastStrategy.dispatch(tasks, instances); + System.out.println(newDispatcher); + } catch (Exception e) { + throw new RuntimeException(e); + } + +// int i = 0; +// for (String task : tasks) { +// try { +// DispatcherMapper newDispatcher = leastStrategy.dispatch(Lists.newArrayList(task), instances); +// i++; +// for (String instance : instances) { +// System.out.println("第" + i + "波" + instance + ":" + newDispatcher.getTasks(instance)); +// } +// leastStrategy.setCurrentDispatcherMapper(newDispatcher); +// } catch (Exception e) { +// throw new RuntimeException(e); +// } +// } + } + + public DispatcherMapper getCurrentDispatcherMapper() { + return currentDispatcherMapper; + } + + public void setCurrentDispatcherMapper(DispatcherMapper currentDispatcherMapper) { + this.currentDispatcherMapper = currentDispatcherMapper; + } + + @Override public DispatcherMapper dispatch(List tasks, List instances) throws Exception { + if (tasks == null || tasks.isEmpty() || instances == null || instances.isEmpty()) { + return new DispatcherMapper(); + } + + int avg = 1; + if (tasks.size() > instances.size()) { + avg = tasks.size() / instances.size() + 1; + } + + Set currentInstances = this.currentDispatcherMapper.getInstances(); + List needRemoveInstance = Lists.newArrayList(); + Map needRemoveTask = Maps.newHashMap(); + for (String instance : currentInstances) { + if (!instances.contains(instance)) { + needRemoveInstance.add(instance); + } else { + Set ts = this.currentDispatcherMapper.getTasks(instance); + for (String s : ts) { + if (!tasks.contains(s)) { + needRemoveTask.put(s, instance); + } + } + if (ts.size() > avg) {//如果一个实例上的任务数超过平均值,也需要重新调度 + Iterator iterator = ts.iterator(); + for (int i = 0; i < ts.size() - avg; i++) { + needRemoveTask.put(iterator.next(), instance); + } + } + + } + } + for (String rm : needRemoveInstance) { + this.currentDispatcherMapper.remove(rm); + } + for (Map.Entry entry : needRemoveTask.entrySet()) { + this.currentDispatcherMapper.removeTask(entry.getValue(), entry.getKey()); + } + + //已经调度的任务,不做改变 + List needDispatcher = Lists.newArrayList(); + for (String task : tasks) { + if (!this.currentDispatcherMapper.containTask(task)) { + needDispatcher.add(task); + } + } + + for (String task : needDispatcher) { + //每个需要调度的任务都需要选择合适的Instance + Map instanceSort = Maps.newHashMap(); + for (String instance : instances) { + instanceSort.put(instance, this.currentDispatcherMapper.getTasks(instance).size()); + } + List> list = Lists.newArrayList(instanceSort.entrySet()); + list.sort(Comparator.comparingInt(Map.Entry::getValue)); + + String currentInstance = null; + for (Map.Entry entry : list) { + //逐个遍历instance + String instance = entry.getKey(); + Set onlineTasks = this.currentDispatcherMapper.getTasks(instance); + boolean available = true; + for (String onlineTask : onlineTasks) { + String taskPrefix = task.indexOf(DispatcherConstant.TASK_SERIAL_NUM_SEPARATOR) > 0 ? task.substring(0, task.indexOf(DispatcherConstant.TASK_SERIAL_NUM_SEPARATOR)) : task; + String onlineTaskPrefix = onlineTask.indexOf(DispatcherConstant.TASK_SERIAL_NUM_SEPARATOR) > 0 ? onlineTask.substring(0, onlineTask.indexOf(DispatcherConstant.TASK_SERIAL_NUM_SEPARATOR)) : onlineTask; + //如果该instance包含的任务有和当前任务前缀相同的, 那该instance就不可以运行该任务 + if (onlineTaskPrefix.equals(taskPrefix)) { + available = false; + break; + } + } + if (available) { + currentInstance = instance; + break; + } + } + if (currentInstance != null) { + this.currentDispatcherMapper.putTask(currentInstance, task); + } + } + return this.currentDispatcherMapper; + } +} diff --git a/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/StrategyFactory.java b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/StrategyFactory.java new file mode 100644 index 00000000..e36d3bb0 --- /dev/null +++ b/rocketmq-streams-dispatcher/src/main/java/org/apache/rocketmq/streams/dispatcher/strategy/StrategyFactory.java @@ -0,0 +1,29 @@ +/* + * 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.dispatcher.strategy; + +import org.apache.rocketmq.streams.dispatcher.IStrategy; +import org.apache.rocketmq.streams.dispatcher.enums.DispatchMode; + +public class StrategyFactory { + + public static IStrategy getStrategy(DispatchMode mode) throws ClassNotFoundException, InstantiationException, IllegalAccessException { + Class strategyClass = (Class) StrategyFactory.class.getClassLoader().loadClass("org.apache.rocketmq.streams.dispatcher.strategy." + mode.getName() + "Strategy"); + return strategyClass.newInstance(); + } + +} diff --git a/rocketmq-streams-examples/README.md b/rocketmq-streams-examples/README.md new file mode 100644 index 00000000..ca220cbf --- /dev/null +++ b/rocketmq-streams-examples/README.md @@ -0,0 +1,204 @@ +## rocketmq-streams-examples + +### 1、File source example + +逐行读取文件数据,并打印出来。 + +```java +public class FileSourceExample { + public static void main(String[] args) { + DataStreamSource source = StreamBuilder.dataStream("namespace", "pipeline"); + source.fromFile("data.txt", false) + .map(message -> message) + .toPrint(1) + .start(); + } +} + +``` + +### 2、分时间段,统计分组中某字段的和 + +#### 2.1 安装 Apache RocketMQ + +可以参考[Apache RocketMQ 搭建文档](https://rocketmq.apache.org/docs/quick-start/) + +#### 2.2 源数据 + +[源数据](./../rocketmq-streams-examples/src/main/resources/data.txt) + +```xml +{"InFlow":"1","ProjectName":"ProjectName-0","LogStore":"LogStore-0","OutFlow":"0"} +{"InFlow":"2","ProjectName":"ProjectName-1","LogStore":"LogStore-1","OutFlow":"1"} +{"InFlow":"3","ProjectName":"ProjectName-2","LogStore":"LogStore-2","OutFlow":"2"} +{"InFlow":"4","ProjectName":"ProjectName-0","LogStore":"LogStore-0","OutFlow":"3"} +{"InFlow":"5","ProjectName":"ProjectName-1","LogStore":"LogStore-1","OutFlow":"4"} +{"InFlow":"6","ProjectName":"ProjectName-2","LogStore":"LogStore-2","OutFlow":"5"} +{"InFlow":"7","ProjectName":"ProjectName-0","LogStore":"LogStore-0","OutFlow":"6"} +{"InFlow":"8","ProjectName":"ProjectName-1","LogStore":"LogStore-1","OutFlow":"7"} +{"InFlow":"9","ProjectName":"ProjectName-2","LogStore":"LogStore-2","OutFlow":"8"} +{"InFlow":"10","ProjectName":"ProjectName-0","LogStore":"LogStore-0","OutFlow":"9"} +``` + +#### 2.3 代码示例 + +[代码示例](./../rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/rocketmqsource/RocketmqWindowTest.java) + +#### 2.4 结果说明 + +这个例子中,使用 rocketmq-streams 消费 rocketmq 中的数据,并按照 ProjectName 和 LogStore 两个字段联合分组统计,两个字段的值相同,分为一组。 +分别统计每组的InFlow和OutFlow两字段累计和。 + +data.text数据运行的结果部分如下: + +```xml +"InFlow":22,"total":4,"ProjectName":"ProjectName-0","LogStore":"LogStore-0","OutFlow":18 +"InFlow":18,"total":3,"ProjectName":"ProjectName-2","LogStore":"LogStore-2","OutFlow":15 +"InFlow":15,"total":3,"ProjectName":"ProjectName-1","LogStore":"LogStore-1","OutFlow":12 +``` + +可见"ProjectName":"ProjectName-0","LogStore":"LogStore-0"分组公有4条数据,"ProjectName":"ProjectName-2","LogStore":"LogStore-2",3条数据。 +"ProjectName":"ProjectName-1","LogStore":"LogStore-1"分组3条数据,总共10条数据。结果与源数据一致。 + +### 3、网页点击统计 + +#### 3.1、数据说明 + +原始数据为resources路径下的[pageClickData.txt](./../rocketmq-streams-examples/src/main/resources/pageClickData.txt) + +第一列是用户id,第二列是用户点击时间,最后一列是网页地址 + +```xml +{"userId":"1","eventTime":"1631700000000","method":"GET","url":"page-1"} +{"userId":"2","eventTime":"1631700030000","method":"POST","url":"page-2"} +{"userId":"3","eventTime":"1631700040000","method":"GET","url":"page-3"} +{"userId":"1","eventTime":"1631700050000","method":"DELETE","url":"page-2"} +{"userId":"1","eventTime":"1631700060000","method":"DELETE","url":"page-2"} +{"userId":"2","eventTime":"1631700070000","method":"POST","url":"page-3"} +{"userId":"3","eventTime":"1631700080000","method":"GET","url":"page-1"} +{"userId":"1","eventTime":"1631700090000","method":"GET","url":"page-2"} +{"userId":"2","eventTime":"1631700100000","method":"PUT","url":"page-3"} +{"userId":"4","eventTime":"1631700120000","method":"POST","url":"page-1"} +``` + +#### 3.1、统计某段时间窗口内用户点击网页次数 + +[代码示例](./../rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/pageclick/UsersDimension.java) + +结果: + +```xml +{"start_time":"2021-09-15 18:00:00","total":1,"windowInstanceId":"SPVGTV6DaXmxV5mGNzQixQ==","offset":53892061100000001,"end_time":"2021-09-15 18:01:00","userId":"2"} +{"start_time":"2021-09-15 18:00:00","total":1,"windowInstanceId":"dzAZ104qjUAwzTE6gbKSPA==","offset":53892061100000001,"end_time":"2021-09-15 18:01:00","userId":"3"} +{"start_time":"2021-09-15 18:00:00","total":2,"windowInstanceId":"wrTTyU5DiDkrAb6669Ig9w==","offset":53892061100000001,"end_time":"2021-09-15 18:01:00","userId":"1"} +{"start_time":"2021-09-15 18:01:00","total":1,"windowInstanceId":"vabkmx14xHsJ7G7w16vwug==","offset":53892121100000001,"end_time":"2021-09-15 18:02:00","userId":"3"} +{"start_time":"2021-09-15 18:01:00","total":2,"windowInstanceId":"YIgEKptN2Wf+Oq2m8sEcYw==","offset":53892121100000001,"end_time":"2021-09-15 18:02:00","userId":"2"} +{"start_time":"2021-09-15 18:01:00","total":2,"windowInstanceId":"iYKnwMYAzXFJYbO1KvDnng==","offset":53892121100000001,"end_time":"2021-09-15 18:02:00","userId":"1"} +{"start_time":"2021-09-15 18:02:00","total":1,"windowInstanceId":"HBojuU6/2F/6llkyefECxw==","offset":53892181100000001,"end_time":"2021-09-15 18:03:00","userId":"4"} +``` + +在时间范围 18:00:00- 18:01:00内: + +| userId | 点击次数 | +|--------|------| +| 1 | 2 | +| 2 | 1 | +| 3 | 1 | + +在时间范围 18:01:00- 18:02:00内: + +| userId | 点击次数 | +|--------|------| +| 1 | 2 | +| 2 | 2 | +| 3 | 1 | + +在时间范围 18:02:00- 18:03:00内: + +| userId | 点击次数 | +|--------|------| +| 4 | 1 | + +可查看原数据文件,eventTime为时间字段,简单检查后上述结果与预期相符合。 + +#### 3.2、统计某段时间窗口内,被点击次数最多的网页 + +[代码示例](./../rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/pageclick/PageDimension.java) + +运行结果: + +```xml +{"start_time":"2021-09-15 18:00:00","total":1,"windowInstanceId":"wrTTyU5DiDkrAb6669Ig9w==","offset":53892061100000001,"end_time":"2021-09-15 18:01:00","url":"page-1"} +{"start_time":"2021-09-15 18:00:00","total":2,"windowInstanceId":"seECZRcaQSRsET1rDc6ZAw==","offset":53892061100000001,"end_time":"2021-09-15 18:01:00","url":"page-2"} +{"start_time":"2021-09-15 18:00:00","total":1,"windowInstanceId":"dzAZ104qjUAwzTE6gbKSPA==","offset":53892061100000001,"end_time":"2021-09-15 18:01:00","url":"page-3"} +{"start_time":"2021-09-15 18:01:00","total":2,"windowInstanceId":"uCqvAeaLTYRnjQm8dCZOvw==","offset":53892121100000001,"end_time":"2021-09-15 18:02:00","url":"page-2"} +{"start_time":"2021-09-15 18:01:00","total":3,"windowInstanceId":"vabkmx14xHsJ7G7w16vwug==","offset":53892121100000001,"end_time":"2021-09-15 18:02:00","url":"page-3"} +{"start_time":"2021-09-15 18:02:00","total":1,"windowInstanceId":"NdgwYMT8azNMu55NUIvygg==","offset":53892181100000001,"end_time":"2021-09-15 18:03:00","url":"page-1"} + +``` + +在时间窗口18:00:00 - 18:01:00 内,有4条数据; + +在时间窗口18:01:00 - 18:02:00 内,有5条数据; + +在时间窗口18:02:00 - 18:03:00 内,有1条数据; + +分钟统计窗口内,被点击次数最多的网页. +得到上述数据后,需要按照窗口进行筛选最大值,需要再次计算。 +代码: + +```java + public void findMax() { + DataStreamSource source = StreamBuilder.dataStream("ns-1", "pl-1"); + source.fromFile("/home/result.txt", false) + .map(message -> JSONObject.parseObject((String) message)) + .window(TumblingWindow.of(Time.seconds(5))) + .groupBy("start_time","end_time") + .max("total") + .waterMark(1) + .setLocalStorageOnly(true) + .toDataSteam() + .toPrint(1) + .start(); + } + +``` + +得到结果: + +```xml +{"start_time":"2021-09-17 11:09:35","total":"2","windowInstanceId":"kRRpe2hPEQtEuTkfnXUaHg==","offset":54040181100000001,"end_time":"2021-09-17 11:09:40"} +{"start_time":"2021-09-17 11:09:35","total":"3","windowInstanceId":"kRRpe2hPEQtEuTkfnXUaHg==","offset":54040181100000002,"end_time":"2021-09-17 11:09:40"} +{"start_time":"2021-09-17 11:09:35","total":"1","windowInstanceId":"kRRpe2hPEQtEuTkfnXUaHg==","offset":54040181100000003,"end_time":"2021-09-17 11:09:40"} +``` + +可以得到三个窗口中网页点击次数最多分别是2次,1次,3次。 + +### 4、Rocketmq-streams 多客户端消费 + +#### 4.1、数据说明 + +源数据由[data.txt](./../rocketmq-streams-examples/src/main/resources/data.txt)组成,反复发送100遍,总共生产1000条数据。 + +#### 4.2、代码实例 + +[代码示例](./../rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/mutilconsumer/MutilStreamsClientTest.java) + +代码中读取data.txt文件反复发送100遍,发送1000条数据。同时,开启两个消费者,每个消费者独立消费数据,然后做window聚合。 +代码目的: +通过两个独立消费者,组成消费者组,同时消费相同topic数据,达到当1个消费者消费不过来时横向扩容的效果,通过打印出来"total"字段值的和判断两个消费者是否总共消费了1000条数据。 + +#### 4.3、结果说明 + +结果数据下所示,可计算各行total对应值之和为1000,表明的却两个消费者达到了并发消费的效果,计算无误,达到了扩容目的。 + +```xml + +{"start_time":"2021-09-27 14:10:10","InFlow":1144,"total":208,"windowInstanceId":"gYZ3tv/5ohgHrwF6tIFgoQ==","offset":54915025100000001,"ProjectName":"ProjectName-0","LogStore":"LogStore-0","end_time":"2021-09-27 14:10:20","OutFlow":936} +{"start_time":"2021-09-27 14:10:10","InFlow":936,"total":156,"windowInstanceId":"gYZ3tv/5ohgHrwF6tIFgoQ==","offset":54915025100000002,"ProjectName":"ProjectName-2","LogStore":"LogStore-2","end_time":"2021-09-27 14:10:20","OutFlow":780} +{"start_time":"2021-09-27 14:10:10","InFlow":780,"total":156,"windowInstanceId":"gYZ3tv/5ohgHrwF6tIFgoQ==","offset":54915025100000003,"ProjectName":"ProjectName-1","LogStore":"LogStore-1","end_time":"2021-09-27 14:10:20","OutFlow":624} +{"start_time":"2021-09-27 14:10:20","InFlow":1056,"total":192,"windowInstanceId":"4YnbFAgSzeDt5qpo+Is/5w==","offset":54915035100000001,"ProjectName":"ProjectName-0","LogStore":"LogStore-0","end_time":"2021-09-27 14:10:30","OutFlow":864} +{"start_time":"2021-09-27 14:10:20","InFlow":720,"total":144,"windowInstanceId":"4YnbFAgSzeDt5qpo+Is/5w==","offset":54915035100000002,"ProjectName":"ProjectName-1","LogStore":"LogStore-1","end_time":"2021-09-27 14:10:30","OutFlow":576} +{"start_time":"2021-09-27 14:10:20","InFlow":864,"total":144,"windowInstanceId":"4YnbFAgSzeDt5qpo+Is/5w==","offset":54915035100000003,"ProjectName":"ProjectName-2","LogStore":"LogStore-2","end_time":"2021-09-27 14:10:30","OutFlow":720} + +``` \ No newline at end of file diff --git a/rocketmq-streams-examples/pom.xml b/rocketmq-streams-examples/pom.xml index da6a130b..3b18bee6 100644 --- a/rocketmq-streams-examples/pom.xml +++ b/rocketmq-streams-examples/pom.xml @@ -15,27 +15,19 @@ See the License for the specific language governing permissions and limitations under the License. --> - rocketmq-streams org.apache.rocketmq - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT 4.0.0 rocketmq-streams-examples ROCKETMQ STREAMS :: examples - - UTF-8 - ${file_encoding} - 8 - 8 - - - org.apache.rocketmq diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/Test.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/Test.java new file mode 100644 index 00000000..dd2b9f38 --- /dev/null +++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/Test.java @@ -0,0 +1,45 @@ +/* + * 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.examples; + +import org.apache.rocketmq.streams.common.utils.AESUtil; + +public class Test { + + public static void main(String[] args) { + String string = "{\"updateFlag\":\"0\",\"pipelineNames\":\"[]\",\"needThread\":\"false\",\"className\":\"org.apache.rocketmq.streams.tasks.DispatcherTask\",\"nameSpace\":\"cloud-siem-collector\",\"configureName\":\"cloud_siem_collector\",\"state\":\"started\",\"type\":\"stream_task\",\"version\":\"1.0\"}"; + + String jsonValue = "ddMM/yBUmmMsQZqhM59TsyvACcOJRib0cnDl2Y66cHbMLHTpdnCGEN3/mYUv0rjfiFlysmHRgYoMWqhAtA7CkYlLGEAPFop0lI5D/tnkBFPxa/hrw5LhUpZk+QITdtF8qGNBLf2g/yrNWqtmra4gzEbltPQw/zVYK2oRFqf8UPsBOnQhF55lw2iE7T6D7MVqFllTSj5MGKijPPJqLIREFW+4vg4ofhnnDsxDjF5qd94//mJNwOBWHcQ86OcIUq8pZyypLHjzLJLbx37h6BmI2XMBjmrp0LLKvX8yPJ5iooOkpWwvQJf3UhqAHdhjv84Ei0YqnbfhajS5v7NQAVLS+HJMF2rhh21hQUpkiT7YoJcIFf8tljJofY6+Q+4i3apf"; +// String jsonValue2 = "63p+wGuaPl8RrC2LWAsjWL01FHwE5Ts9Vau27xLCiE2mSmcWc14fJiKHqh8eIwn38GSwB54REvo0E0YwQOM5ubBLgBXTQVrTieWCed0dnbh0iFhvU2GcXb3tIBgjc0uUjR23pdbettqY4MLzfEi3U9XSSC54rSsmOAY4JC21FMOaa/mG5CLQDh0G6zssK5QlT+AoajMhUiFpmOigc22ItYtrC5Pe6QeK6u8YBDYZMF+1jLdqAWSwgzUnvlel7EXjd8vFpYbKvOaqNcIIL7otDFxLWyxcy+vmwE8sLUnARokTVO8KeXVCfwIGK8SOngNFfWsR/LpBjJdpuxzhHTl2wConMU0qJrkvfsc0AC4dVLJ7zDmtKJkBy8HH3aBgnyfLcStKRKanbJuAN2HAod3Jzj9ffx3gwqTbS9DuaeOAm2bWfpNHjTT5GPOjWwREOLypxf/BMyQtBScXbxFQ7mcMsAzSQ9kGBwsmlUqamuXzRQ1tihIf55sGvcdXzOi91ogqunmUa53kTlJRr1uFzFp04q/X7CJlyMhoYdUiGl6PA2RnxnpaOZ32ayncSgShMTfVhIrt4LxVy5iunQQXw5Hf/zPfUpWODlF3GpzhSrtt/zXaXR+ZJyLfbWg0dG2EWAlwZe5UP84aK5sut2bAgxHNBUehFV4FQr2oOIWiJWfQueoxYRUTLoZUlcxyD6tt0fh/W22Lig1Aa84fBASA1QKRQzrZDT6Ul/ZEs0OYw3DgdxuEP27UIlW5fo99xnDrVujqyZ49gAZkLg1PQZ48pL8/e1DMGJdyFV0ESS3RxT4+sBZbrBO2vkCuP0JgO1GWdzoAsStomav7vQ0KU1SKimXBYmZQNhSjrfUx7bLJSty+dxx/odSdXQAYZzJ7MKuSigD4uIomlgaCJe/eCJoxnlJjN4WSw+00yYLfgfBmW5rtr6VJu28nXUYUyOp2x0uD1+nmmLYK/2WhYwuVd//mEsdgR0UTcHtDNPB+FFZ+AQqakiuC2m5H6whY7WD2LzJqgWvPOxfSDy8iVq60nzN9SA3RkFl1AUacsgU42Z6OALJCOGUU3bXXYYTIRBe2j/RpCiEGV8+1zxRsEK8Up3IwOb4pXcyXGfxosfzdk8iao5VhZfkzJ1pp3l6k8eZyCnXsfJ6Rw3/Z34lw4X3JLnJ8UvuCtc+2gaAZN1VpCqweaj3TsWdMDF6U3f957KxFyDpAKNA9Zwh0cyFqeCO2vleP9JrXv4o/buas0mLDwNzBweNmqxjFg5gin5LKzSQG401F/+YLXryU6owDYFDQFL1yPH0DRTunwldhltAYBI5LRw76OxonAHvmZDxAj/hGkpYiftPsizKTbh009GaRb9hYL7JSSEWacDUTWBb4u/3haUz/eS3wC8HQLt1RCkn4iqY6/NH0NVxvV3DLl8L1m51t9fhgoUNxfq+x22FTZXCh7WtMnXdElSmb0UY/2QkgkxVGnA/BaZGDAv4GPPgh/v1+sbYRIE7U8HaDkXDk8pnFyxlFhRHbk6BJ0D/VgHu711YscUj9Lk/4lj6lwCaeMPTT+dTmq18BVpqQcOhykFQYF3+RbNpo5jDADOPw0e3Ky5EtiH3A82ec86O6Kxtcvcicg0y1AzDyAEdvjd8J69Qf1K/+JqK4gcPH4bA3EPBoQYGvPv7oaqMTW/WisSoSIVPv32KH2pGKaw6BefWzwuuJADmbZL8uIL8/yXgWkUoW7fI09Sec/ws22QGKMjyLtANbR7mDKXf8q+Qx8r4yBwaxnt5ZfYOEtk6Owve/J8xMooK4IpydC4zNNv5s1EesDCUbuxV9RgG5ksHEfPDwKpLSc6qV5LryOLtyBoJ1ghUevAhtCgV27Bom8NW7rmV6+Y+bBcF11FgK3HmXTvQzGSIsV+5cBp0tjS+Pj9I7yYQ9Decf7zfa9gDHAi0dtX2IDAjX61luRlJAT+wlgu0usdfIPl9x+OUSjMkfO1gXo3K/x7KtCcFRzZlXTwHCEkGD5bhZJIfgP2bg8ZKPkALvJgX6flH7ayHGzJ8InUS7P1xUcENlrPbABEV38mAD6SXhXJIX+Z+3zNuCIRL2gpl9/JN1I94yl8PKs69euLgNoqiwHxsb7/lYE8wwFvGT326/zHmrUpyBSNLUsQ6h9AXmwfBjFI0HOnb08WQJaTHpk8nAIDU/Hq1T61tPNzEFE/YfTkg4n5WHBKjLBhAxkqAvlzEXnBuEoGFPvh6tBvkQjdj5TJ3BmOy5E3Xebt2hMxgm2vGsb+U4LYUJBZQeH6l0T6WKnjTC7dFXJxbfF/rKGSYcOaMVeRbnAZseCU8ilchUnAKgebHHCTK53hTORwkBtf1MCRWA9qy2dJEtI996JfPbO5yT/r8CE8zXZKzvWWnS7S9thsfGeN0lNBS4M6Cggky+6wv518d3oPAIc+EQlWGTQZEQCCc3h2UuvHrYWR1TvHDyt2BGNZM7ZPB4jG+EGFLzQ1FHisWo+9CwL7QtlcUqlK0mqu9Ew5b9XbUNseb9DjarQHx7dFVjy2gPFasWF1LD8swLA4VPurCSJatYsIAAud1tn33eRa0nTuV8h4MDvZl35IxFUNpQhEQznlJjuAblWSf+71GFzJYsp56N23wpeLVhvS3YkXcTLgIScNST+0JwQYHH4QWgOO1wKBezxpc3soxBmM2fn2Ne8XG5KR7iK5T/wBE7Js0zc6uH/sNbF2TqWslk0LKqnLEyIv5s2Qm22KATfjYwGkxIlQnC2aYKJc8aeV1B+Q7kMZ6QLzOggn03blGng/Yw4KVD9zsJyvkU8yODGVDttmvYkrPda+iiSNKH40nYMU9oAO1B5jOZo/QEjMVtVblz/YYzBQAlRkwaJV3EHugmPJSw7P0MWViXXHa4AkjO/3iZVO1DucW58yeR8oZgQfSf/xAsNnsxy778GkyoxJQa+eaE1LC3cwHDh/tkMtoQd6HCFFmyav76JFui5QXVOSP9Ao7N0vq6mBtkWrq1kPSj3z/TLi9VLipP+kFtd7OZeWMxrqbxFmcJvvHKQpvGGGqgxPtOqKnTt8ZfedYnAaoVfBY/m2wOV+9FVVABsVhGcbbO1cO1WWVKmnfj9iXJ24y2u68aituSLESCbsOLkSTcnZwn8jlA/7IlrOcE/QpPB0LWlvqpzl+PZRVZAQGcQaaVAdzfzscNk25rJsF9BItKtAXw35Ta9ayUKx2CLcWWj2sjufwpuRPiaJvuPROn3aZsgr6n+zif8YLN72NOXhia8csONO94CnXdwSgmtx1Dc/y8wdUlBifdZ2uQa+XrF19XiY9nJd5PrUBZEB52aLC2xJ60+8U3BP3qCzvvnO23iLSPTnO15AXq8Kx4jleUvkZ6U6KrXZ24AEES3uisX1+fD+0NHiuTuQk3x6gE8J4JP7x8cWd2wpeySXg/A8nqmJJboMpswyH52wdBb9rmyU8mRjSlRR/+ituJ7Ps4nbwncmDLBzHxRjbeaN+IQR9m0rHBB1WaKon6/cWDymMAteysfHnZHCJYHVgUtAZETJ7bvrRf2c2uYM/+XTfTjRP3QRE/AsCUnTVQ9J+qsvWQYPXKvSk2qM9mleMRrUNZ4jveXlzCOSIlhOlWnY7HQzd4DrUpVImZU8AASSUsu5VsuMq6cT9cNAGXXde1Ys01DpAoBflu3FeSDSByOSdkgv2PlS89KR3+CqfyLqtLU09QA5GtDZ6aWsTTQTwar3myO5wzXlpcwS3bCsZXAk0Xwt76DXb5ni2fzSd0HpZOo7/AZYdq+1O4l0l+oQerQ1OED5sY+olNx3YJL/9q1WiDdGaDgPg6s1Ok4aeO6Cey2xHj7L/hazUOeXrhy71P2rvSHt6c/LUVWGnTKd60YlngodCRMMpwmnPSX+XAMui5rFf9nA/A3FWLk7kfFcOxPBYo11mtgIxvDK6xh0zdjlJaQvbRXDkh+OmgMoYfmcYotxrlHNrLCvLijkmK+mmmSdtyR2GT54xBs48zUtE3ADNIPci3loiOuufpJ2Li1pMravlSRGLHnuXeTijTRGM6K5oqRW7UM+58dN4jzzODycU8UkpKMtuSrYBFE8IXfIADT9NHodMUpD8waVgZh1kzHd/3g3PldeeO9GFwa3yO5uDFeXLgLWrvvZQF/RaQ2WKDLH6r2CZI+Bcrv2Ca9xlU+WvXMA2pUQof9iF0tcs/i/R6Hsy1LYB99meDjVPgCo/GcObEtSUTOMhCkPud/vQsF3jt6EvBGFqAHmGoc6LaRvaJ64RZk/rDKvvV6h90w2fmYPXgFOad+R8tEm2C7gSZDpnpl/P4ZStr2Ps7K2JsSbbIrPFJHdzsCVsXErQuV1M54k0gzUFYfweGuGn7cMsKRfoSOgkuuvn8eIyuUFXvHUADKHkrk6kdlrXpOl9kezbjmGPvrKK9jftKgy00IV7kM4KnujV9HPXz2BlgR3BzlNupF/PI6dU6YlMnzAL3J/kmZb71OUrsgmg2QZMTFSN5Zhq51LQqBnSFu9mthbp2AvX1Jr3GUe3NKLrt6AnIs+0v7RJJuWBDEzyjH/5vV/taQndFrAytus7NW071FGrMkDtF2YMuExMfkbs17QiUHg/aEFo35jfpQxAbkRdlubWhhLHVKmmOaHhU2zseGNkMw9pnHNLMJTZihV1KjH0a0Sb89ugv7nHdGDerGrgqptuyY6p5AqUb/h5rPEHUOteTfGgfQhCWB6yy+PvEtXpi3u2TPjEQB8ZOVLr8wHfMhOI3fYDTpGd6kNSLl43IGk5gBDrZ7XqicB0Nc+IzQIrdjZczjvVf6kWYh8x+TewDDa+ttlSFsApWJpHCxrK5ynV6jGwfUfx6QfC2ufv/cYbJSVKxcpH7XevJKj3eJkb"; +// String jsonValue3 = "/OSLeLcPabtaf+Yrift+UWqFeWMC9diSGDwAjhWyyttB9FeFH3cAx8xDr/MneFrUrZ2QHdUIga7zG0mOpr+LmoliJalcNv/G29bc7iivXEsqIJKoXLfQgVcRmmEFAZKloWt7X1ppA4zCsUZY/X6RhMVPXh02v9k6RzJ64XBcMrWyye2kXp8chKgJGRxukSmUatj/Le2tAymVtFN/gCWdDScMvc2FMvHuLby4WOn7ebHlG/GjxoLpScmmv3QWlCgr4lYlK3/RyYDMqXdzzM1RoCVBEUKls236+7qoKzkj+n2/eMJtfvYyy0RT0rvofyXyX/PD5o9YezCKa7i3SmhclqOJlQsRav2x2D+ZBvCfrctNk+43mUQHd2rbOvFD0Q3G4hnw3twO1/imDm0eeUbUTSdTEhw3XDkC6g/CSuEFSa5zb9NuGyFbokZb84AQ6d75mVkTSHmJr3knufhcO5mWexZnBGPfk5rPBhkL1IpIU1JSrXOhiI9Y880jHTBGpBhQzfvxoViARiw9hNyE4275OzUcRYznFkixWqo8kSY/g/6ycq10uGY7PGRhEVaZ9+OOTFm5djplYIBCfAOh7sowXRa62wZR9HD8ICLQlEdEHaSzgzghe2c/tMdYJ45o8ZKtMgQBiX/RaJpZzY7SJM5J35tG4YGWY4QhDai+prRddavSjsUSVHqRLCtUoWYpGUjTdkIJcx7D7/daSQjDu7oKhiYZgyjcVN3deROHACwPv6FrkKXEAydQ+dPY47PSc65Get+MiwI1jEA8+QTwvGeqgbXI6iZqNNEipL3A+6XRcQ5xffPSxOEt4upUSMx8p65fMqhOJeTPSFQ3ajB0AUVze1s4NHWAJAkdZfV2WQA9ZIWPZdApw5LD2ABZyDFc2w66hYJKml/WhVj2H7krO1XvEdnCwby+ZYq4DUxxieC+eCBx5iC3TAveTotjV8CYfGTTvTHbHCKdFn7tco8vV/3sQdlMCRksngTp2miCyhBvxcGXfQI4mFvCoe9IMiP9s2AKoJKBVQSKtmn3Dr4ZljH5RD0qqWzBGbmBWzEdNpcXRpCj+A0ltAvl5IcQQ3zQ3pycB3BCA7bcD8g0SVPV+iaUNUvODlZSMjwt5W+Z+h7G0hldvJGS/3kBVX1y0dPhM0KnbTleyAUlFuD9+sDlRmsOb9/4ldsdph5pToDO1qFxIMgvYprDF8k/pXM6bo4ndPquiAF/BkiVK1Jmc5CebpKRVVqmzo85kHh6/stRNszZNINfpDvpkV8a5cRi+eVrn9RdqkMr9L2K9BpfRhxveFOOsXRTgY1ViBWNAkGcKWHKux+aKgFWXqiQXpCB5nfk1EhOFX/IoqBjS9CmQy+lqOl6jWFPeum7Y4codD+hExq8gFTF31S9SwhGLyFTXg97LNiYWLoAXZVqTNs35z+AUp56wHc2Cqa0lb/g4FMCDgoocwArKQkjFzKzZh0ziR689MxgalwhV7E5iczhN4GFL4Om24geDQzDWwxsZ6qttxQeRlun2S6NqcnqdQX5X0n0KNmvGPUPGuhifapsh9DKVRNMbx2a+6+qLubqnBME5LGOU/q6HCDCXaIWmHu+RbW/VLEruEiXgYQUEBDZs2oJcCKaRJosW30RwSllPO92lzGiGT/ZF22FwAzHkPXCu3O4I7Rx0QnEW2INMtgsnFPUhHVX1gQj6zWvUbARPFeyLajcY9C5BcN3LBwIdLBSRPi8sTt2sd5OAwESnZCYrEHfb6PMjLOmxFFSXSrfHYxHMIf/O0C+jc4JIviTOGNik3mxsadzZC+krTH13a4vC9L4yrbLkv7fT48++T/qrYqxNl+EcYSzUsw2ROPYIQwQl+fvoyh6G5A2hMVeDUwU/przlvDzfmgYwDP+HWfn4g2sQoNQFMc2bJ++dkNOwPX9xahZCBqC8zM6uJWAWK+2EqsuD/tBsSM5lHaazQRGF1mJaMwOblBodBcsVJIjMs8rY8pfHgjdBc0F3spDCCzeaAb1F2+wg84RNQkWTLzMxktpHGkW4GLjGkRD4W14YqAyQ5FEpn1zqXQlFUOXmPEmw18G4EKr60xU9cMNYRAh1rxQ/DEj9p+3kbFE0Oe+Q5Z7b1+eWUUFJAbGIWVqmmo++b2vF8FfbHZUJ69VVBdg47k9FcPey78/fq7SXurSowJ7yjUPkwoBh/ya5r0XDk29sDXobr3s1/ccgz4SeG0OjTqiAw+RA0On0xF4O4lZuk9urQxzH/VduG9wREIEnxFEbUw8Cnce1UmM2R0bsUy3CluE7SdSgQIUgGs4Bif3+T2P59CFim4rmGkCmLEAuhXGdJC4y4QLdqSMEBTpT/DtrPPi1hsNCn1ACpQmH+sPD3c5WN67w3jpReObdVvDkm4rH0O0HIS2il0/KgdsLupbiZSW795N3NHFgbeibQOgOPni8WG19Cj2T+fYuShA7SnuogR1r/BOVnho3qPVUOkAvww4Kixw7enAyEDvCsggH8rTtueA8KJtyUByMGGdJ1e5eoSI0xBIfsN2FdjonbhPA4tMdYopM38GYB2whFFGgD6oG+2+L3Xy5syXs95uKijKyWn89fzFzK4OfKRcgtwlo4w+K+IivPkkEo0L1KSc7leCEZ3YFXkTUPAgnhUjeyCNNgBLp8fCQHhCGVX2ZBvNylhMIzvrlveMWTHycKXFWQ/SaNLeklUcQYwSpaHNS18FexIg8gFNgc70+FmTfiTi6iVyscZZ2awqi0uqH4+4xWeyMsBI4a+0qhqYBp9owid0tjnLxznXh/mvcMHfoU2NtThV78B7fz9sgsxndJvIaGhN7RwWTPrHrwZvz4onuTRKNjYcIxANLJNHf1D9uQxYbyFtq4N+AlEgA9W+9pPpq4fLYWTj/gpqciiE6bO1d6p/6+V6M2YwVrv2parrFpvMZbQ9LleckInMZHormv5GnzYH4N04ReKNkgGMH+fmYVGDXORCTX1XL46iHlv8r8aCQmCaSE8ZxRdcMdpzZZtB/mpAGSqHoUYFnVdVu66tYzczTZwE2wP/O1m6XgqTzlXyKfdpb4buCzbUhE7p5vtLOIOioiVaELRJACpSF1fFWrzLmtA2sCZD4umQYZSECSungs8VBAUXyfIzU2Aohc4jN/uB/VoALb8wsI6PtSU0X6x3E4l5bD11NCGr0DBBRgGOj+lXepBy19UGzC3qZsaQHCbneanyCYG6SfLl9hnaW3e5hSNXjsQEBHXv0T91OT0E1lJad7snw99otJZsDmTJiG/KXVkhBueVgBg5JcW5Fqo1TYXkIzTlTV57bm4Vt7S5in64yreesUACFtm9CU5zst5L/Wp+PQ6uIOAHKNndAPFp5AxX2ykCBi7tYqW8fOHTv4njnLjTMHbd0EAwtK2NufgxUVJwYxWrlMEQlCqwn6Twq52vLSIb13Y7GmIdNQsA5IEMSRa4bEsXIOYywkoPEtaaF/wDgZx1JwD4Nwn9bw3yRrhttVn5HB/AeZ468MunMcK50L592MbGV4zXobU2rzzcpxjwwaOXb/m8SBwlsUGE75Y0QhuWlzX2gwLeSaR9ck1y5QeFaWROwn2w+QF4puw6H6BeX2qQ9QrFCN+exwS0Ukl9priyRWgBAG7fRPHhhzqqXqn9KxRUZ/+ZH/I3XtTjZOeqnY8LKoNkl4h9M1PJZFTftqzLd7z5Ow68VsY3FvfvxFB+ySyC07Z2vPl81oOAzrbFH0x1AjXqxqAcqKU3WYXZQG/FYWhpN9uIUzMQsgxKGEmjxYa5SzkSNj9A7p7IYc9X/Oe1r5QLJ+JqdMlZBb6TT9vAvcsED6+rd5+wB3hPdaVtQmT1Xi0iYSVfiLrB+MkRf/7b9f0jvYakWuh3+K5zTNwFUeegeeS2EKGaVabFk6F+XHA78vjcsbctkV11m8a5S7pBJEOhtriftgLhkY2IPEAFlzXm4bnwoS3CDtb09FsQnHCZlmhDbsGlzshWZABQWciRi0s+XRVYQ60pdh4SdSsAzEpCDBKIJqde4H0vc3q4ebVPoSgO2XfBzRDDqHd5lf6x87w8fUkFEbR2/t/pycu6chr6JGRRii0gOepgWeHX7cobHXYAQgCVTLBEfXH1JQPeXVt1XtaJgC0pzN0ahErkRRC1DMOEsZYe2ihehB0RUWf6pdfpfRzwvt/Hm2k0bmkg5fRPGovVcjE9d725z3hzdWSASPWLlx0CyuOF0a9cvvDqOW/bW6L3fOGpVCApQNzmT4gLBRAUFGnA8mRBbf2lGnT1iEWvOY5+EUM7iOitoGVtFiUHKXONDFqbbfpL8nvOqsEIA/sKAW7mpo0bnClHRbnlpfHfgw300aLIWfBvCLunfvRqojMOG2MnlqyYNQRPlQhqfWgMUTwtMTjl0KOYM1zOV61h+MT9GrOn55HPyjc/EL+75rBi3Zh0FiXLE0Gbx2Ll/SSWczMi5GzOQGTSE9Ua7/8/aartRUrk6+OnPB/rgH/2Xm5IrzheyZCAMEgyuDB1F+pRj9G+gi7Yt+1yzB7ZdLi739eW3g5ldlKiQV079l/gHMterTz4YZCtJHi2k6fRz8t1S2TIY9Ua5ytsyb8nnhaxTnBxkRzop79mNVwbEgIjLZQ9q5IwEL1gbfS0qd6wa5YUppwu7MHAWym0/ObZRJA1/CUV64hbiFWOu8vT5WoGNgj2krDiZX9hnI3h5Z5HfLkPVFPu7Znmr+rd4rKkBYCoDkxZzCNV2Koa2koqlAXp6w5zsB2tmxrYP3T2qakoYQtfmtuPgYmwvielsm/vttkgtCUltZUbdd+CbZqQzv/B/nZGQMgrnTCuD3bsINEWckuWtTzyxocHeYxijFGbBCoMuq7FF3LaQUEkgMrh3kWflW"; +// String jsonValue4 = "z6m0o2WoAVyQm2Fcjn7vOS4AYWMVKtjo2AMyvBj4R4AUtYl2wEx3MRbiHDmlB1x8GMOD0xt1Um5rSP0kA8I+ubwVkp31NcrZT7QL8S4KUSr3SPBUD4y7D2TuS/ef53vNUX3Uie5hVzVLtioGczL7+vIDMW71lVFE9r73dVkTtYfATmDJTe9k6TQ/4bW0dfvQpIBanx0QIFmDvSYk8SWFoaNJYAjNKkb7f3Q3tInF19eBBbGEqGkofUyXkccufW+Tuw+BS0WoQhTIRNg0dHy6ZkZlNb/OhMJjulPkKEUHkJVnaq4yCxb3Nsw9Bz4/myPIE8GspXAzJMwRvqYnU3E+PzX8IFRSPQqoqxo49poFUw35+8IfWIJD2Rn+Pz5PTHce6wYMyB2JArL8o1WxzR4VB2DAa6F3YR5NmSqzwA5jlUhS+Tp665PutjLfi2UYkcpTEfZ/IIsIGGhpwgpz/S02YFD9sXcyIkgwKvOajkfpLdm7rG17H9lbmY06PU5+UAHF1vy8GOafzF4IAtyAa5iux/pLARHLKOS5SK0ZiFhlk7Ah7qnmA4w6XOJmKqeroFJ/RUFBjFSpGUd/CLx2amSb0zzk/NlQ8tL7nCbHB5LqQBZ/S2qWJul5XZ27grpboskz7P628NuSHkhM6kzOnuLuQda2BuMLA/3uotgEcc9aXekH16OkaiCIaK0qWN64A7XA9QewTXpT7Q2RNg+/oU1GChX9JfA1w/RurNo24YtHm4n5TnA6tx/1P8hUXeabLfz6BHuhpXS6sDJUlgi1pvVwUY+Vz0IchWDFB7I7ASqnTSJYs/0DVde8szigxJibmSkeS3BAA+bC0012JY6fwOhBMMHOAOEwXrBetjSmMNzQPfd3o8yjyvo3I/OUhYTRwPdxSer35tyZCB9uhMgQBlhYSLjhnrUAaY1kF++KoL/I213xIgeo3zau3prEi2oJEOo8FzKKPBm/20WlHElwlE1JQYAt6bPXWQJHUsN3RZgDoRIFuzwDoSZ5WFGE0LjQbkCiNKDGXA7AGfuNi7WpfhMYJ96kDSr2/sye/MgopespCcsNccYR0MMAcy7Z/D1Mzd8WTPy/co0kJbk4JXECVqGMRj6xL2Xs4k2grqtjS/2BHff6K7D/ZD1oilUSE5FQcm477+dUn4/n6M71KTXV/4YF9RhklO+NiGwY6UqgF840W/FtLSAeQl4AgFwQoRXEujuAaBPD+2Uno2n4WaWwFMGlZ1OeBy1cOitNqgNl4h4wbEtrULEvHpsQZPbcBPIgzH5S0njj6pIkvsBhvI6qxLva1v1yG8mlApsxA1OGdNkOvkBYHlKv2Pm/7B87TEfn1X+gS7o6mAudGdMVSpg3QELg0Gb1J/fDZ6YLwO+8heIMqHcVM64/Xnzlu2vTVTotK6tkuvHefPvXGTGPlyw0jABe0kWX0aterSBBGrM8FzeSChvXc3wuuNcLnesbTTx0TkWlnHx/GhOXCPkFlhojjgL17OlY3XkRKlUVjPb97uXHfzwlY0sGb+2EqT+ZJcx8sb7uPr7hcckMxP83Lp7F8uzwqE0M+f+DZCPctZ1256K38EKjhjd18T3GAvUKGVRNKG0846j0XqO919eqLpRvcSbxPSTRA2ITDBvA3YQ+1H57g0SWT6yLtDZtmOiLr988N9iqRQ0ibnbZaOzC9TwuDVJ/DTCf548Bbc3KCWDHULxsKqFoAX5mMTiyl2UpZBi+vmXEwERIlIu6OuiISQa0a1KqJ+o0ecWPBUQ6zbJ+PDo5JBTl5clHafsIVR/VtYV3yMfmxdYIrvUiKNqpp351ZVJnOKA7B35qoSR9QPquNiZpIjhwYwjOe91Gw/3Gc9cJ9YT7/D01zDvWal71H0+ygl0mbI4MWTUGSHeGxEzMOLqIorhr8IGBnT7Xb0U8oKiInCL4szizIFpALNEZDjJxoDRE7dV88heDQ2YwgjMshZwOKt7HqKjT8GoxJs86cdlvIz2FmyNLxLZ7o/Op0Y4mZzo/OcW0rFQDiLaKVR8Tg3hQFVrlzappBGl784MuTCeAYQeobzvozpkQbPguKsEljOXJNgawMalUTzGhftAglQ2/DSzM6XzwuMPasCDdbKMyFhFOFzi03eIedPl9e49Ez2dVk1fwYTfVYpxn8I75xzLVzt/8xnFLPwgWtL4bjsjf6hiw8mb5sWhVPQjCzRR5mGI4IWcuISy+D+AFEqE2rlBN0kyxc/KTudZOUZjKUtg5OCKSnL6TJGNhUNJWYWjMaTvhEDyT/ctQbp3dJiia05QGS5V4fyBwQFdSOZrZTTAx8C1HZCMRh1E1pZL0P6rH7S4r97oqLZk9RO2vyIIsv2OnCz6/9Dr5iQXJd/dQxVvyoIWv8TL45SAtmaSH2cWdaqbKujjVP06nCMzUxWm5oHiky9LbxGVI7uxQBHhUHmG8Zv/wKXAR5LToEQL2draSGUcqkq6FlHyYvaApwoffwjBzSrImw6ZEWoxgvK44/X6n1HiHXvyLIAp9WmqCMoyz1N5RXms7lTszg82T0NQngejGctKd2Kh7WM0um0BC/QQGKZ+8GonB0pXBVphx9W5sBnGsdys0ZPT6PRHV2ij8g9yBxK+h3os2qWX5a+93eBOn8eKhNiXqsBIPdIjg1PKNx0nrRbc+o6Ci2DminmoG/GfAdmot4XfQEVr8UUsuq5RdM3Q3zeQLSodba5cTeIE4DYVUSuR25MwDc0zBnRTS+fThH6oBN687R4i9H5mfutmqLGTHY1u5pTjqgPZqkScrPwO1xj+YZSsQZiE+B3EXHsf/ucmXD7ZCOlOs/V93skSpZDtNPVZBPq1o5CQ2gNx2d5NZMYt/zuKbc2xch04DFqft1QNYBCyb9sqO7MEP5dSUFxlmLJBy8lxsUeDOoczcnko6kuu4ur1GYCKiSr/QVBUWbjfulc06FTemEYl6fllms8ZUoGCuqIX9ehUkiwYgZ8tEljWB62YyhnkjsXk1dm8pPvtD+6YJKvsCTbs7Dw5z0KERz4tIBhxsZ50nFitcINUJEe/IAMq9yI8296KZ1djfDUvqjRq/hA7OclUYG3spbMsd8nsmp8ujvtm3Kk2Q5kkEkQQ9oxWK11KtaidpPuh4LxHqcPYspW8fSigmQsMRUZ++XYyqDbaBf4+0Ki71TAkkwam75cZR4LEwNHY/J3Mrh/19jR1Xz/wtCt4WiKslLpPTPsBDMLztioASFPkHRekpDaCHviUx+QLndIWpjgdIVIyCv7oNhv8YgnnVrUxRQLBq4by1ND5cMlhASxSZRivp+uUXaiHu2CqfSOVCsNDsf0ET9FEloL+fNSRlxxLKPc9PxuB9tYfkLJz+fbJFWhqPd2UJaoV/2ntbhFRtwCfzgFA99E5bKJIsduGdxuYCYMntfpmS6Ledy9CECMrdPAPMNXp+FlwATjC/UyCLBmrAml/Kji44cgnLMnYSQXRsD1HrFJ20tBc+lHXg04/6gIusV3Scr2qMZ2/Qb907gKKuxyLg5ZtwaKzk09O4sYUXg6Ji4EPn8G1lZ/j81vpi4AWcp3wIPMVGtPEmA9TN9NWgTTswnS7N75EoUBqtvlAml3fwkQ/6H4ME4gKllDFQL7pd8Fbh8aKNWMPiYkI4XPZZ6vRLBaQDiVOTjObs0VjJME3Z1v17Fq7yAgrrOWFSOvzp2JWxba0R/y1bxBPq/UzwoXKLG9772Ukuykn+Pr4WPKpLdJRRJGzGqbHUEbn/fz6E5vUa0MsCqoZqlq8wIefiqt1bmOMXP/jPJ5TvBbORNPmgjm/YkKBQ2ZnwnWC5ZUiUBtweRn4da9Z1zL80mGKlHpkpy/9zQYZMrit4K7GUp6CYFTGfb02RXppDBR2N7feUnMgcTEBMqXj8KGHddpqTR5yjCixTfiUdkeYzpK2nmse4g/Wufkfih9vCORrHyVI0sbBZiialvFX0w4ty3dqcv0Xd5s7gU6N/0PSmXFKAn6A//LQFbpo4ZT46/An1pmPjzbycZF9G6EvN4yw6IUgtxoYdQNObZbepSGQV0YTn3+LsDzsjBtpq7AnaVCPSc3AQ6FsxT5sEqTQR4YoslbVc2E57aXObcsbj80hJTnZmKl9vJO2ozi2xDeCHVn0o5v9frWJdkjNAHcoppPj/H0LZ6qEHVlUM1AUF2oAksSs8UE058wkvnz9msfoM6UmhGL25C9UAPGjzUIwrZR6aRlDhvy7OKuP3DmiWzQX+narlvyU5GkbIN6JhrDXWQw3ZFJRiq6WeuT281Uqw7whKsW9ajVk4665txT6h+k6ZvzecF3F8Z9/towtrUNgCn1QZgQcl01V/+QoOLzAxgH8X+b55Na5T441kAtR4aXrrmHH8zmZopiYwvms1Gj8CVvgzeIn6F007YRS8Kk07l19tslz6GzEOxNp50caPxEgS90SeFLqSWcj7Xz4XUFsXOY1vdjuL+hJ2PgHIbtC4pAdFDEE6krbwOGwc8/5xkOLQw5NMtPdX9Pr8fTQUV+VJyaOhrJ4dnshs6NVLnTSgjdQmHvQJR6erpTNpLDw/UT7li46Dof963yWYPoIBoH4lCm5ydUHnodU/L5gZbFO/7I5EEJEIzv9ktnu19LKCP6c6C5OfnTNFiNuKWzW/rR+DbjBPnQ1ffoYWHONSfivKI/XBtvOXbqGJb0G8l0r7DjqcvEHehL29A3huDiNQ6jsjF8CeIZ5QMoIVKCL2+Qf5M9R1dUEpIJAL7QVKfbJiFDBH1Fx3xM4BTw88ST8+VWG+wjZVNCWVmQ2oi03XcK/y8SW0lgQsKA1ikMG9A/x3PR70obVHcQn8z6r2AbeMt/P2Sj+aAzBOcDunOhdQBOKpkfA28IQkUTqZ/WRzhEwTh"; +// String jsonValue5 = "72x5VOUSpqJAOpNpDWzo98mBcjeQ6wy+0JY+nVuPDRAtrKtH1e9YTc7YA2Wk5ufKJIPVRWYiJSsigNYE0vVt2yCECzddRp43Sva/e6WDys6D5FDzt2iLr7jERR2DNz1ESdNL6/hDR4ovtjl+M01j0l3WxCHuMrHk7PABP9Syysnblf9ecIG+/7r3nqHjPvo7DICS1VBvhiozWGOSvsFuurW2guJIRgM7oCcVBbR6ZbJOlGwz+cWS0hmIEMGBJ+EBIxJOUqn4zaOI0xZyNIxntnLSwj6awryHXX469CbLN+ietHXe/oN6T2/VwGTdJa9628/aqGD43DMT5LkWVuNvwv/Lnww3WM7WWojacqJ6Bp+HOoVRjNJzXTKgk2gT+P19cXEtzc+ckFrusV1O4m7bUBP9AQ2ThS4SIL9I8Iu7D0ldUOl63sbNzcDKTuxzNCpaeeauRRVewuA3soZ20/0+g/Mz/LjrFgnz4RtSsrSjGIds5wuK2/ostQEiTQwBjZQKhrHkhQ5f7l8jXv6IjaggP+BptLJ+Ie777GobSs+iAth+E0to2XCbaH/qtSteHIdfuJ+RvaZMWip1Sw3IhUEQ5Tfjah034ovU4DUtYk7lyIP75QrjJr41DqJtXJQqGvuht1R66Ju8qj/c0Vat47v9m0Duw1tef4ozTNl22JkRyb3xDteDEegnUaSqlT3ppO8t66dNvCqqWVAy/kgkYKfTECHOlPguf+TPu3efPuweGZ8EhnzEDeNkV0yNqteLrvGX/sbMCevHtXSpkTPxkhRWuMTtKKhMT3dMaebMdgbEf1PJ3fDzObtELhquUSUeH5/IrdvTm+wGTPlbvQIzQsQSTEqd0oTo6rwhg7sV8utIEsMtA5s/4fBEMgVlJjOdErQPTDwA6900oqztOnwPuBv0NuSduJxX/CW8kPD6vLjhlZbDygxAIiGnejCvUKLCe/VwlvuGTgEvKJCoSJ3vIkUVmGHfkAwDN0IlRMWjLtEKPafO6MbGkRUQuhM+bzOKjONbI0XhgiinfqWPm53rqnsJ1s0H4STd98Gg9w77XgfxE6bVi9GbBuhhOgki6LPED3N5cJjrgct/oZlkzNha/s4Dy4QX53Xud58p5pdeWopSHxZ1ehA898tjK+HP9evg0jV/wGmUvFHVEU2B1ueFTU8CIbEYHOY232J2uSBggrnToLHd+qYxRcsA0IMhYd2CZnHSO230pK00MIlnC0+Y90qLpAL801/kxbkXLaXM4zG2JQ//FXyzwoVXqzjw3Bsw6etrII4/R7Ula2w9Y94BD4wdQ2eLQF6bkDLvkdCoJmUBAxQweqnUNBPRD/QLwzPSJM+3VYdpQrGcso6J+hXdvSDZAjeybe4xc3qD3KUcWSsXNYqfIE8FEGyzXpW27iYR7PJMW1NVCtsL4C69LgJYko5LnPvDfTAn8UqMpfMcZrBYRo1LdM92RIkfvVFPrCj5YCfxKdM7leQ9+cfgRjs1iXamcAV7L9GGFgJfXYK8Ea8fARhgEz7vhSiLyhSIJxv3VsmNNWxJwQuw40d+1Klr9x2nxKjMZAmwLpMdyHppDegxH3DkZqHOiiYhPQCqGD6APc93738NncNKiWt/O0yBhIToY6zt/3898Dfexm7AkkzFgPgcWW8VUiJaajxz4XwmD3BLmxzuqrqGXqNHnYquMzUjkeCAflXkZmVANKgHRVx6mdjHXU9Sc+1AIyl4ov92tVEyuAyLfmmVrYQbmfDO7YcEJ2ED2Zyi3/hjawIhAZJZ2O255wcOirfgOzyI0TXzOyU56i5zjo8YpmSOrJX9k66tc9VNhFWT4uyZuf595B1w/444LZX2sqRBnTT1avO+0iUl6ScZ+TEJNjPSbRVnefsFb3EVoirCHRvPtRnz8KbHRl93plvsg4BPOGQPNYWS8rMegi9YY5p/I66wm31g2DrmVb+g6tfyle/2H0GCZ3QlgYPAc8WwisUDKA6Fpuen4LucNyyT0UySWI+eVh4EqzcgEeCzbN1V/O69eKucH3snZVBpGxrjQu93nm8T3ZQWiG2rAZ0xkz1yY42vo4YktUJ86622kfK+yUcO5KBnIs1jnZriZbChgQaQPP8F8Q5PWYZWED1/LdnLhLqn6tgGsQKGKvdj+EfDpberoK/zAU1KEuwIzgXhWOaJsLJ8nOowzvyeWbe3OFOZq6bediaS73xnuniqpP7QwXUvQR/5jYdJlowDX28MVhK5a5zzSW/pYHGsIJv2H04fPOkvyr24e+33iRuT/iK1CTlDXQ1fRFIPrCgIn3fvj7qR26Gt/VMV5TELGglFfWky49pCF9U14W4rXCmdAnnCkZkw502ku9Rgy71mYNIBJjor4I6t9u6avVLZnZ+XqbgL/sf9L85vVVy8iCJp0cLSP4x4fC1bKlMj0yGTU5wBIlRzfQkADGifTA90H6XPd0JiBCbqAhFAn0iRY881Zmbo9SZepDaFz7/53cN4067qjrornCvJ1Yo5edufTSF0Qj9GFqmxY7jLT0+6Xwbt9Qgw3tfRyxaes89quJsYgoDET6aOcr0CbA9ilMPJq1HvmsGE12XwOble/3tdpDQ0w4sgYYPQqQ5flBpvqIc0m5JkEcMj3n7VGP1csqXfDE5HZV0ptTz/ORCCJbczjNturRL2qOI2Qq5jcBix2o040UGlgqtQZ+ILLaQpsMuzYb8OJiRa5siFsUp9HVLq41NcrC+1Of4WeUCSyODhzh8+q9GGn1zEm6d9y6uHqwCq4fTvcDTrF/VcHVyk7+JQDw22FzlfPg/Tsa5i4ifFX5FMv8M59p3HvMqUcyEJwJI1l0LkQjqvYSQ8l+mEaT+e44WnWdRcamADqiFoXIiQ71Xh1J4jb5Qmw0PzxStasl/vruOF20j9rTvYCog2N5H+CxeAkZurXn6Z8Rw/wqm7PQSy7fPKJVIUa7rKZFksYGONTk8muNdV2FImFOkH6yjkPgTQ6PeMWyieMohPoytQOs9Wm67wZ2dxt9T0WWUToqbQu/YBYa4Q0KW7kUNtxIVtLcSzGh3Z0lxWPPym3Ewqo1YQt4A+1JkGyXgNQKVRTz6dOOHj1VF7DHdhYX0f5UZqLD1nQ+64/1W764spqQPEF2pGRcF6CAf1f/rof2EdICmiy9fCL971dBbJ79GFkxO9bZm1bJLLrOw55dbooFWSaLsiB04xkO1zHM33bxCruABUczm7gEeoqg9/kbSoaWbnoaKnhDGtV7D14c92kGe+n/lMjfD0hGAxY0/J6YY+xB5JtP4MjI3Al++rZdSGpjdnRf5c2GFvqaz7427dMQKYndRG/V4VkERVypwXMRxmEEunPsBX0Rv+SWKUc458hpAYPFGX5w5hhz4K/klF+ELdoN0NPIc491Uzt2KDxlPWuY0en+FwKpdFrPpMYIuhz7AFTdYD1e8h3UEMlyEts8pYinJy8gdWLQKN0u6j0WJbrBJDf7j+7h2QCCjVzH1EZsSxk3kXosQA7R3oDRvDqpk8F0JzHDXFNr7dy1jvul5d4k580+LQiYfqfxT4MR/qfw/LED5eL9MgIfOtA5FV6dzaeS73xJ8+UewNR5A1qf59pnasUXCWHT86R3d3+tKMf4V7MeW3bCwgqO0pvR0EcgGASGfpKLwpzY2kv8Dl3AUoKm9bJUkB0fBDGPRR26wET0Vqy00aImp2fwnDz9XzUDG6goZg/f/QXcJy5aLniwv0zD4BtOcqJtzVyQFHRLf2PWgFHsgI3sPOdDEZFPfzIuLZ8tDXk2kYbJvs1e+xVKnzBBdPX+N0hh61+K+8T448T8toYwv4stV5gZcJ8L7Zy5Jaic1OMzSlUPMIsYGPithrMsldko4c2fbfaO/wAFx2CJShH0C2K/QCFLagNvUX4zeDSCScWV9CyXSBFPWrPk/dB8bp5viTFl6SL1+vyo14+84UsH3AJkKhkWP8MLYTMUs6/fo64zhItZq8INaROaNdmL6+AMs0fIdeMA3hf5OL0c1hNaCiMyS+kDsIJA4Tdw1G9cHIoBJztmLEHux4Cpfv9GQZKtvmfu8Z2TtjDIDh5FsYeJbOe8Z0fyj/yAYh8EiNouqI/RdB8W1IpuuUBNQh0so3SvQeYcN48AdAaWqC3Rk6T59Kqsj0CPeUUF8y49bNyvpbTj+pgs1ABkraSbu/a4/4mxtwpH6pIZnoMJBqJLLPb2e2fA7x8/n+DU0sSqmsKgAD2ExIo0ye+gh9+IEbBAQFEbwprKfPoI8cCgXrmXZHKvczcLsz2PF+UjF2B9eD97rX0ol5XFDtTJxjPYpbSMOd/LkuSM7pm/6/WWKVvUDmnfZRrcrIyvV9InanJFt6nLweoTu/AIQE0lAqVMTY2a8Q1fkU1vR1EyJH4DQQ2EU5iK0F7emCSu/a79xhDe326FdyaBcJoDfSL4Kusz/9w6WVAxJupQ9D4R+wfAKsF9LeFqJ0PF6rLiVxg/ozJTjl2C9xqxkrA/SB3wwQ9G87HmLkH/WRGbCnlUrdtar7G4FToqiZfyIJY914YOoSTETL8mvqQLg9Agw3H4FJNf/Jj7HpM/YUyPzml6AZuP03hG5tRmV2kNNiWD2Y9WO9QeaNzZcXy7G5DMt6UWM/x/rnx/ybw6ICY100djXL9eKrue5al5iG5BcpsoOXtshXuDSBX8BirF2O0B67gHyK9U0sOXRZr5LlsyE4pyFu00iLesPSqNZV44zRik8IDROnX5KAcoOOOorD0rbMEiOhNHvBLvXB3S8XMyzybxrtBP8Vlv3Hdv7cLDViS/Tt1kU/5WlXE/QcYV0RkDQtH3n1S2xNvA7QtfuLSTH8oWXfUzaKTi3HK/t210w9RQxrpt6dR7gHmrOBaEJM5RTJW4dvn/U4yOqXCzkgmgNCiSe2JlCr7fVVkk/76AH+LlnjVMXDcFXmDcgXP6ZUORYNaWcfHnrC0HUxMavbPFriVqx4ZyH1hm6YQzRcuhd7sR3js6ZUvAfTmoqFUTd5TMWBywjqUj0NY5wp1hDyRRja853LXI62sXAAtWfvEkQWtQD06gZuPV3S/0jZNVO4RUlz7cUMUXcQHlo7b67RSumQilYAYR4FEHRKeu527BG6c10t6g7POgI3ZeiuSCvmR/rxqv/x8/HnIZtMvrwExHQ/U9vnpHlpEXUf2ceoo+5P09ps15g/h6E6EpYTAG/UsdOR82oxsxBVqDhIvsUerjA6INuExdNHH4wqWXtymifHEgtkBWikGKU4Q413gyQVWLW3fM5MxhGPRyc9MUTx4ucfXnq8gB31s2ze8hc9pFaDeExaTqc1qU8zdlpDGZVqIzJa0aUMd7wu8UHFcbZJTCZ88CIbBHvCGsQjqOX04+pSvSFw40SuQx9ns8ki4BPGE2SuDA3ErMt0G1qREKRVj1SzvP1Zd1gWPYRvi1QWRZuTSg62qXIjp7cKO+fNp44WlsdBelydKXiGjFS7P1rP0OEE/2PEc0vbiQ6ridqTeVtUECz8Tjt/NDumHPfJZ3p4NieWuQOEUOiquSaVfLjnYFhCBqDVC5h6ZrdpJHCQC6fzInenNnn95DJqvZcDlGzigXgQtVkinYcDNoMjnUd9xCojhAZ3JDNuB3Wfe2N5SvF9hwa7bRr8xznowTvTudjfrqocpgiVqeQTFin3ZiDy/n2kETqZ58XZ9sUqez/JY8s128oJDp6diEMshEei9bMZrutW/SwRRmOxhfkAspy4BlGP/Iqg6ckK79Ize0OH+BB1kyn9QWJKWu0w0ckauVWDNQyN0gEJPMTLpBahv9wHGOu0Bphhq2eyl6OyUYdj9kq1DzF+MRBho0cLxSKPknzGpia80twRgNexddzoX8g5m64uboYKljayjAyrSkP0KBHZdMOTErZNwh0SjbwNqvP4K5DbxNQOtNahxS2OWQq2OF5F0r9sH4npeP8jgt50X+n1mhZKlVAm/UzC1/OhpdAmUxPdGN2UF7nx0qnfdc95jqkk04rgcxq8DK30i2lK1werwKOGmSRRJtF5BbQ+rnjeOHJjdlCQLSZI+4TdwaDEjXQJyJkBdSAO3PwlHbmZfxM9bQOCsKmxJbkJkDaGlOa8Ac07+xS+1/PCZe7JeahaG2jKQBMgAnt18VL4Al2e51YHnbep88ZdFI+VhJFvwXSiBx1soAGTAz7Z0xyaouPDlUbyCci3wkUC22raJ+6PSEWsbH6XTo8zI1EA2XPpXgUgO6bRrzY6ZxgMZk61hq3OfGkUUWYrHls2HQ4U+WgSv/kvq2pgp+ht6M7vj00F1kXM0PNBuqDd7043js5CPEBZTAUZ+FceMAPVWGPQbiaJbjE0+7blW9GvfO1HySZYim5sSfmm40CydXCNSo56aP51BhEPUvIMwcl+hQcGurhZcPo1B1nq+Wbhfo0wK6+AHEaAtXd5zY2rDv9d7bnV8XQCCfVUjNOV2a1kdSYCGWoXRPlG6t5GA61v5/TO/LFVhSAz7dfZBBmLnWZyOtqEjzqavSgtlZ46czj6NWpOfJ00JXyO9aA2JIv2N2qohcS3Itc9gpbl0Y3MRRauJ0Dks8Pz6Xnub4AZodAXG9/Xu1wrjG9A5tB8NUts0pRNtBu8QHQA29jBjfIZK6mlcao09nD4lwhvkXvX3RO5E6hn98Nq062sgCUL/pK6qr2manLpHZiWmxExG54qJre+iSNkvFg9iaihlrHXc8R/ygZlyt263kPG8fwPJTE8BVwZlVMSIe7xUYpHzyR7j3j2fknXCkPA+rLghQSidJSHWdJHfgGQ+1dgo7PvWU7spGRikPVVKIvxGyfKZiQscYOPd4otIpD0ia5/CGriwimWdPIYJTMx0aL5cdZhZytL16WibT0i6Pj4aIW7YzDN3ylpIEa0qsapuu2CFJ4BptmCo52zHeiEtCvlvA/yIiupWG7VC/FvEmTz3ZDV7atjVGZ+NlyHRy+IS0WJGGH3jt45Fz3ssdBQjpCp59h76kKVqfzGB278OQvliKvOgyZt6K1mewwkl+IMhh2D1Uf/vk9g4WjYMoW5xwKOqjYWDTZ8bTGKrDVZS4aB45IaRuU2kNnamO8SSAqiYSwFElIRbTLOKytld7Ie3G9pUns0/jxSVC9p1Wy9OFA3qigF6Fis1gWR7OPbvbYUgBsTexjBh+DF9FDqp2/pLaXh6CuGnNoK0m3CW3lhHD7F4FONh2mA+4fa1ix6JJVFLTZ8l9EVRjH2YeVWVhLvCj4zq8d2etSZ2petPXnHDu9TeDCWTZhgky/eQqKd0KG1GO9evdVgNchESA/wMe0+de7poDoziOCPqPw5RqPx5zLwitC4ToeAvpvJvMUSwLmhKTnMuEUmW2oznXmU0pR7j5x/EomCLb6QaaKZdUkg95dqmhJb5uC7BHjpT7P3gHf3qnw3EwGUkFV4rxX5M0GkeSqqjFqdAUv0IWFoTdqBSSpL3M7Q46FthIS61KB7KsFg7n+zF9oGYGCdmR+gAdjKqAba40z8+CbPT2yGBw7T7gGjpRxXUXFVyC2SI3/+R7XKyF3XM6Gui9NeZC5qjEkAuIyUHn/mxvdTnXW2AFPLn+BgFpjT3xuzoBJbtCGaNakEqvp0FFfdgf34nE1R0x0sQAsE5+Ot7wzxSDpoPh/BMtCwP09CF/VFeh/PTrGxndGriB3I9ax6QYc8MVzZ/DOJ2ZyuqGfMtmFJ5/G0wsEK1TtFboKNdAigKvllnnym2ZTHy19lHbTNcO/mK9GD5Z03VQIXVPSd/M7v94znTW0YIGvICTaTgzT/R8SXVuJlJwGHTOvoFwsBaw2AB5Bc0h3hzDoMwcPX9TAYhzm60YAstU1GoQESRhOWw0lrX6ak7GVFpVP+RD/xEA42e7EFPsX5bQ5np0L0tzUQxU+Klq0cD4W8g/ZYwOiQdINnzV2Qt316rxwzzSsifIvRm33meuMbP6MNTI8vzbPT6mj1x72Jsamq3k530Xi4jXyg6MnM98pE4ZsR/5v+KvPuTW1MTaHkOyKRg0e3oEw67zgjhjRlQWn5zq2vRVeiy0jO5wq83Feft79sBfO39oKprqISpO4qf7JiS0aRqxi15NMKlMwlIvc8QuvC2KYmMMO5SEOJNz6ccg6x3/R/lkfapodqxwT5Lwkwl8f2cdNxfzoxupl86sUbCD86T+vjHpNLzBQzKyYFYoRcmzEM9xyLwnJVvtmiy8CM1bV6/Qt3UTzn7BDCbEdQcYoha+5lQ0Twjj1uejqFacvwzodTBec1EtLzwfx3wrDd8d/JLRKI0LXNEsViwx26370mT/SeBHKvi1ExbZ3AdV4+E8Wb2C2OLbfplQbTyvxyZhq6MBnFVz2Md82xnJdenSCtJh9PMqxLmsMcYrVX8VIIFAWisl14sqJbIqKo8n88s16dy6ypyfBzTOJshCLxlMljLM/XgHXLDJxQCop2BD1GZmMBwqCT3ZANqIUXQ1gbzmZJ6m54yk4kt0NiXzLuzxduYJGr+ECdZpFi5DNIliw+ny9ZWf6WLay8vdQxpsgofPHOzHT9B8W7vJ1Vu2MigcntycnAjOMorqAC/IzzKeKIZ0C4fW/Ys0orDSm2aoJ6Bp9IuptA9hB5mtVRkkDfkt3W9oE/Kfk3NYWo1LpJP5km0qo5LvvxIThBQtP2rVsKySV/WjZG+OuAyJuBwrdHo5mTg52liRUnT6ayHQlIBtQ1X4QzSgsGmszdtwGvpPh4SZIWWeX55OoN7aupLswiq8kogJkWPNoSrmVJELPdJEWkC3ocgerWNWXr/VlUTgb8N2WqqWbmw+spUmaauqKvK0ixq5jyf+tGwo5Ziam/PqzXW8RKi/xsmiPaAAsAaCxZJR++MrHr0K79PQrE02Mj/x2KZWVKvKrISROGG/YXXnoTmuX/ShEbhRu2xxvg/mgYjTFlRnJo2DafA11azeJhA/nggW8XDID5ASTKA8RmGYjzfgedKXMV2E+Uw238TFGGz7puxQY59ehOYfU4n8WHnjB81xR3GfjQXeWz7PCJwT1x2rbXGpB3XpK4vvb88y58XpeTCitUa8BZLqjAnEFIRv+ggzvH1nM6RLZSamlC5aH/JUHAZVYHPjAnqAKBpzJHEeSSb4l/LZ/78AtZfZ25FRjqy+Turcw9IM3tGKGEMM/XCzDVZqB6oWfpJuJUubehj+JChlkkt0eK/oNNhchLg2HTrAzwlPnWM0ZtZiGAbygRsRcOfcv6q0oRGpPkSLGkdL86ci1UHZsyvJhgFeAwZ088fu8FFii2CYzx+/eFQwAW56r/Zul3x3Q9lnqnkPEMcW8KK4bEcrWPAZDYlJd5aoehWyzdOg4aO53MVkDu+NhozP8fbibLqWOeo7TBwA9jf7+nIzpATm5xx1dvyNPrl/dbPxMj97GMyGGcfplayGkGStmoIWQNiBp3W7X3BbWdCmvR8vIf08wEhvva3xfV3p/oAoYx2+2WonI5E5j3DEOvTRj6e7A09i7jNSnrKt0BGrzZ1ZWdyCL/9qmcyGVegQZU5XQOHe+gatoeGDyEZBDOGJ9QwRXxGP92i2+XS21e1D2TQDlXHGNGKBiJU+W8sudgWaTD29EOfVHr3F3VX/2/2HZWkU60yy5U72Fo/vux+blWqo3+sdDh2Cl5EfZfxvCkmVo3SqfKYnvPpYCMHFjbCdlpwB1JRwc+ciSRLMNVa5tnpoNDYtZ598muOtbm35mP0Smme3g6Dc1PG9dF5Cvoi+yzVPZHTM2XcAmWVWL3xg3xDLudouMWSnx2iOzwC1LAHzTTzFc1TrUkJ1kvZcchshzUwEed634NNBCgH/N3r1Dh9Mg2DW/ifpq0xvNq9ZnjLxgJIJGJITrbn+O06RadgtUqMRW4JHqe3UcRLo2loe937PRFufirDIyIB/h1sf8aTFlcPRukqq7ydv+Wy2UPAw31Pq3aAFtlnJZqIsianyNiLlvtTNxjVckxSa2UKzGpALDmEQRjoQcBjyBRQt5Afyg+YmndibIfNtkIIQj7oqYwfkAhgaA5BzNj95aGnsTEbfHv3kHCFKYQgqxSDLQ6zQDgsbj/FeNuBMjh8m9BZcilNlUlcRshKv8fUfvk0wFdV9tKYmIeISTzrFuHLX8o5gmIn9CyB0kwaPaXP9nQB+M/psO86HWlRPCh/ljRsz1oDsDNdV4LypKxkZ0cO91+beQv0PWmdTu64oOzqkJT80777EWYLABwzkR4FNjRfS0qflm8E9h/ELy9ZdiGHYcC03jT0t0oV6RHT6Sm1nyUoLo9icIH00S30l5GxMF4YgXBeqXAQG6QkwxJXEA4NPsoGNRI9WwDPGR5QJIYF+KM3+7rTn85IZQVvTjQ+sqppCMSt1WCpnX5SDEs1/OzEVVVmw/r26/YxgNE9WFUoSX+5OtQv+BZKtIGRu2VCqsQk8qmfZjPqmk11883vRshl7Yih4m7Uzsy5YIm9qUbjyI7tB+HGTvtWY+Y/wlftlvwJ5U1vt9g5MUmUFr18rWvuD0nXmkMSYcZaUy2OmogT7kn6iFJN+LnMho5fXiwDw6F2u+WuNi/ImxNRIzoL1OjuSvYYqiGHnydb+/FyIhHzo2a9Arzql7pGHpUwrqiAoRXQ4iNESrS5gMoX6ZjUlLCx+p+gwig49pCvPix7rDiLUqq2q9SFkojFExDMKuDGdBXnSGbRosWFXFGJ8yA4JB6d2R/Z90m8wuVSuD6BwvH5t3PGHrcSS5CSObSMtN+zo/s5eGHibMd7KfPpF+HWkphBYczc3mmvW1HzdKJOJFXg2MVLcO/jHbt4LOxBKletC47eH0iBaW3vborsY+reo/GGUCxwL8NZloeFheDFlThU+Y/k514S52zAPXKTZf04EDbPHfh0fICPMAOLnQ2XAhB2FzJDP3Na/QnkoYumv1Cya5G+Upi1RQ0OYMbEPc01TxS1P9o014W7B2pPqv0RWvrqOkT1s/eQ/drk00TR2rC+M4S8kLbeS4PX3HX1RVeIwnIzOdVxdNMgTGfG2IXPWm8J6SST/13aWK3qUzKdpDwPVuFBgmfE4lJS0efH1GvZbxxgk2Cvez9X1p3GTpi92hnq6lMPGGMfzEwYOQJkA7u6hIEk3Q+G83AsNtiyX4tCx0iM1e8yio6I0n5cE9YsBmAHN8HnU74G0k9FSjf/3rXXG6VlaXAJva4usdt8dG03qSTy/OlUCnAuOPWkf5ukdQ5hbfFHrrfObhUdYkM+ghZkUoqEfh5gghJZZmQ5kFKRPGN/4SH7aIvXjGKueI1HD5g9JRch/Gst+WTcPTD4Ej9gJ8M3bF07FETMLsIXGKz2/ht843C8yVc/7zTbS3PZsbqcWRdKfZIzJTrUZJXF2FVCHLaZv4qTExJ1qVCMDEeK+ipUX2R14pGiy77LhyaMgd3sKccL/kqOxCwx5cYf8Uy5O7hS6ZOENXw0bNs9XrtzKMjy++KijAWfO9eesCrtXvBfxgRrIMwMH5QyB/+w5QnKxTnwx0eZMkvY6W0nhgpxHex7nS8eAgZgoi+lPu7NLt9zexOe3GmPerHcvDpsvIrnzl93EvEFP2gSaq0waB/DqEhXk++ejpJBHv3n8QBCKOhcrjXFzyWfWNzJnCkQUvt/LJoVL+E0nrNIbeSfzLW6yBpLRlCVeVhVQ0sMVxqTV2Ciyl7C19E9sPRtE9JbuuI1cpymEqOwTJwRV0O9SsHPQilP8aKIvW2KUP/My2eGhe+63X6VGLA7j6t8V5t9NyoKoIhcVYgVPQB2JmVYKYcr+CH/hlZFmFU7X7aYhGN6+Nb4ibngzMTxDsMQDwVe+7QH2oh1s9qSe8LVtQ1UJyybD50ZLmXLNRVTVm0rLD8rub/p2wW/u/Saq8Cq5XDYrvGp4mlcYydB//8eLMyEdLM+b3MifjUzJzVXUfPbk0TCDBdW2Ik3y/zWa1nYvXUQiw7VrJOPCl3vHGL63gU7qRPtzxUsGPXax5WFJ8GznVGtnIcLgIHhcWoijk/Cy9LPM9Qygv+Ni8si/wYj1JnuJ0SZ8XA5Sk1wGQHTTuEtyG5UKnIRkM8cjgzkDhgwOWBuKE3lHF5rL0St20PRbw60RNJJuT0RZD+sfWLNxaiB26hvTDtBuxPomK0i3VE6McjJjVa+NHzbLRtDSwCsLFp1sE10RQklyQBdS/wTKiv8Jn7ib4LsvF//tjhYb9gSgowYxxCipUGgc7gVdePV2vsJRZ2EY38InVhcZBcWoBn+hT/AeOEkmhTXbasaI6oQqXb+jDTjIlBcsu8aaWOYsmaiCdihSageiKdhAMWJ4Q1ErhZx2RORg5ydbSdNRgKg/I8tgYZPeptGVBvuBiUczqS46kwwVNqFrMsw3ZLEKX9K/5wzLcEWtGuQtyf95urnHep8EvYW2CuYOPkmu/U1oAo7dbe/5wqCKrlZERgNRYuvkx9ebPOSltxOifd7OJ/azFI0Fi97dkbwva7sDzmDNPLb3Ws2sR8qNkAAljhA1GoFoJlw2aYTn17m4Oh7UmR+JHYL2TZnYb1yI2D7jtoIjUoRyW/R1S9zXNxExJbFQsSk5c7Oq2ktbNZOy78gkzhvXmbNIfA0pYqecLf+PpvTYkwa9AvSixCdaTDkrEEAWADUiZ4ry+NeKA527ia+HD/+7JPKor9mqdgY/P50nUCE4BdJ9MFaKxRzgJx4Do8GIC+Cze0ka6zL9hK4JCkfrpWmVRkpGF69BwRSEXLPXv4qHBPhTUqFqX5kVeExxNRMe6xrYSNFmSHoHPb6RPPHqDtJScZOzxqSP60o1JypuzB3tut/UtsFXke96k+98Ynee9W63KAPVqYamo2Mn++n+hAwTyR6ZIbZlwQ2PPNCmwraMSzhrvgMQWwbkiAkrxZScIslFKpzT6qmcTVPsQ9A0YeaG4NBlMONFQgz3s1lDjSSGZkYhtfZV0V0QY8kQK6nWNGZ647hTJP8E1wrX7Ly0lasq0ef8F1wBlhF9z3IDiPxrIi/tChdM4MugJynBhjHEsOrfBXV9hpF5y+AjL3cKQLt3s6dT/VMuwP9dnWCehUjv7PhwVY0KosogYQ5Hd7KiP0NATM2u7zGAcgmWZ/npde2iQEj1taWO1OVwS8rGgUwtTEURnMzqfbRGxdIl7/X9/RO0Z0kiNYMMjwj7wvAo6Hx+YG7vHoreUH/4mPprB/wu2Pu9SLrqQWtVCuweEPbJwZMy4AHdIJ1r4xcF8WQ8ACfGdznVfXxFI2SB+Tck7OxopZ4wzSJdoAQUQU9NP0YB9H4G/uNcwWm3n6iGmnF3OA9y5A7Jo9ttPlwKDKmRXBfsdGCxmbHC3VxP7zU2OFDCYNLWDj5h+tf9bIFeZECCOX5uXSMBi08ZT3opY6Xs4naJ+u29xr4LhaqrPu+mgs7siAm6NEJmjbMXN6GyI+aeSh3BXBfgFYGCm5T8GznSIfi83UFK5YKZ1rMVksaQOt1pWVQphmga2jqo/rPgzsOVE/F/0z73K+9I7d6J01hLjHdgAZwz+A20l9e4lgx16P8Gv9lFVox0ObJWCWwhmZ10D/ARNA0lgR3PI4fGV9Ir+4LxQGtvakNroq1WZ7PI9ZfDzGGwtNgPcZ8fzMMKGuEpyyVM6lESTIGCk1bQc7k9n6gd8oTDPSBzleBWKOPz/dU7suhAtKzoU9i3FWEn9impkWzCXf8rok1AnlQBpv0Hv5gpJXz8uCOUrwaxkeZfCVh6hkkbmP/Om8Xc2UsxMh2MwlGtQKJXCm86BK4E9XCva6pJxuirErEEJCciYm3SuSbAjNoLhAbtqcQrUCwidQAix/T04qxlGO9ggSMHh9rE53dIbI7yMC0gyNh14VnSfreMea60NtXQYtvMRLEDlSn3pWJI0Bgy4hfp69Bkj7gD9JFg594YO2cudjWnMEIefzqYGcYZ8dZLAP7gxv0wkD05VZxkfKfnShv1UFIGxsmm9jSaG0t2IG1DE1Q3nzUindasq0+OHct9XeZAYdoFlDrsqwif9w4BLj60Kdpz1lYO/2WlVufswU3WZBYIIn6pCtP+V8aPfJH1Dtpz9+nwyTyeh9JaTPEQxXWD05ydLDN0gbgt9kToV0aTP/2EyzTgK2D2vJuTw/t9s/y89Rwhc/ckDUPfbueKR1KS3q4VUmy3vczs8dOnl/+ok/A2w6vhPy52npWysMzUc8Q9WG/EUwEUQReoo5lwLd+qtve2uVBJCemNSZ+qZWJGDRdM8nGkM7TEyoOLMvpzS3qM7ZrxPd4ixpcIZ/3posE3JxlX5vtPQP1RiMQo0IMMGePWHxx3n8GxQifkGBA229bmwPDqhi6xez2t9seFpl2PkEytOjzwOGsNtMx4WBsdTjt79qDjMXKkDZFYMrsYwTXdyC1UoAXaWiA0GSA3ghrfBPAx8zMiEj+TBEGP+dVflYtMTAmcROtY0mzSpoxIsCAKP/aem4ZKucrBerf1ATFdxGCXVFVP+zexrW7fMVY5oK4ut5lu9IIzgJj+W9YFzuw8zqT1YaFaQ37klEu5Lau/cjgrWXwkaKye1/zb4XZWVlgNvVcrxR2TlOwpn9fipxk7THWlERgKKqrf2DjK3NkWNpErcXJcI8bPULHChoqDIgmHUVzl4eG8i9JYX7k9NnFzAALxdGOzmo/6Vhxde3Y2jDPXwCE1MEkGGcNb74CmcrvWMeQUM08QcjvOPBFtRr9+uETQ9py6lINuZ6BV1hDtevAGL4LPbIw7qsK/wTFGE4rSYsNKZHkXJlPcWbbwhbUKhT9JMTExSaeagdwap1b3MMrENo5K3y9GPPKNBMkjk6hEQkWkRQ0s/MAsbxOp01ECxBmTliOfodyVDLyga3OSKWtlT6CyxudYqhyENMZ8iphuAQIjd8QohtBt+ch3JjqStj6BM4VuG9ndf4rIDTOfTjopHZuZAunQnKVr/WFNuxWpRL0wA38j+CBw8T9mlLb4Gb74U91a+wGA+O39S1OJbOWsY+z5xnT1g/5o6yo/GOkbfht/p9EPlPN5w4tLNaSZwIzIbJNjo9WmhKqPMQXPzmcyxg0O8rDEVf5cIikusTwkJoPTj9GlU5MmpiOC8KiyGNQ/KMvo/CIcusQggxjaDdaUU/SjTi60NkTI+Q1705V/SWFisj3L8kTAkQmVVJrEr9+IwAmv5AbwHpQs/eZfnP+mQD2xKAI3QAsRuF+ovQse2wrXc4eCS4OitTc3g7dkFn4DLFxwJJj8a6iQgvlw2GHK3RpfNRuvEP4nVQx9FPD7t9ej5sR3NFnO5vUzinCHFaO41ydSxTN6oOJTrL9d9NYaIcTzBTGMwo89vpUOe8GvmRuroNBIdE3jX/whEQcdQRgnQzAaY3yUyczWhAtNGptNpWWbWRYYJIox+IGyG0RxfsU7aTyBEDAA4GO7h4u9gbM+waXyxtekWtRx/WORDfz7KOYMzU0AErueXEmeECiOh5gyQEVqhv5opmijQZ5s3qxBIZirsAx8r/aF+mlm0qpieyKSOHXB+LB8R8USIN9KHqN3d+fQj64Ifvwlr8FSaaVzqLRtgnuepdNnoJpH5aXogFrOXg9MqPXyBcjk63+oiMSM+XUYpoxiOjUQHYR0CHyc3cNlK/zXpCaw31wTq59U1X40t0i7QbYgJEXT9/apXqRBuZFyTaR5SG0ncMNinqrQCG3fW7ZUPYRYNnMUE5Vis1yotQjrvBp4gxkcRKyu+Udzn6dQTi0meuv2n6QK6zSMFm+aeAnFN+mjLQuh9lLmTNBxGfMRu43kPhObeyJuaoqG3CUACRSHD3Jkfqn7RGkzId9m/q3ZQc3Qm17tBuUuehQJrCblEWbXgELTVHNz3N7iT5Q1b1SyVocXj0IupxUZV9/ue0G0LWaYLmcpye6D/6SDcy2v9Fb/lDmon3lqge82tD2x5dEjhLki+DbjEU9bB2V+W16m/YHanQZ7L/FaSK5A2kyveTQ61BjYBys8P55nI4xP65Q/gh2BGu9X6YBWPmJCeLhv3L+Yjvn08tfsQacJ5yxdZQcIE30I1d3DeE099+y37+FjSrz5rVJ1u5JM0nuWd0uJvQHkDuSLIEUBv14Ec6eyFO8q8TxHdhKfT+rvM6iS+kg2pGwlKwuIoS267o371aamklVnDGDaOXyr0gTbPtOu5UAvXOVhaDmtYWnkHDqSDn65n0p8l8Zt/0xUBuN+ilWA19HcfkHDoqfDULuAfucw1SGRQD3+8oCo0q8sTHAP7nBMe3CQI995jRpfm1ycX3k8cpTGspt51MHXU3RtO88b3xBf1a5Sviyd5QZ6HKHQbJsKlGw3QvgK2tKne7tAvS3Y00JCPuu/4VIdUzheE0Zh5xZkV7GsXCiZEXDkEc/rMek4JU2i+JjPh78Kb7jaicKsFKDYM8pAC2FFiR9iz4fHOiCJWSU/9bn7YqBlbMWMG0TFCahad5Iru8C1fsiJOJVtm2ta5ljZ9AL1D4sl2xHnO8KcKJb8GuGe0H9o8Mzr3kHrTetlklBB8UPSq+PlHyCL5zDO0z3Xtz6TUV+pU/BRsTRW1YfXnxX9xMmf94pNbpmHL69HRidfAoVa1trvei0s5ovz+RG6417/VXBi/7zzp5K7FLXl5j/ihvctPlxzXrYvbJupYLG5L43NISwlZ4gzQqyNk4Z7UGkIL9nwB2kc416ugnl79/B+yi02x4X4NodrfeFGb1ZSCYLDFND1sXlkFvylrqNIlW04V3it/SVr/vzmg+KQ8Na9F1xS/pd7cC4o9I78xeal02svMJzVTPpniD+LZRtaE9z+xt0LXqGa0d/bvqmITjgDIi7/gy5dtttGmlUUqgprlTbC7AAHmEaMzBXaIuUjTSvwQ2sL4Tl/d7vGvA6U1c7gZIBL0lWz1PoR1UmrhXVQ6I2mUKyD9iulYJLkprT3PFCpFv2Re5s66Kz8x3YWcchpUw3Dh2vfZnPgflgTEORDMa3bV/puuV2IHutMRfayJRiY6HTCCK5BdsZYSAt209W379cpngO2aXjyV3RrS16oOtc2jCS2NQA9kyKN8X9VWFXMITYFd32FD7OFtMh4k6CEEvGWarii/d1S3bst0A6vV1ZpFLeSHvZen/KRtnKk/m+9RDo4TH8HK3hWg2yozIIVfQ3qDW5kAWO7diUupatnBcfHLxULGgEQxIbIC6k9peC37M/agQq0paxfkOYdjhULrp1foLH43ot+HnpvDKm4xwD7lbueZaI7bESz89mesd/dUQmZF+SbFq0QfapqpHYknubS9QYDK5ppVzbdCfKV/f/WbC3Uh5r7+6cw3JyPJ+aJ+p7+zhUydScGPhUAfz36x9SPBAMT+iyErfsZyChBQSGN/A57KJ6NbgOwarFIqygiDuUorr6djgcRljt20Zg5tEAe6vriwuPkxMFY08b1iBpVcJ2I8EwClsnJgFaJOusYHuiuJgGnzck8GN81iUJIa3zw7TUS0S0gt7VWGzYBePTKN17KjgGzLWKgTVhDLGqUVfQry3HkXHEPec9wu/Lzqqt8o1uzf8lkhKl9vi2vcBM0gdNK51eRUNrcNHs7XR3N/A+iB/YX9pYVnvvAKMeMIuzhmz1N1K9bMvgypBemFuqeeejqcbBPzFAUyYIn7tPiySuNstM7m56Mx9PzMvsoSY8qo7wV+A+2qfH0eisxzhf7AIBa+EiLY/hnXyFfjJs3u0EMLAtUPVNqF1uKrzPqKeA6T6B08rOFcyU6RZ0AaGqKqqGRMtq/l6gKXdNzX562HKs4bEWFWd0MJ44dk+zGQe04qGktD36NRDayw7FTMaj7WAKcJa7/2mpTJkKFLEV3KqkJPLKtlF+OLwbQar1Hq+9dRe7imjEkG2uxJFLjTQ0Cay4onlc6GF7wYuXnU/bcS4KKyaue6lhzB60Io7raWxbDweFJnb598SL9IYEJap1KYSME3U8kH6B/jkPmrCqIdRPMHNELNbI6nt4UE0dSNtc+DlVPbm4nqXP9SHe6IamMGgCnlg6wWjBDlu221TRccQRt/79eO7CS3kGp2Leu1v/zKjQqhNUiICUtcFpBgT+snSo3h0tLr/mTfoe4QOoh8ySnVbx5WxZjaBbTtiUfbjH+hcyFmLDnsXqyY4XmONyaJrriIltn0Btc1qbIUyh35gIvWmMqEX3zz44h8yLpTKiWsifKlkacIyRN4qeu9x6hwapNahNb+Bik/8hUhDX9352ltP5phzxsrLV7LQ6wh+C/Pp+11wyHsptFbObLAuKxCNT0jLTKu+9JShoFPJaMfBvojqNXR/1QGA0MYz6WBGO4masDh9+C5Be7m47Stlk2FB4HGnNdPIVWRIMUEamg9F9N3IQV00/z5KOOYKx9iyQjL6PItD7/ffdbPG77fdM+q4hvqTua6xhtZmkiuQLSUXGmMDTgGta4eqB9TkbPjMx0c8H27pHgt3OUUd55Of3qC6Y+0qu2HIEYn0PLZccr5qbUmNB4QP52hH7BLbniO3PBoyK51ln8ntHClb3E4BPABYUzHt36wQvdEDHGlXWZoCs0jzGKfVfteND3gA07XqhgL+wRgUxkbSUqwFoxceck0GJPBk68FzOC81UhhNmHDt4G9A+6mlDSFKcTRcrpeksPgMrbFAkFpA9WSep85kV8G3LMbWZqXsraBJq+vUg3+ohKsvv7ODikij88h0Nqaa0GhRmrYMchJ4s8IK2vJW1wYIovxfou+HVvc6WTkNmMwf7UrnZI1xw6+rh+dIi4LK3K+MoUkpj3gc+O07Vbq9+hEbo7xwFxgRuZYp9mFXJv4G6zFmoyLwyOeANzlzK8YjxLgdpu2JmXLjOqakOofJspSZJ+yfcTVbH8sotGlY+bk43QeD9x+WGJJw0SDFEvdWrwDSbXJ2Yy8pcECIIGusJ46S2BbK6vk0chC6E5kOzQPa0gSy6fsnU8xHuwy8gFOpKT1zMoyAsdXLw27oHZOEALoxCBCli9arhRaMFtSOeQgllwe5mX0Q/lJYu+tl5pWtWLMOuyI5/DTJvthNb5RIEseiYBaKnHjB8RhQ5L7tyeQJGLv3XQ/vd8p5NBWLNyLLVOIdOsB9p5fbRt6WPqUZEySWJ7yYwdhymmpa2P0MzMPdpTGVctxFnOF7dx9LAQB0q0XplY3f6ELaqxXNhGfS6l+a4thFqTbn7n4c0Jixu3bz/SWhc27ZLuNfKTUFijuGjluZT/O8awsd0eeKUqzUFd+VLDPGBVgEz8moVUU2g0lan2A9W0WunAL4YDLTsfsN7ER4oKY5NAQ6OtSslah/mhmA4oaa/euQ65OKOFWY2Cj0JEj2JXri1hhdt2GGNQ8w7+Dvoe21sPTPj+6n6bHAOLrxOLnfQaA+l7otbv71bNdymLf4gf0B+Q67O9WMwZ/BAOVHZ0tYZbl3iUNH47rS3n76mjoJ2fmwbZQ/a3Fjt/MJJGZTZiZlN4AoiZLqqag4UzA8iyWQX79kkS/p3vdB9s9i7jaJQCK3hiktTfo/OSEtot/JIH232Z4lw+oiUdeK82T6jQ/SALTTf02OXj4LKiREEiAdX2HeXhK2t7z5ZKPMHzF1jQh4i4zlQZZfodPqBnOeStVdRBwHpp6Zm0sftNFusAbE2c3BlHJlFbMDDl2eZ5msY/senWNZRqYFbxQnOZkQWL3n2TAPQySsBJcjDBkQ4YXEryLMwtHUxnFngaIB5s7HdYz/dTPrVki3Yzq2laP9vCaqkWKiVPMHUVCI3zYhrPZLVVlFs5GvaOB1F8m0tosDvHgtFo0+dIXGHHE2oICICv0cgy1rmrDiB2X1Itee0ObE6QanASeyrmhpN17iM1/kpPBU35ixnMCkqfySs3mvsTuZYMmhysDu/sTG+O3uQCVDIHwi8PkjtZo6Cf0p6KTBKjok8Y/yjh0mnSAoWK4o99KvhrndEIiq+h/wqE6urY5e9v0OfK86HXlAdj+2WzZtMrOQG1bqvpr3Fzx9MdSezeILrWQm/XBve8CWtv4SU/u6BlFr3kPyw+Af4dm8sqXasaUN3pRcsLuywaU31vxUqjncCAl0XhI1dRs0Js7ZQdn5VdiPp1CGEJMiBTwvDa8MylHLBNcOyuqvaE+qwfPvNa+R2VAitcuZU4BkA5/ENjujuBiQTgpDPTDhwWrtyovRLS5OLl6F4ldXie9FW5T8J+dC7AqmrLyRnuuwV6Btg7P4aV6HZvp19V5QxQtVR2m49366BrDmep5x/ALi1grIQbS0b0yQH4Esp0CNfAIOIleI76HdK+ktG6Ylc+wm8Ds6H2Cy/M9RhELhg2yXoR8fpyaKZHLCLaOWO9vWnzZT08bZgwZNHnl074LPPT1ncgrMQ71nl/G28OM67lSVYnTKPmGvHrZeQcoUk9LiMhfgMuvVGODGRl9SsuPVv482Br5BCA/b0UvKsx87DIqMQDJT4Wr1awHeXyUPZypPiZbU0RVnTlG+2jEtaX6A+M82oSLHavWr/KNuEFY8aQC7eu2xHfR3kJHLXMymfcGSlCJ68N3MoJRjhfxQmBz4aIole1XGGdruLCWA/+2LxXNx/Y9NVp+/KbhSlLXQ5F8Pzly4YIFxTFVNSGphJsvFlyPWqzaSzleGgp14mAs/IjvKqjmEX/1V+TIDZIJeJz/pFH4loT3CdoCQqK53iGtydTe49yEmPQ/9snMUrA0TaZ87Dw80/PZdg4x1cQeyktOxyrI9fg6MpAOnb83B1F8zwAKC5l7AgdCb/nmxhAsgPA80kOI2+d5lYPHNsI5Ne04n3W5s7aTqmB1YxTEpSZs5lxIuSU1cuQHwt5DSAAdL743Me6Lyj/Mbdlgn/sZre6tSkkTdnf3fS87Tp99/ILovlvQQ6b50G06dzXZNbhbBEEMCXuDybZHHSlTEXDzRgGIRyqFDoYYwVWDg84FH0u3V9LUrdbe/pw6dP/LpZmitosQyTA4GwrT4zqryVkcijdg0XWz8nJIaEKmoe9KIdpBs4v2A08MllnHALTNXGsI4CFQo1yQd0DycnKpsoNGDNR0LzSYlvIDHIkuwang+K8pDCRD7kG8ASvRW/6Xu+jCTN0AVVkeJwcfKPdg7Hy2dyPCJWzXa04Rs2RXE1rwFxaPt8DWEzgCvf+2YPjrk4/kJkowE2xZyMlztXUZSTcWwsBTw83k6akTwnH2ed6G4CP0tDECp4osltV9+d2g5o1+3ajQNsnfCKKzamDvH3H2FbGavCxCXUlyuYNJC9k6Wxp1Hus90pL3+L10YmMgyBVWszP0756NexfULGmGDWtx/EbIiOMGzkmdpD5NxsWWOM7b+YvYZx7G1KPOgfcbKJWoCc2Tn6/eNjb+32+fiEVuCP7CRTFuOKf9r6SilERTidSndwOy4AHeMkZaZUxChtCS/iTQNc1KsNa42/GvMi1BfWqIbn77Wz6N5PfuqCiV3jtwzxr7D4gNUq9bMob9EOuRI7VwUOnKVRSUTl/7BvTRgAmnsdfUnSaGI6Y4rCCa8JOU3n+pn5YjQJG+k/gqQlOIzyRqWpL8zSAvHpmSYyjGZqZ4WLe9dC8+r7kdI4PNDKjKblb2+Zq/eDRK3kruEUG7jpU+b+TquqxnFdUbNWbQACKMjmSC4pVCRnQwEwr5FvUneJftf3xWYKpsi3d+dWz5v+gRiHU9wB6bxsPrOAzJBauLs1oLkXDl2jDDLEwLix0yAVbtOKFhgZJhmaloVYGUkuLpbqs04pTLoO28OFgYy/Fq6LQTANGY6fp26QIYwUwqYkpIPLOKMbtwEJjurFJnxFcGvi3NDPO/SLJt57KUt0P+umRkT4oUcufssJxtPD+cAtkw5B2ZeR3SplZ1NDJBxW6mUP3PV/t8hzx9YsyZz4IYLbVP+UJcuTNRZauEXCf4lfnsk5RGVpKYagzCrvVf3vL6aflhpE07MgkwKiB8cNKyiVJzTvnPQudyDeBWmCXD2aFYoEEBBCG+JHFOuOXukzjsAE5snzBoFhU+SvISAOKhBoIaXfWNalKM1exyiSFGK+XxSSGzcOVTlcVvexFjNCES1VVKchdbnXHPy4CCAj7qgeAsFJfYKHwk8RspiGINMabmgmRpaCcDY0CFojkU75MIrRtD5msw2EbTnSf6jFKGBn7qqxu1Hnq7XhPwpIDc/OFrhmlPOD2/yfPk6uxNF7wMac8yusIgYRv6lOJi/YbZOHOZd+U2LqLd+zcwr2A5lt+ktGa3tMQAxxHusAU8aXHvQ4U1VVme00pfMNXCHd4v2y60Zksu1idjFz84OoT7BNOUWiFy/hniPrBYBPZ0wroK0ogfWRaTXLH2mrxFqNBbUjxt6jb9SFn27RM51h+VoeYj+Ll9m6neSO2VBNREd8vMPiK+RmrIq0wcqt2eEOmmAEKa1LWYJPzWQPNPTOl+QtDc8zqXTEN1BglleKLET0brzWTZ2uuIqq1DKucN4DoOkYHxG00QEIN+dHWOynrrJUThJMHqyStWZn+4yS0SpXhtA5qcelGWDEQd3S94ZoEugBdmt3GsHswlXSYGoDhmf5bsT1JhyK7gY/nOZgL16Uh6zGCQyKWfeiBXmXTUTZZXtSIYixiKl/tzKW6EDznQxdCPiAENmUG0FSLXve3Vr/Qohv/PJ3AA1KXCJDIzcuGlbFUW4+1Wn82YBO4tFf7/LMFqaF73PMzOKZXJjwHvz9BhRL2uxqQ2Dy8/xvEVmzlPNA6XtxPkvRi/V75a8jrOnz8oDY0r2haH6jFLaNSo5gFfjwUiSmy8juAxlYSTZm9sXWvMhtn9oZxfYI75Wi/Xbhrcok0cLR9KNVmwnvkubWEO6dNUXJawfwBRYY55coL4YAmUX4Upa2G/mXLIbENsaBZagWA2QI3LxRkEWUKoWE/PlFD8DSQDlZh/IA8jTJ6CTDsaiY1lNcjSgBrgd101ROSAKUO7LUf6/FVPIZpL2tCkJHWGac++WGtOsaSztSpyDjHaMRP9kD+RgixlhOzQCuYQn0+zfDJp4n7UCclv/PEVS5cCcZpTcfVrdMgzJ+MD0gYXVJVGl6iC6cLwEZEv3VUmH5SDt8FxFADSh+RtsICUbrmF+TAGE/6nQo/fjKQnU0qCrB83DLAqTFkPoqSL9VGB6q4seGlYrH7uO/Kz18kG4G1/UXScDcTobWRwUzUwaX/DbnZs83IDoGcsP+pIMWoHEGRrwkJ2bepNlRWJiwqmhoRngU5DkEje77hicglYm64VIOk5WZjAycgc1XhSgyDLwD7/YCa8CtqYxDj1fl6KmqiiBY5PoxWP8lo+zohSBSoGZlC5A06oBS+K8eoJnVYyH2IPFB5nXnp6lVm9C+WEYFAHEr11VBShSk+dh3zJBOoomvGN+F2pU4xQUU2sRBQjTi9D32qrvZ9erl+6ANL91pK7dGyvBhYu0PWemMapm/m1Shauz2qaReEd7v7PaWUpNo4IEPjogJaQU2QKMvhlsl0WCGEMBDNpYDQwQNDYoueAKTBrFRVSXo1ykt9blgWgJVfLCXUUZlKr6Mgd4C9wK7bEO9XwaIHqZ3+sDhnG1Q4lxGyUbBWo0hMOA32toJNfurcKCWE/UPsZk3Q8CEGLCQmAkJZtMNsjy5b1SlO7urpRw3wbOxosWdNDWNBIUJJcE4ozcMY4dFhD06dvRHhF+0haMTHZv4Wx39D0KHtrufZsX7FTNEirnO2sr7JnIID5+Ne/jcatuGfZPUzWLizjELRERg8xefpzeS5/6t4oNRsrNdV5Hn8RmkmeRvZio1lo2hjRHSUDGUoW31VT6PdmjGGiUdb6FW4u7/He7mW4lQZSmRLwio4U3eHf8HEDV13ravFX2exG3MZdVJPHMpyDK9DJrmqo/cEID6oEbMIHaOlhuMxGwDcJnGxis/UWq9MtRVNuuN+B7Qwk5fPVyQ4dScFhJHXzKB4EPH0mj5uHP1xo0R8KlJjHq4MWGc24wx+MAi7EPmEvpoElVzQfFezoTiRbQGOmhmj6xWtD1qcbd+XD+PKUqZxLxMHLjCdXyxKsVlJypUf6CxlW5OwBdPqb4AQd0eiTNiGA+apfbVA6gDKBxgv8jDsSW2SD2PF8NU/AHDE+qhmchGWcBTCERixwsVFJZ2aBS6TmdW1Wyz7Lzkd2w01qwQI4li0gJXfCW15wQcBU662l50751Fof3/d/H0yp+xd8Xi/UU4wpONTYe3v9KMAu/omxBIDl1fprbmJDHDhgxm9ap7WoQLCQv4q1IQB5CYE1yuENfrLLMDDvKZN7J5yxLE8Z/tYAj48/Z6EvaPJh+ci1PnHCX0T88yMAOZ/lw719EA4iVeRpp/3oblLDcvBxPhn2a6VuFapX/QFNU6YoO7H4OSqWL1+ux/L3LwmO9HTyApgyl/aNnhkvzF0VLH0oSFuHkZLptlfbHTVA7jNilJmoaQxHDZCEI/JizD19zYGkeVZvwcC+eDKGZN8JZpXXtDMxbmnTXJaHjwrN8C8zIT1bsZBy2NCA7Wzr+8TaKG+Jrb2lnOLAtGBsc1LJl6giF/pNa8kgCM4Hfvd5/59+D2z8J/oae+DrFBrEx9aIJQyIP/i6v3lhFRKCqMIisAay6AbLAV6TElFcAAfgASp4BB3qbEG8ewkx5cv0U+wh5DCPRALiRgnHZD0Ws13DvH9RQe1NhDW9lamtdC7FP7hqznj0t/DVRSP9BTlH2bt2FSVzz27zdeYTYItIVfQfMVAZxnz/q3Wi4rr2XjdMHxRCg1ca2dBoF1Kb/tKPYcwu+Rk0eLXYfN9C+yiBboY+jgSeAd9IhY+7Ed8AScg+0ZSIkGH9AK9tmwfCN/2TqjsPFhgrlbqVoRt2vYPyInx67w531y2GFXh96q3wokDyr3WSEd545GoiPGocxPiOH0tj0FdMHygAHiWME5yW7yce1MMHQXVzxp4XKmxvI+vqmZUfDAZ3Q1rQHaNoV0oE5xGd/z5cOy0gs5h9XsLTDa1uMMwHSLCF9evqODrh4YsJ2kBr0lPlWvnUlY4RE8lukCypB0UEo/6/tUzVQoSU8C8x1x9nwcyvkHGKRJi5/AG3b0brcMroLweeGR21Dg3aagHVhve+dfIykFer4OGBPwuzOIUcUpRoG0u/JsSUuQA7pIZOa9UG0X1qdd69h2uEnkWeL857slfaUM/OQlG0kr03Dpd5hl9dYdDH6p/Otj+HCql09bUD3AA36UUK/GMXWnQ4ouJoowqY+RvEhr9KTu2JDTDhf6IjPderhDh2N+9qM1HyfdvRXNAWqGPq/4CEJEE+3/K4PNtOs5PS97JMojhvC2n1h87Au2yVSXm0+if2kXrcLc4TDaOjLt+O2r/2OWiqKhA/xe1vovzCSDVKJaQutpYAj2EduGavFVu1090zTGrN5cO0kSXG5oNYQcdd9hvHpdwLiWebeK0GAuaqkhkOejy0UWuZNAwHP+MRJwUZjxTBuC6cHigmUMWa7pXQJf56ZWga0QCtof20mv9QZK18b6z7DzCEtWAE3TX2wIY15AR22cmqvxixAg+y20t3+MXbKcfjeeuiL7I9OXkJdOWmuKWvPD9J3yGGT4C2qq+CnNQun1zb/CuVyHnU+sx4DOQGs1SQzLTb3bD5DAb+qX8UyzmcQAYAFrFqz7pUd3KiOs1hkGhN8JkOqvX1wSDUXHyQRjHt72u7geRUYaR2GK6bSSq1YBHPnt/tFouEGJPAC4N7p/TIW+O+nBsH43/J8DG/dDPYe/8PS1tvRBbwctfhjpa2cvVzyvQmrolJt/L2DF+ZjbtmsQVOsdhoIVJ7kV4bZrVcbQ+4HGRV4CuSKLe3K5n2RQjdi6CBmJoB758a4qpsf5bHh38GyRmY7B2ugdauV4tXvrMZ1q194MK7CcpqV9S64PKkIPlx+zVm02FvjHhn5NtUkNGZvsAZLXehGxdA2phrx26fodDzoj7HWObkLJyWJHu0/mbKXO0mxHmpIl2YORxs3lVFE3h0uDibKuDjV4cvBqhDXgMMDQFHEZn1wuEoR3Fib5xlEoAffmsNFq1+tDrniwLo8CTp91sq7od+EAJQvs8Idy8YgF+ziVvJD5PXarGFJnMuQHK5XwZGr9Rtdj1gQeVMj6YOLN2oZKS5UBET+KlKZBXOub7rRBlM6BtVzkxmXTmSZOUhgRr4qLXxC41kNTaUZm5kQhzD3qVMNvKi/W8sPX1k1B0/T54gcxqYh/z+FjVRdM5uLXrqiz7HhXbHH358v/mUP3v3PB+TrAe2IPBTxs60jJQIjPUzXolVG4xfPplMHbC6pN4qMqVt3rjGJy/piZGJ6C4B9fKPkGhudUSxZo/+GJpSzAeGWVWa/vCKkDVLRutrxPQzKRDcqbwiJu9IwRcH+VSCjKFvDQIe4+4aSw7RbKo19YeSu6znjwFtha7UpiiLndZXCSkXYGyVZ+OihguImTQTVfMAJXpu35fc5QdW/GuY3R0mPZsLieKNM4rUVqHZNpVClcNjfIvAXzsABYc2PRz2zjQgN+nLKrFO4Y9L3WoUfatbjSpW5doHer8dfLaIpwm0CttB11SGbQWBrTzjU0W5OkeYiXH6XiZzwEf28nyp1mqAbf2utgVfv1ird1w1/4kI5TMxL7MSn+ALx7daIDVEYyELKedCPdPKZsj4+qO/W/TBlrjYKhNZxE4l6KtGXmGxU73wVshoO9BKnXl32lPYE3Fv8zC0gya+tHkQUMuuETZZblFfSFg+Rkwx3rRIUo72GiKUSX3Sm6IT0FbT2W63qZCduhBouM02hSuWBFmzQ5b5b6UkU2yXGfXw/vd2qkwza93l+1Fxd/aihLhP2PUqxFdYwSFoQ2abiolJY/zkXljaKWsrnA4CUg2xc83ANCNCWuOr4JLU1IBJeDeSxz1436uQOZUst5ciwcde+nImcWnCfMqIyx2cuOiZ0KES1yU8jRPMDkuaWs5MUMx7OzlRje6FmHTA12MGrF+BTXUpDnLJ6lkcdgqX8pc5oHtC2tYCzuO7F+TnRiaF0BHLQr++Gd1jIoFhymnqdqYXyX+MgCV9yU42Aj8xlHwTdufb98O+wSfCTVynmYXO813aqebkrHB7k3jrr0suev+/dCpmDMZ/lPBhiMSWur3prqwcRpVzcLqu7HvvNqY03MH54Xhe1WXVVUDquudSi5OM3uGaxH6P9roDjgZjHWa28Bo4ye9Do1FD5zWiL63n489wA2yP/IsqgpyEdPRbTqMJcK3awro5zVF6g/DGpuKLRDGbDt0bnx/4F2mHWWliPO4mUXCAjdRzEHNRZZCWBVM+XVmFKgk6wHOEZnU/agAx8vl2/81+luEZrbDUCUm+YT76nmV3jXo9sPoOEc8fFPAdTMhzU/7IqASU2YTAdOePPbbqqAMOgc5M8qJKjZyAoS+/0BwVQUE5aMaR0NVt3fbKEuAcaP3SPK6FVRPadhrZMfFQq0HEBQMhd/9091eTdOn5k7EzmZrQERhwAcIMj2L8R5AlMi26lP4CLHNCLZJRdgFnHus4lCR7sJvGcCXrePNdlvclirigsnWJ8pLH4127ocTH2/gRMLDp07nl0hubBfyRXNNJxrQFByxlUFagtcwPuw8ykW4HOa4rxyXWylNCWEL8vypo3adCQ2m+Pisa61Ppaz9mB079vbgKJCB2TlvGJFMRZtMWPsq4cFf+a6vXZ3e3g4uN3oGNvAmDvSn6KM14whRnqmTZC+9xIM8POjHP2BqEqDiaPujhSAo67nk8Itiut5vsnTb0Dwad55NS2qP4JbML2RHlr3mFcYkJlvdDtj06Lxw2WmFLsW2woh1bE+pTPW65dXQwcf29mHrCPpnAvfF/niKXzmni5PWWjomCOOA1hCswsoWPD282OYA1zzhLus8ZDL0+CAe6JdU69IiuM2mDsnSFlzY916Jyph3CYI9dZek3ck3Ca9UwSxZ6tbn2A5ih3Nynf8IdjLalzF74ikNt4gCO/+6CoIIU5InUjBFLhXiihSJrgrovVZXjqYEtqiMI5SM+2hxuTJviI1PE42OUuw3mVnkUdceTSt8BtIbqGAUTqZMDJyQg2/a//pd8rpDnRq+USriYU2DdAeulKSMt05bpzW354HRU9j5BKAm3t04QUxYtLMHgsCw64W51tivDM7Fz7vo97i7zAcmTx7leGA5RHHO6OwCOm6mfRuzYcJg47ewple/Hh1vJb2Mxc1I9BI3okUIUryCwciXbAnSN7FFxZrxYL5jZd/JSVbgBFPmktBShWsAo4WU7xmfnFjN//yIpefJGyUr4EjOUSLyGGXqX2C1iJxAFFU8ID7/yfvcG/8CWDmyVXXOX4Aqpqbq3Qw2BupDMEA2jG4v1Lmp9aOBwicO37wHUGFm4JU+dV4GDdg9cFcUAqYtxz9cQ57rarCUhlPrk/OaTJqb1YWpN+lMFHmE9buGO+2eCW8W1QKIFHzgPmbonoWZ/CE6z/WFIEdNPYvLmyPjzMJcEGKGFD9woVCIG1hiBj52SJLEKxM3Tl6n33XzAzhbx2J3OM8RSWGC1/2pS/fQKmTu9dbgoR1F9bIpUWnk0sk4FoHwEgNS6KJySC1eh4E+3aey+l3cvIjOqLJ/vkVfBVnCsIfVuic7No0GBf9dp3433xuU7dgWGbRvygxhTgmOoWN9DcPoVsh6mGeRGPBDCbG96slH2Xo2r/vjga4FSRCuel7I/nsYYzy3fBpdmO9MX/HW9JnM9e4dhOO5gl7cSN4B1smE/iZ4TnlkBmi1bVu0QsaFeoM64U+GyIF/tQRmNtDnROYM3yX8BlKSqS1ElHXffKPhrP38FD5g+XdZrAqD35dy1yAa4pV4V+myMEwqvNxyVmRQs59VAlZAK7M7P1u7bwhYl9/Q9wuyhFcRu2g7HRGqzsMvlbXO3pM/qeXdwyxwvhhSUtEv2+aK1qT1C2ZM7T49RRysexuRn4clUSDdgri6ym7q6R1x9eqFJMLXiPnOsU1cZq4sXlSj/qmhrQ7W94tjcmejMgUViCYq4kqEP+F5zP+qz1b3ekTgfmvtYt8C1zpS9fWLLAlzUfuZ2YhfyyF0HcDE1F70pIcwPjs8DUy/O0q/sbryItKB+O9mBhDQ0BVy3B0uXRda7Ya8HF8Lq9sTPLLLi84ObHeFi1vMtADDLGzwq1rTodqdpZgyLBV1FL+vmg4Q4B1D+K9kWu0I/IGwkJKLGu8MKhkXMbIQeSXbGdZ+LFJE3ziZmSw/rmcIvwhCnp8Q3Fwe/HIRXxoA+XQXoi/irly6GutTEDOuG9XOFhrSlYMr++5Of/3DYtV34CU0zbAatxkw/7exU/FZJyfGNCAbHAlsWQ1enw6ecJszUx6jTU0TvpPatHjBe2J9+D19oY93El0qbfjw9jgags8RT/NK+dmgzNk4KE6cTsI2XW7uKm2A2em2q2cZhohw1M6CRX5b9URaR3kHIvImWYw/ftO13rMzQLK+U9J76dTiTUGzz4ep1L9S8nD+n84rUvrdb6lDq9hmRz7znk+0W5bL0h1wNJDxTZoSS/IwMGPnuo1doqPvlDsXX0S3Y8F4Xro/Banycrbj4bqn99DuXrw6wftvdotMU8WB4ppfet6yueLerILdRo2kgtY0p8kR7lrOf7Gt/ttRKaUzKxgM0kYBPCQgMfrwMm4QV5U9zD8cH8zF3NeCJNw3vrU4O3G5DY2yuX7aioH/i5T9ITLfHO1/lBaWBUAf8qP5iajFZcmCi1dkrKJlx3a/6LY6ducJP8b0Pr8K+t49RFImGPrk0DNWWRMH2z+35Kq06JgTJOdUBxWqP4keTNg2+cI7BVHBJU7Pj9NcjRt3FYmXkh5wqzMQGqcsx2yINav7lHAjv2jl4PIxy3cSWKRBpFfZAr5/pjw5acFUNp6adA94/+97uLtHMUCcUY13MA23uaypdXKj6VjEfkBeZu8azy52wKIYJdA4fYmkV8r0MgCCv9Ke8RaIIjY28uRxjG3wf+h4moV7e6Y3Rh2yK1a7trZQt8IEMoSYyXv36gS/1dv+L8saTuxJib/nwhLNGZBQ+CKcXGtNqEceVHyGNQH1p4UCfoQCGlC0K/4QgZ5dMoapwTOkvpDKBnP5Q1XX0FtFCaEdLHF7VM78yFDSMMay5nP2FXWMYNVsGLvl1AWVQHj6j6w+tflFuzc1JKqu9/ZPMUKkoCMCFam3XLVscN2kxacPJOVLbqPvpYyBlH4yMp2rk9o6j0tGI16IU8yjU00pi3SAbpIWQYCTXY8szK7zDYmBKJLExztPOP7J04w9SuX5Jox6Qy71vPozMOSpjxlhINLuwAdPTcDK4QurSV64wNDqx2RzwyJVkr14usKChzaKq48SWK9ik1K2EgLFzGX/G0vRGiJ6R/sqTmgaElkZ3zCtCcX/VqonNDJdwZGc4W4pNeVZGY1Om5XKsPvCp2bg4TS2rtsKGJUE1OHRAgbOPjMQNgidv7AwqFkignraj7U009tl3prOR5oynfc70Sb9fWYdBmBS5skG2rhtT7sFfWKqD8n2/TtrwTLwFuOgTriLH1eafUo/CPUY81YLnBS9bB0NZo7fw55v6SyNEZ4cpjY//ioswePRx26y1UDICNNs3mnjMfR3Iy535DAgC8LjSMzX2E+7Tjx3rXoARP0VAxxo7kOkVI3DgTq+I9vyQJ+RDAJmfkn5Tg1CUVzJTeq0ria/05EZm8m3wCqa61ud17fBkGzt7UaGFcNl0AIeTWz8IV4vLymh5rdd334KKsrsP3me3xj94hjhhmXL5deGTkg1iWTx+xLPz9csnM9Mye+y6q6+vaoCfEb4Ocfur5bQc1PvImdtg0S0C2TS6a6I36WGb+C/Y5t30tY2TgiJ1XAKZn5PnFDDpKvqpaubppKual/Zjep5gyKtglrnMcIAeprS1uf/gotKhz8aV3mkOMlOMOJaWGfFuGp5Vthu3SGGhVRuA8vr//IOT3LGVZjtUGqD/nPtneB1F6xmRIydjWRnZOnNayS7kMUjRct0rTvtqBT5TMwLq1dkQxQKpki9ohVdpmyDWeAojSDI6799oYkqDZkem7/Rg6v1W/X/cG6X/CYS8uCfrguyYVrlvitoI4HsVOn/GqZltLWZbQAyP2zLmFexJ7DtN6i6w1kIpFuOXQN4Fxw5ZRLFuHFz86dQPj6qV3ZndBvnlPX+O0M4nWcRGMWnQqkRmk04KRSnaiakndeq+Ozt90FzN6JTzMxCWt9UPhUR/JKBq0860By0brC49I8rUx3uBX/ciWgwpNbSkcwkGXN1TGbwTjIaEckMkTB4klFCAegn66V+fH/x1KVMGzO67isak79RoUc18T9ay4c56HKohKPgogbXOd92aEfLaPw+KG0/n5dU/JVTfAZRBWbAT15qPi4cvchh5KTP4D/mosYoOiGKu5aJ9GdX61zXK6akE60t+S0Qt7ogLKO5LDTdbIVm3KkOTuU+KD+s6e008c4eENyYuSGQAuCPBSY9e3gy/5qxCeXRiQXI63a/U9n8klu1s9lh/USPWf/FMtOTu3F3WSc5hMpiGaKo77Fj6LDAHC/+5sk/cK8TU/erNYZ8XWbIr3N3OrPFj9kHgLvHdf5dZMqlhrre4PzRY56VufieGeW/3kvU59AWuZwyuyW1agdUY5lpOZ09oIWeUvXxVcfpO2qkO+lUCNIZQSgHtXQeS+PFWffM3S3uISLTjtscbrB9aAGUcG3JgbG4gWoPc92QvyLgewiEgYrZ7mMWAeDNg8QCS4Uzz64XgRP7XJAvFSDBLCpekLwXFqtRswX352g7K7Fnb4nwTm9PfUCqfcYULosVBKdeKq1VldBtI67yNCJbPCgPsflOc84VixdSNvk1GqflWv0w3mmjEqv9UAIxnB5vAYCmtK0L67Dyjtz6tv1v/20/NRA6iRh9Gny2jK7oHqtUjJLSM1QJtHxuQkYgy45PtRPOg9THbIu4dsNuBaxpOeD5LEbydhcgat3W8IjoznwtWC2kozr7z0e9xJgYde2GFk4oPiFwTQ8osEGmaVZbRY/xBf8jY91rQ4Mdvr47rfRnksOOtu4pv0jYdBxTtt6gMo0QDWQKpVb9dawIsN1w0nk01RjoPr7SQW9BLuM8y9WQUj/hKitXg8EYHBv7acN35dmB/dGa9Z50G5nYX218+dR89I99eG1LZduxCZfb62d6UUfgMDxF1mmmxatp0KDweClowvRvsK+EWLTJhi7F7DJy1KnEjBC18Fp1ZskzvnH3l+e5t/AFEXofdK+Pk9uJq3WjwLzqD+G3/ATY8JRvslUZbl25hwZhNfXvKMDbJQzSUBV4NNgVK4mWitJvjfJIy8FZ2uuuXoDvx+Z4Pnhavv2aK8eMfbtusJ5kW9QRfexCOcrmEDd+nCo30Eg6+pJpN567lYLSaDD8VYsdvHcHjarZBW508EOgLLSDMJ7V8DyKcm57bKJ48LuT7iGxKQuDUfs+/fVfWLX4hp87YhZQf/6JMJCnwup5T5iLutCVd9Cu31VyP1DhxG2ulPsTLCp6EVRYTIzvAvwV9W4mAUZQgiCxh52wJjti6IAVuZiRb3WQiEI6b0dp6hjl4EBWz/XZ+EbCv5c8IMsD+R5h3SdK5FgTd+G+LYFLciX3pn7Bio+5cCJ0jsnR2XqMTyRx2uSFGv7/rej8Pn1BBDa6vlEmioY+oF3rBUAtYJ1Bjmw6tQEfYFurRIZy7eZt1ftakA/p/dAX2jnUrcrVQsCF6jwSlGE6xTggZDu5RcYBUOnWUReRdUV1lV2/CptO8wAmyFsbl/ScS/81SRON49tAidOPfLtJW+DYUP6eJLVFe+cJv2nYWHzLjIBOft0H5u8Q9gGGLeCQmdBjcGMjo1NbXkPCSv8ZTCsahWHTXGzJMf+9DennFp0/tb3hIG3C4JfRHr6lHgKr0vopEcobCoEcjwcI1+1u0/RM3J+NDmvClFtvlhaVkcA48q3PnCMiZ5YfSlhqPIxVBCoXxNozd962VSn/l37Fx8RArQU+TRpUr3CVaNbWdUhDiJOUKcDEvmTh0dixqbVecgbpugLJCZDSDFTs5UPtKrf72MMOtuqp6+mZE07OwSOPPV8VavXjJ2UUD/rtYyJeofrnpmRQ/fkuuNFaudvy5rT+0fNNBwE8NLRYY8FVhLZ/zK+08sjntD10yjtodpZm4Rf/KvHq/x+2M5vL1a6hKcbMNbGMwgKPKBo3swjEyfpFLo9w+uIa9dc8OU4OfX5LdLqFAPqCABLORPg6wWVob89ituy+XPBiGCSUHMJbbZE1FKVtK3TQf+ylG+gcUmGNZszVn9Dbv5VuevSTWNfJg3z3ZAZOdzwFQlkPYp+PiOm6wc14bjcAdmQb+U8L77KQycS819Sb86/KTEp5mb+xohz4FU1FAiJnPbGC+GjKGtk3guFQwtFdbMi0Iq6tjrEq0qZzP0mpuxut23gzyyFOfpdePUAsfnas73DPLJsZSyP6a2JnA/oAhIk5buHSppClA9Fr/yVRV9FnWvB5pjQSEtHsSC2Ocecn/r2aLHvWj5tqTwJM1mIthXNQdPB88ElS+YaVhNcrV+hF8GjJavC6LZ2cqNhFNbCzbiC11ulSJ+c+Tm7Iw4WZ6Cc+akwEoqB/+HtziBGJirPApzytTDwHY0Vi39WF/S+jnou3FjMp2UO+B6KGwy5e5dw9Oja3/JDxtIQbKVM+bgqYAUWxgOyv0u1GrBT8uvZviw+MTk9/FUoZgjvUwdiB5gjH9yzIEdGrs1QBSLeTtaV6ixF+l8UhjaE4+8c4qJboFwMDXTf76FwrViCR/Jljpg/v7tL67tCQn754rekphzkDFDP8TVFGhDBmH86BEwJmihKmGmzL+nRIlvRJwQUv2tlKO6Am5/D8/ZgWEMBjNX5urOmQm7nxHxysIVtHwVyw6ngXpfC0VsU2YWUMk7BpUuPPy5xD20tISkMJcQgXpQbKCXXM48yaDU8o99t6NeeYaw0Ml2Rbri9Cl4ajt3KVYLzIej9pk+a7nUoCq8xFMF0BnuFawvpDujXWBXK/HdySOztBH8oYHltkGiMcL8cu1NM2ugZbm3W/nMNYNxAZGF7rRgFS2Fi9CuRu+4+P1AJdRDUGHvU9TBwyxU5oDaCxvgqkyis2wENalQHuDt8z4ehl9zrowfvthIphVEujcnR2LYZtfz5fZ0fUHlZAmbbc0D/b0ddQz7j585REZUnGE6NSneD/IrHKF3HBdXeIrcryP2ep9C5dWDvPIl5HHko2ygSstyvX5WS4bbR+/arpqLho+AJI9EOrhuGqHs2ajKYBNCIBwUQYJ8g0RvxGNDiaHbqqZrJtVn3+EwHK/vCUMv70wUar/yWzsesPs9cdh7ocyJoZFBRaxN5MTr5OAx3i/NfkGfh8S4/guMw6DPIs1y2BWXzpKsKG0Vg4eegyb10yadwwcKa57i98H7ZBFELb0lzFevuem6VYx81F11RPlxPJwZcf7Az0sHW6zv7MOQV/BX+VwIVCDB4tHgNsJ3qjga3Ii7CsW9piTJ2DNYncY328rcPek+0ir03NSTUOkxPm12sFOjEHB44GiholKH0n9ovIzpS/bXGYHRVOnAiZ8gUNjgqDM+8sOR7aXUflJaqbeJwScgi7gY8vu8MXxQEHkYm5WRy92/I2/7NwnubjzozbW+G0VQ/W0ADtO5qm9+g3YrgxjYbC+Z1bdbH/UrjDxcqIodmm5pKPBViBD+nQIxTvV2/3c3BwctyKdR6ylT2JLVGHjNgSjoafyiz1fSkz2k3ddrDl4v7e68kMRS8djXT5Omh37GknRhO77joWuNhfd193eyTOZUZHZuUpUKd94X6et8A95yaPbaPp4h5SqMMQeazrATl0mo+REij6CGO2WImO45o5iFwZ2dL6UWTd+XZmlxEBhqh2m3kQwmaP3jyYBuNsKGRi1AupSrUIls9IZqQQU8GUWieUtEqB3HXd3knSO2av0HxC0ygUytmZX+MywXIaVfgGG2FWY4+u60U3jvMDa7oBZovRyC5CvRUQy7uvA+APO4bGOjoVTWLG9fyiAE5AeV+JnE4sVOJqvJuus7ptIvVjXTiN36yygQ6INy5q7DBphmk3RSICSrF+2bY5W1R3eThy8XUT2LYMgyPVu+sX3AUFv9IWbKIMkWuIUQC1LuNUtIc1Zxjxa28THUJOgN6X9VRnI4pjBGo3Ah2Mwl9DoHo4/ttuHNnu53U3yRuA0KOxn28qmzf99Asm9GIpEdpt1MQqaGxPyxm/ZDtOI1Ivq4DcVPfmvY6QYp+ePSBF4Ao4ANAlZUaHUsUiwpNlfa/nVhwuPcQ8FOjq2QuIjZtbz3rIZnxqn8WXuqV6O7iAb8fTUGT2oMvosUWzz8VeLTBJo+k+/39kpd8qwC5n1+/XpLhNCXN9wKJEmfnEnSUMKHJHnmWX6gMc37m2R957CU5LzDtmKcZe7jALCQT+CuaEYjoTicKStN21epca5A/xfphYmZm7EcYRBQSBXXmiMYtbxsnKk1UOkzXf/+5vI7lHf75ADhdXjV5++enydcf0/3UPnv8WrZ8s8GUi5B8/kLlhH2HbS2O6jiWmXns1I1w+SMb5fyX2BNdoat2csy0VqXo59NxwHQ6kJdAHH38NwNF5hKvRSzw72ZV5b1d+2qMvTjt5ZMJmXCvEDar5RRnFvpuuyo/Cng5F6Sr10EWLlFPw4NSd+j72XPrdyvfHTC2+wvcxk6keGVoC8JbL53CJN69bS04u2AHl8r0/ONZ67yJjocLCqV1Ke9l5ct9rKll/4cqwC2/ldSWmIPmckRUbRcMsu27s5AXmfT2ufF1N7q3UPAHd/6hVKKAI7fUo/LWAARhItvZbAAyQ/WJ/dr/NDYmXRkFs1HJ+gMYYvGuk1lBSNuhJ4g+DmXjd6iBzbpTLm9JzjD+7/71vPPiwlk3zLnhnBUffSemmBIglTiQcPro6AQTj83sa+gToMpInTvjJ5OMEGFQRxYXUd8UPm2B5y4ff7g+DXe49kUMtB+zjN4PSTFzamrZAVg/EeREV7PCG+KNaMwWdYQjAtZG7cBYWpyHeMkpWe6aJdVEP0VDHKyIHatQnHvv6FmgPhX1xFvXrgDqQuGjFhsHqa+fehacM+6otmSp7aWGGOKNzdUKZkPfOsbCE5SFScd0Pl7DZCCvZ23iaDw35Je8l1CZq8lfoH7FiN2CheiMdf5iXGRLVi28XP2yOz1R3QK/9AKkB85bXrbdYbJn8j0aP0fzfrMuf1wLw1uhv8eiFpt6LpenCZUFp4Uqs1reuONyefl4tW2pWvAsvc6e6rF/B/Le68/d3e0Z5kSgPFjjXQHA6t+UgMyorIXSRmm3OQwjtVDt3TVTR5pRAHcCQcOIazVXr66FC8aHlWZ9ORDYzqsOI9sC1wcSu+XXXOrrtwd1V2OCqVMQirPOVQByV6VVER6ht2X4geGFdo50gj1xF5NAJMikc2eYE7yI+xsz9uM5dfE1IGkEuDLrymrJ4qEWRxXEk/EtzCDaUs5MXbF1nfQG/zQghceY6bNplHSgQnGVIuGJ+1EWqSs4FLsLFt2b7wDIi72ozcUCt8VXdoadafz3QUcS3LrNoGtBxTMOl48ytczFvCGyVWEmONIrjcHF1271+tv3vgZFyvyUzhgllZjQPoQGFSMlhNuvqjgzhCqa2m2Guc4aQpTqo25EKF9gbxFdYH7CuUhVfUM4ic/q1LZJSOGznNlAK8s09495oJAWdHVIGz4YgZ72YfVzC0vNs9HsianeDmi5/Xa4SkgU0HUtrJacpOToXRwmAwZC7Bn9FYrVrDi+JL0C4CQUwLpB5MpKhWM/toZZ3r8Ue+tQEm9TF0gkK7R1FvJuUI3AV15qbIxiJLJ0rEP1ey7Ac49G7b/AzvPF4uT4kvcDBXXP5UBPHHzfye5jn5g2ZbqhRv5/w9wMploPEpwspR1W9rlh0fqfZ9WncMJu5fFGhw8Emt9KnLrF9WxrT8IEFO/LVBZv4LVi6/AST44vP0D1x+j7ffiV56p9nul2SP+pBFdprwzmJctq52CGo7yMI+8F4IQcJ9L5s8z0X3Ql9mWpygObdgNuEFcg/3sDft51EiFsnw8/163j7XUoxL9vnkugpi7SN+/qir+PvGZ6jrrPEA0CiPC3E1uoM+3J9VU4VP4o8g5mQJ8V5pBxCDlRrbRVO3EXe2jfjpJmZkzm5q/XgcVKwSr/wmMSXLfSm1pStc+njn4tUPd50qoBC2689pc1O4kbs/AvDkZsPaRyRicETn1SxsPjvK7O462WrLa6NnQ1shU4PcChJ5THcDPh16479ZoFZjmqZzyn8WBKJ6MffgVwT5cYjCFoNzkccfKpV7OAKhfmpAlwJliYhdAasWlZiy59/YRsMizQgXa7z29pEyjbvCYEwwglzIGTye9DDkC+26sMEJ/4zPZbqsnJNhWI9Eji+PxMeal7YxCbEiLhRQHS2FZYgtnPQZw3PV+rTEgvn9nzSTngC6lhQIZf/NKz/NJ0ZAio+Tc67E0zWPncQNntKT2ekbMHm0/RvjsMhnoijLX1LWLctSgO6FsxzG2TOy6gyfkxm3TAc0hC/tUwK6oxuAnu7F+6OYS22PKvAhDBU5LW+Qq/qlZK/bd3QS5zwgUd1fN/P+Dhx3OTEy4z4a7qw32cq1DT/AwV0QQHG40kYEXGYUdveyZd8CnKjyvrmpyIXEQ2YZGVhVxDLj929laGWkcjEv6EZcmpGOFz/rdMxtQsm9yf6ObdvtOGd1yu4ENQMtSeuKUnBOOm5ShI4iqNva7T1sYP7nLs4fPze26XWbQc0viXGiYQWlPPJlKsUGyea/1X/NvAESVk44NaVxY7+eYoHtGJ0WrTlMgNaed29Ms2rJqGbG5BkfhNr+CvOUbApslkghYspklvrqO7jM/rKA1YcYy6doziBkSBuTdiY53nnGtzTJC/z9OYJH9heSmKbyVVvsjX13C0rUVvT1ANPpmH4JmCuvzSHU6dvaZIhBIbTF5RqlIYCWLV3cfA60qeyeGcfSJQUS6g7sR+RL81HjOy3WjcB9X4zj4p4+AX72opzvle86FB/MnQ7X76zZH3rUY+rSz0fQF08nzdNecY4mBuGmreHgS9xg4/JDGLsjSXtDgFzS2MOYV+rGFE2ByFeTdgtVoGFMWf+WLs9nvHbDmFEH99T1tXy4vjmfEw6qPtuXoRRK+ZW7GJELdymAqbP7wA3xERJ+ls9z+0IqXf45bVRYRPn1vYP9IDGrwDXtdYDv87nB0KnDjNr2PKcaPtX2jnyIpxpevKat76OIkOpE+bS2Y6wT6ryWdWSj7PUttPlpZBurUjgJ4C1o1WlGpW7a4yU8dChvj6aftVgtUsfKYWPFcZ+J8EKBBnfBA0XjMtMRnoejIb3u3Mfx4fSGKBLhUeDvkfZgyu0ZGkA3KQhO+ndweoCWUB5v/iDuVQIw8n1xm+ugeWOBhRyYq0BowrxOamwCwYmODDJUG10qRsOT74VQt3WsHG7g4dEd3/Qq4zxEOuFlmrGVmC6FD2S5qUOd/HE4Fm3o+Mrdm2JNeUdoeXWOI3khYTTARywIrrpPp/OX4ddenn/ATjasQOJkp5N2QiNzifnoyH2+C+3Jmick7DyHSuyGQyQ5zLBu2CM3x7vtzmLSAfLH9DxEQVh2iFyAhjy94uiw9NZUAEZ298duVDuDIqoDkMvPUld9wHocQGUCnHgu+UnFsDSkajcMpauZvbw2m2v9YhYhnla1AtDRbfNFl8LhYagpC2/z7Y5OKcc+RWWq9c3FmsAsle8MJECYoUNMvabte3n5qlIf1FoPamSfj5klwvGhVM5Lu3FROkcSKDOP6IYVw1e0yIetnTiv6/j+5b83bKkvB9h9s0J9AgFV/ewE/0PRWHzDO+dZq9TGrNGKV08HSXwkT8Uw1nEh5c82jw8rVPCNn8XQC8Gtlo1vI+r2TRTIVSd1aXde0+l5SC8Nc2SnEkcCu3D7J3iHjUFKbTAWMPLWhdv+MmxhFLa1Uc/IG2PmKjv1OKk3TVy6Ffc4hIrCzu/bc10W0WMvHP0wCZIziDV00buPmP5r9nPDzpc7MwFrXG3hNDTYEyvAGV6NWOiybjGJb3LSr0Gpq6b1zeUmEIQhidiphmOGmZDc+rGVqXCVBOT0b+MmN0TsKfUdyfsgBUvKNXzTBMxkXdjAA7exBQW73BeMPeUI5lSXXewMJGy8mlK584hWHu+T+2maMTa3ELE87oJo1gEgoFlzGw/oitvzcjirALWubH6d8jy0LTU3/9N3AI47ceaKX1KbL6xF/d5P1w+or5F+SfDGFKQIzV259xcpEVYkQllIyLzSHXA2JNO+uUx8PayD3NeyjljpoxZU34kYJzhadZKEzfjVHKPNt/u9qU0SOBvwH5FtLE8O4lkzArG4+LRWh3xwngF3FP/RrPJ8mHgZlhxy97w3mtTmf0RYoVCmHGYPffSxIvMOSHiLxfcXlj9LshQhwe3cBhtTQXT+hZD71HZrdpT9k5WkvuF8k83tjUge971E7ZVUZ/smAAV1Q7A44wMSzD9CUxPQk8hR6WlXfWC/AW2KHsetDmkrKqf0gVQnmQzffo2OR5hbcJWz5wOkWYeeMBlNbB/Emrf1F6Bw8C+LhB0/8fGEhkkb4B6IORNnNvR9OAK774ox43y+cUmG53YFGFEa8m3EeELmndQfpBvxBFP7AEbfpcWg/G8Mn8XN3/dUJXrMrnRpbjEfcZx1jdReNxkm0k1tK+mCsJtEIHpBpmibmiQ421uBxyACXMrlhAJM8bOum8bt6w0c8kMt1uz8wk2nxsdfVSHeMx4gpYRPgoRU53kJYcy3jIhJGqCNAEGWjvje8rpYKCdHbs2o6txs2xeFheWpdaLaQmlWk10fxuqjXSL/2QPz6JIUBYtkM2/XJLY76ZRDxiqI5gm16+7CHsKtFofjUcmDhjfJXQCWCi/GC3UbL+O9yZjAkS67WtLpq3KCZ2yv5myzuDYE33GZOXmJH/mPOdeQCkNqfOjYH/p/JQ5DarY3T/AU548M03HM3zmhqxIxUPP+4Bbu8htI5kIzQY786O0BoP3UGpoauf3wzraKNJLr3+ctSjCUgkV2dH0imikUNYaE/TmAm5anhv5rlmk1weETKzfkNF2dMpZ1Sfe2U1kFYwZy+v+Kxo25kWx54wRNbguTDam1JOsdZC/zPQz0niNdcfrltgbnyU9w6nGm0SODvFzRHefbv10VvkPsRx7x8oopNyMJUpfjJd3lzWiaOUh2JGRCuC/AqcUqZKx41fWt1DGml8nekz85Gc5dcWjmlpHOXViLGqHhcDt1oS3KU8aWLwbVQDMb+VNhVIxdYWnYw5+KX1kobA2/jRsPS4+btK4P3TAQ6suv9Cx+1NW3dvVbFkEu5VHSjcXpvZ+qcM5p+2ZVNohzgS+hRUfj/wXDqEHkz+opm+ADxxCx6FqXF7EdiFceS6Bher4D9q4KrEzDKuwa1QW/cdeJH9+rqZgco2jf6Db9GUuWl0zarrgg6eFfTQUhCKDWazK7v/pe9lBcONdonWIttI9FVOyoP2wPJ24Ik4ppws7FZBoaJKjBw25SLIdI3ID6ukVc69IWSyrWkstpxQObfavYRe83q0lViB8PgQSYkvBXkRIevzNrd9+81xhpF7SmqvFyv2TwCH+ONG0BUtKsQa9l7MhkpOy81jaVgRuSxKTGTce7BfmLH7+8gColSkQSzh6Ccd+lRLrFY35ZrvRpEAfIAyyS4rJT/hKPnyUgqdJ1VOAkbPFSFzKFpqEZOv2ZRmgbvT/kgL3XOCqAFeHK6s32sLdjLwyugjnd0iQDOQW8/PPNLMmm1+Xv3GsYNfyBvqPSP/cu0ZKBi1RIgrTtnRavxYtWntZ8uqghhHQv9ile6i6Jl0kUS8w+ASncay7VBZJkC3F9NKj85Ga2fps9jvJU8G23woh1eSOzpxqsfIJHiU4JI4B6NcRJYXAB83V797MJDeHUDb5ohwGYyiPQ1hxenzUdRtbf45OnngzyOjfuXhT2Mj+jGvVxuCeIVEl/RIFj5tMzD55+hxwPip5VSXuIhJr4hki3tRDY5MmteI9wIylLQ81Z42QcUZvc5/i1Epc2+iSBQWxmFGUESMoNT1z6l9OQCy0lMeZuvpdVBPGi0rEr4rNJ7JgDzjI4OjarkJ9sNjFdpZR6bc3/MtFLh43LXEEdN7BNOO26nlYrD9NF+XsNYvgrx5+QZwZnQSBA6D7V2mMmIk/sAJWpQRnSVroMwNUxQoCKwHUdKzUII2lt/chcqBUpuwMrtzVOiscksTnqznvpQDAcXizDpRbJSNTHXZl/ix2vfgdSPQx4oMEXtwxPjD1i1ucICHvP8KocabXAySa5/meAI6VB8/WzalpR02F3euopO0VN4bAJJ8KB60zWIs7kSZ9UgS1ZputHJHBCcRN58/a9fhJQKICT5+v0BgYuCuvMvMtwBryYvrlkoZW80jbAE3jQokBVKd/VIsf4T/i24aI598Sd0BiUIrZP86FXyzPANVp56BwBU54epE0k6hdhy46dJ4oc2WVzU5H1EY7ZJlsmPJr/S1poe42E3dVislfd6TgQzEvaAF+cm3jd9Ul9CYu5n3on0zJyABLxjspcAs+BrbOwB6s5GhQ07pz1sE3dYXb58LFWKdYUubaxdvCRbROK7YGInFIaEy01quzNS0F7ZIKhpRhTupOkW4wdgJWD8v71OfUuI3QwEufADd4jALhwyIoVVTI8ZnIpP48gdcjyySF2SNbYymHRmJUUWbaU29YwP+rzGJxTV2DYgJ2v0JnckPYuCekMFrU0bUyliakQU6YS4HAjaMKKW5yzyrsooJBRxeWxPAZV6riUOg3gpZVfTfIplhRr7gfpREUJ6VFeDwLJKcFdf3lJg3b0YYmGta/XfuYFaHemEBhX0r6WwhUmv6q0kbfmYNBXb/cOe/D8si1jm4PpBB8k3k/Qp/tnOe9ngCLV2i/Gek5F1kJxgtbanJ4phgLZMtvAxcKXeSVuc6AJF1i3i7g4iD4x+tAu/WAZ7jhfq3rlGDnc5hkJWIkNFc3DRrE6I1/PbjXUghgcY/trT9NkLJezB63g8gpbum86hb0glHYctYjjX3O2tEiJ0HtfBBzitc32i12wKY7FHRHFhgSKdUVxVvXWse3SAJ/qcCCWwo/1wWOuBr2jo3Pae0Dk1U8yGZiye+I7WYsiqyoLm1a3xU8UaBX9IZ/R5YgUnoPWhG10IySJxWfbTO5i/V/3CEQm+GnlMudwWdOoxci0oDl6BFBSqm5asBXdHWoPTXPGrUgWyoaEARfMOm+wr4CSbAkXEGKeU3OC+nLQybNMz6fDK5gmhQ+lySUAgCunN7zBZtsJOMEBHP04QiYLpny5+/02VVQSbwyO5YQ0EvuNJrwCILG6/T9X+66EVqsHr1C2WJ0PP3i6zIQL9r44434cMDQJM+zuDoxJD2t5z24v494vzbLsEnaiDE5admCIZzyeLsez5OWk9kjwGahSOMEDXi4b30pDpjHsETku1l5f7yHmjwlMSQ+OYzPxFIHfx/ed3VKDYBjyex1PZ6cTKDPUD7zFbilW9NGQLPv3ITJe1NK6gBT7h4HOSKu33P5o42PoQrWy7+IuOCrQHexIHyKudwrkH4HCQIdJpqje8VIdSMeJYFbz7ToQSzqJFrPgHst9mpM4U/Dn+lz2rWpUv6ruamk+tfJMkEQQ55Nvt5mgiCAiE37iAYy3572ee1VsYNicnNpsbTqYQicMsHQngKbhvq9j/pbVt4z4ORKLx/rwJhf6m6KN+ZQrVTft/hvlwWpzI7OfCCHVyTbP8NdvRUhJvO2z5bMAFzubOGY8n00kRbFPTVm/RGKxPAXSXk3Pym4ZHXC+Vzv/OAAYhl73rEj0IC540UIYkPN+RC4dkrxvZ80R26RFV1U7uvXYtYubAH3iMLD3UZco36PTvgQIMeOGUQa/VOhMAWdsB/T2s67UV89Qppfbz3cwNsraEmZRa9UCslDVBcieizCAfxjQBb3TltPIliPer0SBo9INwiA+mzbu4DOJPWlAybGg1/Hm/4i9JLzO7ef2DGqWUdFctvjoAT6j3snwmNqsPx2ndWjClPbDnAOdcqkJjTI33hNcwYf2vFOwvSLxPpxo0idUYkMO+FM0HLgL2bbV9IVxlP3aIyQz6WayoRTpid6dxfdRnAmkFbmsPJ7+TXd+WVUVkFcrRZfEGxIHNzBF0i6guhIpSrP777eyNEgBoiARyNq7VzsxS/ZT6Z4/EybmaMMgg8D2OGT//E+KVx5GMe1i2AHqZwcQ5bZhG6mXDBBoqkBtNkosIaX3Rv6Hl8WvyiduxxmoaeNG8aA+DmAFRoJ3WieLSwN2PbBoGQqEiGpgRU5hb+3gjnAZqxKWNI89FtMhwJl14DOwTK49y9JkPQ+zcwzBo4U0HEy5MBnDAhfYr/M7ZDUYS2kPlUPhkTri8LQOX8+4Ktma5F3sDRhohJnsbzpyzCHEUZFg1/z+LuQ+PFQxWYe3t97lj9Mz0UydizC/hLG0xELPcDmvU4XxabrbVDj7iykSIYd1n81yAqjLrICGq0WZpkHZL8Hl4ThmcZexKJ+QgvfJb9i5SmSxnESgXhOgfiJfDGPIpRybF012+kq6Q88n5/tDo0q3dBYkgeo3TQirIlcx4+aRnCMWOHw/ePJwiWz70bOi0DbSoYsxpSk+hfsxkHDIsxL1kiziy44WPiud7toiMQ2vGRgfFemxApSj9O25Pke6jdtFrFMZalgzaTupwSp4eriIUaig254mm5thCOT7l7lREMzFMguRTxOzVDZ+S9L/4wceO2TwkryZhmLE9vt4DMG+QCKpiY3+BsEc57xYsb85b7u9z6oRm+K/XKYfdUxhFdQULnuTDuYgrfLr5v1kFSHy+QwZws06S+LReh8G8eb4hs9r2dBHaVLFIPHyYgiVAC7/oF0SOc40lTGgvUd5vaer10zjzH7m0VHmyIggSbAMKVwAlpYzGnxsRpv1IbqgGb6zHDHt4k2qepTGmBzoRlHJl9+dGDfq9shIA9Xxq7jth7/4Ry6XUAbJaWDbqWC1zj3/fROhP8Jgk+xo7Ma7CZDhzkAv1kqTNx6Po1fYec5k9o+1SUgj8OkSh/kZZTEA9gy4nW1Oe6nQ1menurKj2SW7RD0WftbxIfq4/lbH9OgGQ0cc19nw7P3rqjhqTvs+zfQUX460EZ6iBUzqUldcjoO0cEla3kH2s0OkG72KllFN+AX8iFVDTbFSGzKw0ZaTwDN8jgRBrIb4PuGw8icNB8+AcJfGy849vSG0L3bY9qJX1/NdaZ7kLT8NwVPGJHppXurs+auBO2k+jC6Ayt74emHYX/U5j1kZOkeV10uU1BRounUoBlJkbm4BuaWansD8rCFZX+AY+nayb7wyKfzd75TgMV2agtsmaLoBP7eTqICFn7f9RrexH1nVYOSdX9IdA06RUjT+I+oMLBcx0exU4oW6TK7mT3pCDy+j00asZYpt/YBoPt3FIJ4oWr69ps598zofK++X36+CxF7UsBA7E6+sXtqcCmxXAzsrusbrQvHB4M+Gz2jjM4Nr818HvM8U34EgS9iCgxZU0pIBKJvqSZ5goasOojthVuAzvyCr/GO5vi7VY0vDVX3bzs+84jQCokShpzvObkKex+WDfnUhyT/xqBoSMDIRqYg0ebDE0M4Es07/T9gYfcUw1LXJZR/Gj/v1c9uiMW04krEISjCKVgaZq1BUlJx042QwXmTAkqecCn+CFvArydzh+C/CPW9dNBixEhnwNEh3qyrRygXKeIsBbqrybTxcLV7pC0hk+c5F4llooXZIfbs2+jx+uUz8mEbwXT7RsTI9CdTksJ7iasKZstHDiVioYqaolUPEKeNiKfXeuOHkUS6xEtpGA+glZUUDHxc5jBz3ySwKcOth2lb+p1djbgKxcQIAmhLaSbgvuugzsy5GZrF4i1FoMgaPHr2yizfZNvX0/6sYWiVSsVAkinR5mBWtE3KvVSjGJHBPujy9x5/C1oyoGSyEodBmT2COicuTupLKRDCLNB/uWvMaoP8wnbkynVBd2VFlS7D4uVoxQbOHG2V18V7qqRgcuoMVIgaFyYprmtU8Z1LCRIoGdpaA3pce+4hcdiHnWrjiwU9L3fylwEr5xWMr6+7LEZx7aCL84OZzveRasvJ9SrWYQc4Sez1uZt0WLK2MxlE+G7KBLxTkc1PCULdF1GxrfUB5Op3Rq2cTrIODfywu2awd9hjkCcYmk0P6NkjR7A3g/oIUgVkWSvRvMNlnpzq0uvIIs1wG+B9T+NQvll5s4ucCL39QXNOT7l8eNCRYVWobssjVrRyj1p19RYH3S719o4zaAnM8S6vSj0U6VMj/Bp9IcR/g3x7WdkNLP528f5doqRWCgdpIs9kcw9dTll55NIp5/n+rUE6RZW7UR7E0jrCaef3fElDsy30NFaU77aPAQzvTdya66sbt8VG/SS+2feNxj6FXTYAwal5sx9FaN+rXG+sOdfDmrJqsepR6LmyzppRDSZ35KZ8giQ2UtTZukIopFHG5G+A1s0HbMGwQ/PsEZUXI3bNUmZ/cnpxzGWvoxa4s2mHGpKWDUrkA7vWhAcTq23Px/wXwnXUH4DtDG2Ejwq9wQX07gC7WBymgMJqwWyjH0ARyG5jxcP2Rf1LGiTHKXTVnAXjb0ylVk0b9kq5xULN2rpd6GKAAvNakdoKR+qd6cCN1UiihA4C7WjxFelJldnpt9ZdI1Z0NElbqoz+2IvmFSq1qgCm6TOtX9FcQmZVTRpuRPpBvvAuiNlFsB2sZSvCLN06BOonzvC5XeSVwNO2wEb0qAq1RgIZIRRPvdrzglBo8oc/TnJF3sk09Yv6LDl8V1Ds4twSWGqqnfrLoY+j3osJic2FncH0BeeyuFoOJ1EqZPNupmjrxYRNT+EzL1tSVU6nLIDYdVcRmQlNBlld9rCNorVkzJFeFFfO/H9rtxfD7hzsrcErGhN7l2Hcvdi1pFnpqCA/9nldcg50eRWx2q2I2xp96lX4nDc/L2ZnjZl/+XEumwdEUZxvIkru8jhHpcFRuwiyB39vIi9dfWbCEJ057YuXKzVh+RvUB9c2fOoGithVVq8UgNtF9mGE5k8pNhY8UjKyx3OkW0v/FQRH9kjOtoFYNjASTgyvhygU5UW6ha76rjl5bdqW+1IjyFlfvOhqydsW+0iROzmvd5tJnxJ5UpsjO9OmoX1Q1xLg5IO7h/5HHjFJ3zFJCpXXZxJE/bCjiutxtSdfWzfTrhMsu3MiiqQU0tWDQCqSREm/8EL1owswY4xDD/+uFXiiLI4Wp3cI/QTzhxnKDyZY7l3XyMSGJ1ia8xrFSlJOWcwdvamrnp1F5ifq07OlLmMbsdVbmXG3gK1PYH+5FecMk8tx/+L2xFVfp/wvJAKfwyvaSNbIpt8Nl6tLKwNCmacX+4x0yxZAV4a6Iz7P3ZThV24RWLniP7I3ho1JnN3cBG1H2aUOazAtfgzIt2ahx/rC/d2n6iycZkevDPvetnleu7ts0u5DSD9mGri+zGZmPhLVjPDcdPCHOuffpn2O9ebFiiEdhYFfBiCH6NQW06fdncwd8GSYcO0YxVCKLzzenZQuCD31/jqHKOA58fDaVlCGYVj+8v5sWnwFHqGNxOUx/GF1ArebGLtNVelWzN2FXrbzHz8Fp6Gr6LSwAdFwU4dQDEjFXLZNb74MyqYcKy8RKia2/Ct8tFfs/ZUpuwTRUGYmrtMSn9tpFAy9zWoNAHhv+FX8NI4lNFAvawRPkQt4szIAHFCOpvgQkhOzg+AFDC/zUOXptRQTy3AHEhJpV7N8MXsqyBs00bzKfOgT0jkuO2bXbz/kvNs3NA5WUUmZlX7HnhvDZjNEYLsa+kQYa58j9NrAyFRoBOGLkht5hMFswDT2LaIGdNWMFw4+EeFMpTVC3R5kJBGc3sHuw1y7UjH6TRbFhnx8v6LFYxXzOZ25lQPDLJKEHtyUxC7b1YGZMMr4DRiU6wZYZ4s+lZD5Rft11DDytIPlX+0hxjUcIExQZDXXceSyByjhIQAsX6ltUshR1OsFQcoaI8jQwwehn+q0rRVDSBDXxcnW+ImtSAVK1FCZnLe7NZXYxeTm95Nf8qnGuqpg+2SGqIRBFHHTLLW2tIZ1tkdoz7NpboFTbg4pkuqZXiRXVZSpu1ngoCyMZL/N+J6c3tke6X4WMfNxgBWswn3NnrMZVUzwVOwHQYsxwb6cuBZYd+kVHDN6dw0UuFEjiUivcgXPs4M7sOv+hdQ88x9Q1PeXj3mRB27YjP5eS47yowlrhkqdb3THEzfmo6+z6J/g4QHmGcEaSyT3gpqMgH152Jl9Au5fxV0PxjiwO9iasP2L5S2HdNtg0SaM88eH9Aj7XcOl4lGGwnI8DfkrlPu3fG1KWS0YAhbBpFDny7/Lmp36sqP/KSmco4fMYNvBt2/G9J+EsCSOoDhvXZy+VL2YzhDIfagSHy8TyZJdD3umg/E7KAzcWzWb8m13+hFRugf/GIDfilR0BM/U9r4UHcGBeQq5CKGneLUFEwiX57HS1abb7wnkIb7Uot6wPGJaYi6q3JYtUV/ggyQQiROz0o3uIRlW2N20q9RwBUVjvJEtDxZiTHQCy/tOrDgHrwXr9NGNHtR+GSwF8S3UmSZQYROdSxLLXLfkD/t2qPydTLjMtb0D34WI5/k5KuE0Kh6xm6spGqE2M10cbLIvFtQTDIMSm4VJdBbCCygrSLSY2yjbWi4GnksZ4vS7zmO/qvIBsqnL7aU2Uw92x7sDQuH/mI8Bo5yTDcvXB1/wr2d2O5Zu0LJJACUrhjcKtfxjGt8+QwWgMarCJq66cFH7Vi4EnLnbHnLwuhEcjN9pYGt4LWH9KRcA/pbki880lxH4Lyl/U5O9zuoWBHhscpdVN+XcGuOVga6qkfaYTeb3PYs91leh7S31iBZAqSZ0VxTrPH3HUE2w963sZ39AwjNhRJfj104zfQMfkBGWCAIwIgoaTHSX+bG0R1X5cL/7SvXaAuXxUY3ymGjvySl9nu4gz4hxe+QMhM1X45H8UF4uGFQ38rTGndAFY8fdPw0pyuTGzexUimlk7xzkwhWHcFbWmsH8zrAzrhhiyW+4hmZrSI1NjdtQNKxRtBMkrEDTufuKsnhB46q9knpOGxkgPkePP+tlc8yarL3ZOgAqbMdZzM0leirTYg1IsSmjqWFlR6ex/CDQHpeO+RlP1muWTMX4uDf5iX1OSQJw3vbGPscy1Z3YvargZNfp5nwR6ZOCXqRKgWiMnh2Xd51zvOjT6IQ7twJsikRyOqU7qk1csV+7yyx/I0KYlux7FG+UKFslb6TMKDc+Y/Rp3qtBYnyswxswy43crmFvV8BuqARQjhRnxI2mNBlbKB97SuackyJo7OC4dzCObXXLJnaOWCaIMYE9/wEIem8/8JE/MtzCr8OqRV6CWTrFB0UNTiKyZJcAw8RCQCx7BgdAf0V2nUjK8VVl/nYx7/RBGQ9/dhZbNbxbN4GQpEsLJ7FOKMwxKUb86NEuHZ67awlYXqYpTMIhKAVw7NmL4L8em4ZesB4VF4Cbm68vlV79OcohNdGbvp+4TVOGwuOTlv2vcNcQSJlCdps86q6IwlgtcrZrQNqlL5SwOuK3euUOpQ/NhWQviVNsSGBqGT7brRYT/EVeY9CHHai/WX+1Z4yBH4qFK1i9ev29QD6GLalKBEFLGyh2HfJYt4bDmilR4r815vcFZxMWlWGGbDJ46bd9Bgrupnp2rBDeJJJ3hTEB+ZRjV25NqDOyueKXJnyHR1NRMLNl5WGeS6u256x1+sG2/XU8++Au3OSrAgcLZYMmdaBd/UqWotTBX2T9078tN917WkluBl0dVmh/d5QlaXKo9IMtr3WgFVZu5OtLUeJAGitHtvANVmvZwvVrDrf+0s3SoPSs3vu5FaraZj6y4UhxFcz3XhO93cNszZAChRbBYDuDTVHMTXCmFQM41PW1quZpVp5bcFZ57FdQPvz+J3ciFnGTVp67Fl2fsaYHJWOqRqjFpdQqwr81a1XvASRS21drGBZeFg3zaItwB09F897gZe0MRDZ8gUcaNBjlP5dSzysq7FJAr9L92TdgKXOadBo12zQIZDEljvIXrqVBprZTASEwTHLi6+OFHszWI3m2zIlmXgPrKKkUOIOBrP7Ne640FbnKJwaKZ1K9DxcSSOuJgHVrFbMPek/NO4HC/6Hson6Zg4dOu8a1gfT4lKjqPkx4V8Cl2KfUTx93zpJo82AoqDqtf1nX+msbOLotlWIpb+IUDSg7jswTvZb402u6ldooYdptgisD2LMOxaf/A6bW4xeGPaJwx94TzMGcOYweGBP/NJDPoUN1hLkCUVMAwG49SdIgIjUNx0jwqi1D8TnEnyonm1Yuv7y3rfDc+Q9flT51GRGjTKHO+aHls+pocuXoqjEFMp6M79Pm5qXbKtUcLNPfBXUDyLku79h/6skXmP5UP8lrCpDdC5R//woJ5JvVHNAohoQHilaFSQ+9sCeQSaH9DZ0RcHSCmEGC+mApmXfbJC6GPKDXXb7R4do74HR+gZM4aYjTCCFA6vQwPfZmhNhsSbiherhWiMBq1emGVAWiQozwkwJ4k2vesB/GeK/2UFveRqt355nDibvErVO1DVrHs5H5gHFHA4EyUfeLGSfdoVB2Shd6dZdOa4TakPaG76j3P5185jVJ/FrLGeOwEU5Z5H+w2XQaVj9GnTPIK3JV+NhDtGuBaJfEnE/q7Ql5YCYb3dgj/XfU97KvZ8/QoGNd2CQaLgrZiALh8jV2bbfE1/QV8MR6U/WQBAwP9ek/yQscbmBm3qg20Vw2GdXXeIeC9iwoz1puzrj60KTGoqh1/HWVgfUqDGGLFgRpTATVN9v8mfmZDtodDYQ/S7n/u5lBpi4tILuhZ8l5xaxAmmOKuseJPmi9DOZCRiyp9qm+z5g/HmrqCxA62C6GqqgLoae8EvZFgpebmI1LWfgTAXatPIe9sM1TZM4JEdO2r6QyTzPpjX/9SZOquv7kLTChHPJrNt8Q+qaS+UBpGitwvyEw8DZ2VmFx9bbZL8n6ZPm8aMpoGZem+AyPGRwmv743raCTqLY4lgInW8gQ8hgZ7F3YBdkm16gXSpV0ewiaB2K+OUFkhgoS8z9o/9wypjES8tvAZCU3bYqSm4RSX6W4nq0rhcPHKcnS2d0PqoH/hE7KPK/MFuE9DCckKKqhCe/aoK0uAEnxcoRhT7p1P2pm+kNsh6IU2ncI6kmEiJZvFsqTSbRdbNK21id0KnyYJqbtiNvuDgRcZ6tGny/TomNYo0TFLduE4WIq4V+T9C3z4xADnKrHPGg9Ykt0X4Qif7dN++UK5ZcSmvItwRFhFIfYeSRjg9rcuqLz0rJnUS39XLjS4lsrz+HqooHlU1+wosrSzUzasDs1I/lwlsKZ/VOPkZxuwHJY6fCVo9UNx0dSkzO67EPeED/ISuKJ5bk72DqVYLcMm1CR2tgX2GQNqFqqhUjctHsEqWbMMvm9aR/mTP7n5fO5abKDwe6xkbWp93s7GOmkw/7O7OegsAhC6vYoQa/jESjwsMUlw9jlM6zsnLtdOapckeGjwqBFNXWC5jC4ynifKbK/uX5sJmN+LG8ytvF2pCllmFcB9jIYwVCVkYxo/O6k1gVzc1MAuQcaKc8DzuGKyrZzU2Dogv0HuFjyMojcuneq7O4/Rm6r3opeCJgzBJcFZIMTLvb2WL6cjU1In6zfDiDor+MHeIrmecSuz1H7HTgHCp/hgVYk4dmGiShEkeCHtNmyjRzxpHz/uuYdwASdwPwI4hANonAJgXyfXwa39tymEvr60cqk+4gl4gtENt2d6HSa3dn8ib7KQEUUIfzD/4MzXIDOvVf/Znaxa1B/juxmOaxRXxSyWFwcR71TIErJLconDmcY0optFe65JIaEB2SUEyxJTjPfr2tA3eExbwxqBwRmZYUBwrlbw5B3aHyntkegsmPvMZRaw2vixg9nP2VSBjIGnh3VyRiaEOFBGnQSLoiAXf9DilSudh/nT9ua7BTQYKyh7+BSRfY5Dip9KLQqzZbWfP+V+7AnzvIOJYxQzYVausbc+2b3/c2tQDZitTFT/B1v4mSeu/MuGckv9nzmE/8IdtIKop62OQrSjK/ZHKsJqwSum+BzKY8/9fWkgoeeTbe2IO77qyYfP3tQB5xfcIDZWftB+MDirAac0LjCSBDZunkGxzrKXqXRUSfbfWcSIBESX9+pLEi43Eif1DrjbUXc4Kx7YUAO/rAG6Nb7jC5BFNarr8ygYqexlXasEW25M5h+5ChJWo4GTk7dHAJ6l11AeykkplGxw3x6u3ucAiYzmlb84r8HP8yF0o/GtqaQhKAMpZrVq3Y1FQOJu4PXi2be0XTIOkhbz+3JLdJFuZ/bPbE5rImwF1Z9X8mfQoRru+gTGzPB52dEPaFroOK3B+fp1wKR/RwACycHFHHH4XAbLOPj0koHP1lqub9Qj60uj5CzL5SjI2FmpC+n3KlwWO+SF8FMNVWgVYW06d2K25RTRazaOIJLHvFjHeQ4uaaD8k35IBIOoYX3abociUALIXBmf/d/LbhWfuEf3qQrlYDD43N0n0XkgR6MqYIRkditCE+YWiflkBYvnj/kKmtsxlUzm2saQbXlK8xTejxUIVcbGbYzJChHS31o3mNEmmYJTiG8udOYNzL38AWupgwKfqJc+BsRqYQDUIEHwuB6NE2a8ga9+6r5F4DHbUsah0AnavIxBx5/XpMm1AFk23ls6ujpiSdCz8q7aZ9p7q+SltjHwNoR3K0jJSMfhUrlet4HL1uk0OgAjN/P4GNxelU88pBDd+r35qcapkcmvm2ijMPHvtuKCLFEya4AzE6FIiI3RCTIy6n9BkG4tjlUQ7zk1KMwi7RF62jNcaGoT+JHvUu0EsqfMwlvoJ8ktNKDgv3FDHXDaneHt3j3ODGkpOx5TDx4aDB4wCiyAFqGfDmC8Hep+wiLruDHRz8DFnAft9oF3eNv1ChSPJdJKUwvSpgGbsD98y4/n6sTkib8Z43mjRQDJB+y5zBbBjPbaBOlSOEFy/QBzMcH9YsK9b+TbNoq2JpiMxcaDuRvwpxw+9mO7uzTVv1SYtNt1YXPQ2WTt3/QPfuQBf7CVNt0ljts50A3LcafuPGdxov4gavXXB8U5UTbjqKbtOkOVbl3vW4JrjOHU1FgHE9PQkXwrTl0NF1qYJZQ3/233hh8MjwSOFqRjPBRY1dEHK1DbWbnlENdhK0h0R4NSONhqafodjRnIlQxaqA/TIzdRoItbOK2D1H8fK4if8immwEp8JTVNK7hY0bfjtORQj2KHG5Q8s9tjoXtCyhZOSbWzWeZmETTIrbwvP2Jz6VBUngshXfb5glK3kkQdl9TYtpxCC9/+jx+4g0vsvFf760Zw1d0joddsEvmMNtRzKzN4wVWOYt16koqTAR+jRXWacpNTtINGHQ1Egshr9+TYHk8p/Ya+LpHNe3ztRIJI1F10pOlELvfUdqazd+5NdGgKXh6GrXuAo+dIXRLZvWSdcHcFBz0LSqjtUPm9Lr9ot54xYA1hk5o92FTdIfGE8bXCfhL60Y069vrtJCCGt6h3ZA/jAmXAPExUzpNE0/fWXGLSV2nTiZYf4HTz+r2fCPKJGEyQCFLEQD6jMkUQVvJfWDnExtB/pTDLZTjvwQFN8Jcg25WK0AwbVVvXWn10T8bypZI7mCUOtVfelmqRV3RvzWVYXKxmmlrNOSQ7XxhNHeJMQqlhScidha3pYV1Ku2d0K60rtBqzEJj4IUKpAWrRtulBtdkvtyuS9/ZHnv2MuvNIXvxfL4WMVuo2CnoVdVPhHqnhMyuUmFHacH/Qb11TLj8H/4K47D9ywj4glHZoKVpbX1xojM3D7LFhrAv0S65yQQVorVFcCBqx9s2gppgqHPL4utolO0+nmyp7CEjLXaERyz76per0dR6uatV9LLsx2k2JIrK2k5v3lk0faP9eCcQc+pOJLrJRTxZqEcp1a2s+2pmMzvtEZ1OqETSyIF8bmcD95N4JZUDMHdNCXS+G2VhC0+jxUpbYrA6YzJRuEQCK+98IeXMwEH2ngWcemiham9fl8kLDX4sK4oF0lOAZAyYnL5CZp+kwceeTQtF8iRDn3/4262cIqO/WkiYZzOJAxNSuVJzGhGgFjnZ0SBiFYDyYL4dO2/QAayNcva2dC3N5oAM/ljndMWQmkHrx1y1dhWprRCg1bCUopTspOJjJgHXy+xQkEpCwgvdYn0ZVYohXw4vl0pJw277QY5FeJuP4X3B9qXXpOhks8bplSERjjmIuTC+dCfK63fQpSNFWH7FzHvrLlbE+27PIHjCAlsczXazASb68yC5CWpqJ4h/pp3/c7XsFbS2Ec56Nq1eLAH8ecF/5M45dyoAHP1Iu4ZLeyK68rO8eIBKdBwVZTQ/PtuMBXp4KK3G1VlFgZj0zeTdmyNH2Pd5T7rgSOHkrp45B/8Ekm4/DklpGFTIyh41kfuTfc3K69bhXWZS47MDBkWO8dNz80WJ/IVJzlOmZA08OHj2lQFE+ESx0GfmSmrNcNZ8x/ZM3mN0C80zCeEd3n+1SFfN8DeziAG/5qZnFTyELSvlMjEwyftd5JRiXKm5vLnHAaAF++IRwJ2FogzwN0vJgwlc056DwvlxVsfROKjBOczkBp9cl+pfK4RjIt7TFlbqNrgcAB/r2+kJGn2KJjtzP7qbXR3/nucWscX4g58rfF78N/qXAkwIOBBbOozRvyzQQiJV2IiE26/7+AGp1PYbUYDp0Pl/oz1QpBRyB8w4YW1Pfi5/bBC5stC7AtogWhgPC9VK6mqrbYrygToTRjGof/DYhhnkhvUt3Jqfs23oR8poo7eRxd1PRTkjV4X7I9j8E52lniBsqH30z+0UxRUf781hJDy4akSjt1c0HDyBP62ggv/Gh6H6UY9W6qM6U58nqLKgMn9zEXmZLpOveQKJNOEieq6cn6PXwaV9dBfvpyAp3QKdRZikjq0P6i019wRmp4D29OcxPKGY/T3+0Q3Hoy1nFmsHrcR3XOX7dDZRu+d+Naf5+TY4Dvxy0CNistW2Cu3VNHEY7rCvwWZi6D18K6jDo5+vW3hruRQBcgUEM2NGvo2bm1nm8v3MBGK/MWhej/6Nk8ZiswWBuntO/WGUuqrq2+axkzHASsTF0hqCUxbXTnsxLvfg8HCkhgEYDolTQS514TbeCCXXjGFbxzOzkDbGLKMtykQI3wnnYqQWi05px4vBeijb5dcZ/m6EXOoI1c9JOnMSCsuxAqh6c+5OB4BlepyGjUHOQoQdQaoi3ndh3SzFg=="; + try { + String t = AESUtil.aesEncrypt(string, ""); + System.out.println(t); + String json = AESUtil.aesDecrypt(jsonValue, ""); + System.out.println(json); + + System.out.println(AESUtil.stringToMD5("")); + + } catch (Exception e) { + throw new RuntimeException(e); + } + + } + +} diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/PageDimensionExample.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/PageDimensionExample.java index 55ed468a..6c63d97b 100644 --- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/PageDimensionExample.java +++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/PageDimensionExample.java @@ -20,7 +20,7 @@ package org.apache.rocketmq.streams.examples.aggregate; import com.alibaba.fastjson.JSONObject; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.source.DataStreamSource; import org.apache.rocketmq.streams.client.strategy.WindowStrategy; import org.apache.rocketmq.streams.client.transform.window.Time; @@ -45,35 +45,35 @@ public static void main(String[] args) { } System.out.println("begin streams code."); - DataStreamSource source = StreamBuilder.dataStream("pageClickNS", "pageClickPL"); + DataStreamSource source = StreamExecutionEnvironment.getExecutionEnvironment().create("pageClickNS", "pageClickPL"); source.fromRocketmq(topic, "pageClickGroup", false, namesrv) - .map(message -> JSONObject.parseObject((String) message)) - .window(TumblingWindow.of(Time.minutes(1))) - .groupBy("url") - .setTimeField("eventTime") - .count("total") - .waterMark(1) - .setLocalStorageOnly(true) - .toDataSteam() - .toFile("/home/result.txt") - .with(WindowStrategy.highPerformance()) - .start(); + .map(message -> JSONObject.parseObject((String) message)) + .window(TumblingWindow.of(Time.minutes(1))) + .groupBy("url") + .setTimeField("eventTime") + .count("total") + .waterMark(1) + .setLocalStorageOnly(true) + .toDataSteam() + .toFile("/home/result.txt") + .with(WindowStrategy.highPerformance()) + .start(); } @Test public void findMax() { - DataStreamSource source = StreamBuilder.dataStream("ns-1", "pl-1"); + DataStreamSource source = StreamExecutionEnvironment.getExecutionEnvironment().create("ns-1", "pl-1"); source.fromFile("/home/result.txt", false) - .map(message -> JSONObject.parseObject((String) message)) - .window(TumblingWindow.of(Time.seconds(5))) - .groupBy("start_time","end_time") - .max("total") - .waterMark(1) - .setLocalStorageOnly(true) - .toDataSteam() - .toPrint(1) - .start(); + .map(message -> JSONObject.parseObject((String) message)) + .window(TumblingWindow.of(Time.seconds(5))) + .groupBy("start_time", "end_time") + .max("total") + .waterMark(1) + .setLocalStorageOnly(true) + .toDataSteam() + .toPrint(1) + .start(); } diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/RocketMQWindowExample.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/RocketMQWindowExample.java index e329dffc..3135065e 100644 --- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/RocketMQWindowExample.java +++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/RocketMQWindowExample.java @@ -18,7 +18,7 @@ package org.apache.rocketmq.streams.examples.aggregate; import com.alibaba.fastjson.JSONObject; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.source.DataStreamSource; import org.apache.rocketmq.streams.client.strategy.WindowStrategy; import org.apache.rocketmq.streams.client.transform.window.Time; @@ -35,7 +35,7 @@ public class RocketMQWindowExample { * 2、rocketmq allow create topic automatically. */ public static void main(String[] args) { - ProducerFromFile.produce("data.txt",NAMESRV_ADDRESS, RMQ_TOPIC); + ProducerFromFile.produce("data.txt", NAMESRV_ADDRESS, RMQ_TOPIC); try { Thread.sleep(1000 * 3); @@ -43,34 +43,34 @@ public static void main(String[] args) { } System.out.println("begin streams code."); - DataStreamSource source = StreamBuilder.dataStream("namespace", "pipeline"); + DataStreamSource source = StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "pipeline"); source.fromRocketmq( RMQ_TOPIC, RMQ_CONSUMER_GROUP_NAME, false, NAMESRV_ADDRESS) - .filter((message) -> { - try { - JSONObject.parseObject((String) message); - } catch (Throwable t) { - // if can not convert to json, discard it.because all operator are base on json. - return false; - } - return true; - }) - //must convert message to json. - .map(message -> JSONObject.parseObject((String) message)) - .window(TumblingWindow.of(Time.seconds(10))) - .groupBy("ProjectName","LogStore") - .sum("OutFlow", "OutFlow") - .sum("InFlow", "InFlow") - .count("total") - .waterMark(5) - .setLocalStorageOnly(true) - .toDataSteam() - .toPrint(1) - .with(WindowStrategy.highPerformance()) - .start(); + .filter((message) -> { + try { + JSONObject.parseObject((String) message); + } catch (Throwable t) { + // if can not convert to json, discard it.because all operator are base on json. + return false; + } + return true; + }) + //must convert message to json. + .map(message -> JSONObject.parseObject((String) message)) + .window(TumblingWindow.of(Time.seconds(10))) + .groupBy("ProjectName", "LogStore") + .sum("OutFlow", "OutFlow") + .sum("InFlow", "InFlow") + .count("total") + .waterMark(5) + .setLocalStorageOnly(true) + .toDataSteam() + .toPrint(1) + .with(WindowStrategy.highPerformance()) + .start(); } diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/UsersDimensionExample.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/UsersDimensionExample.java index a6c163df..69156178 100644 --- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/UsersDimensionExample.java +++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/UsersDimensionExample.java @@ -20,7 +20,7 @@ package org.apache.rocketmq.streams.examples.aggregate; import com.alibaba.fastjson.JSONObject; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.source.DataStreamSource; import org.apache.rocketmq.streams.client.strategy.WindowStrategy; import org.apache.rocketmq.streams.client.transform.window.Time; @@ -32,10 +32,11 @@ public class UsersDimensionExample { /** * Count the number of times a user clicks on a webpage within 5s + * * @param args */ public static void main(String[] args) { - ProducerFromFile.produce("pageClickData.txt",namesrv, topic); + ProducerFromFile.produce("pageClickData.txt", namesrv, topic); try { Thread.sleep(1000 * 3); @@ -43,23 +44,20 @@ public static void main(String[] args) { } System.out.println("begin streams code."); - - DataStreamSource source = StreamBuilder.dataStream("pageClickNS", "pageClickPL"); + DataStreamSource source = StreamExecutionEnvironment.getExecutionEnvironment().create("pageClickNS", "pageClickPL"); source.fromRocketmq(topic, "pageClickGroup", false, namesrv) - .map(message -> JSONObject.parseObject((String) message)) - .window(TumblingWindow.of(Time.minutes(1))) - .groupBy("userId") - .setTimeField("eventTime") - .count("total") - .waterMark(1) - .setLocalStorageOnly(true) - .toDataSteam() - .toPrint(1) - .with(WindowStrategy.highPerformance()) - .start(); + .map(message -> JSONObject.parseObject((String) message)) + .window(TumblingWindow.of(Time.minutes(1))) + .groupBy("userId") + .setTimeField("eventTime") + .count("total") + .waterMark(1) + .setLocalStorageOnly(true) + .toDataSteam() + .toPrint(1) + .with(WindowStrategy.highPerformance()) + .start(); } - - } diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/checkpoint/RemoteCheckpointExample.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/checkpoint/RemoteCheckpointExample.java index 8f81a436..67b7cec7 100644 --- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/checkpoint/RemoteCheckpointExample.java +++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/checkpoint/RemoteCheckpointExample.java @@ -20,22 +20,18 @@ package org.apache.rocketmq.streams.examples.checkpoint; import com.alibaba.fastjson.JSONObject; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.source.DataStreamSource; import org.apache.rocketmq.streams.client.transform.window.Time; import org.apache.rocketmq.streams.client.transform.window.TumblingWindow; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; -import org.apache.rocketmq.streams.dbinit.mysql.delegate.DBDelegate; -import org.apache.rocketmq.streams.dbinit.mysql.delegate.DBDelegateFactory; +import org.apache.rocketmq.streams.db.init.mysql.delegate.DBDelegate; +import org.apache.rocketmq.streams.db.init.mysql.delegate.DBDelegateFactory; import org.apache.rocketmq.streams.examples.aggregate.ProducerFromFile; -import static org.apache.rocketmq.streams.db.driver.DriverBuilder.DEFALUT_JDBC_DRIVER; import static org.apache.rocketmq.streams.examples.aggregate.Constant.NAMESRV_ADDRESS; import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_CONSUMER_GROUP_NAME; import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_TOPIC; - public class RemoteCheckpointExample { //replace with your mysql url, database name can be anyone else. private static final String URL = "jdbc:mysql://localhost:3306/rocketmq_streams"; @@ -44,17 +40,16 @@ public class RemoteCheckpointExample { //password of mysql private static final String PASSWORD = ""; - - static { - ComponentCreator.getProperties().put(ConfigureFileKey.CONNECT_TYPE, "DB"); - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_URL, URL); - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_DRIVER, DEFALUT_JDBC_DRIVER); - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_USERNAME, USER_NAME); - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_PASSWORD, PASSWORD); + static { +// +// ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_URL, URL); +// ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_DRIVER, DEFAULT_JDBC_DRIVER); +// ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_USERNAME, USER_NAME); +// ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_PASSWORD, PASSWORD); } public static void main(String[] args) { - ProducerFromFile.produce("data.txt",NAMESRV_ADDRESS, RMQ_TOPIC); + ProducerFromFile.produce("data.txt", NAMESRV_ADDRESS, RMQ_TOPIC); DBDelegate delegate = DBDelegateFactory.getDelegate(); delegate.init(); @@ -64,33 +59,33 @@ public static void main(String[] args) { } System.out.println("begin streams code."); - DataStreamSource source = StreamBuilder.dataStream("namespace", "pipeline"); + DataStreamSource source = StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "pipeline"); source.fromRocketmq( RMQ_TOPIC, RMQ_CONSUMER_GROUP_NAME, false, NAMESRV_ADDRESS) - .filter((message) -> { - try { - JSONObject.parseObject((String) message); - } catch (Throwable t) { - // if can not convert to json, discard it.because all operator are base on json. - return true; - } - return false; - }) - //must convert message to json. - .map(message -> JSONObject.parseObject((String) message)) - .window(TumblingWindow.of(Time.seconds(10))) - .groupBy("ProjectName","LogStore") - .sum("OutFlow", "OutFlow") - .sum("InFlow", "InFlow") - .count("total") - .waterMark(5) - .setLocalStorageOnly(false) - .toDataSteam() - .toPrint(1) - .start(); + .filter((message) -> { + try { + JSONObject.parseObject((String) message); + } catch (Throwable t) { + // if can not convert to json, discard it.because all operator are base on json. + return true; + } + return false; + }) + //must convert message to json. + .map(message -> JSONObject.parseObject((String) message)) + .window(TumblingWindow.of(Time.seconds(10))) + .groupBy("ProjectName", "LogStore") + .sum("OutFlow", "OutFlow") + .sum("InFlow", "InFlow") + .count("total") + .waterMark(5) + .setLocalStorageOnly(false) + .toDataSteam() + .toPrint(1) + .start(); } } diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/join/RocketmqDimJoinExample.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/join/RocketmqDimJoinExample.java index 7246f6f7..02044731 100644 --- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/join/RocketmqDimJoinExample.java +++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/join/RocketmqDimJoinExample.java @@ -16,12 +16,12 @@ */ package org.apache.rocketmq.streams.examples.join; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; public class RocketmqDimJoinExample { public static void main(String[] args) { - StreamBuilder.dataStream("tmp", "tmp") + StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp") .fromRocketmq("TopicTest", "groupA", true, "localhost:9876") .dimJoin("classpath://dim.txt", 10000L) .on("ProjectName, =, project") diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/join/RocketmqJoinExample.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/join/RocketmqJoinExample.java index f4270d8a..1b1beb74 100644 --- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/join/RocketmqJoinExample.java +++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/join/RocketmqJoinExample.java @@ -16,14 +16,14 @@ */ package org.apache.rocketmq.streams.examples.join; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.transform.DataStream; public class RocketmqJoinExample { public static void main(String[] args) { - DataStream left = StreamBuilder.dataStream("tmp", "tmp") + DataStream left = StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp") .fromRocketmq("TopicTest", "groupA", true, "localhost:9876"); - DataStream right = StreamBuilder.dataStream("tmp", "tmp") + DataStream right = StreamExecutionEnvironment.getExecutionEnvironment().create("tmp", "tmp") .fromRocketmq("TopicTest", "groupB", true, "localhost:9876"); left.join(right) diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/mutilconsumer/MultiStreamsExample.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/mutilconsumer/MultiStreamsExample.java index 2b823e44..146ec8c9 100644 --- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/mutilconsumer/MultiStreamsExample.java +++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/mutilconsumer/MultiStreamsExample.java @@ -22,27 +22,26 @@ import com.alibaba.fastjson.JSONObject; import java.util.Random; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.source.DataStreamSource; import org.apache.rocketmq.streams.client.strategy.WindowStrategy; import org.apache.rocketmq.streams.client.transform.window.Time; import org.apache.rocketmq.streams.client.transform.window.TumblingWindow; +import org.apache.rocketmq.streams.common.threadpool.ThreadPoolFactory; import static org.apache.rocketmq.streams.examples.aggregate.Constant.NAMESRV_ADDRESS; import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_CONSUMER_GROUP_NAME; import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_TOPIC; public class MultiStreamsExample { - private static ExecutorService producerPool = Executors.newFixedThreadPool(1); - private static ExecutorService consumerPool = Executors.newCachedThreadPool(); + private static ExecutorService producerPool = ThreadPoolFactory.createFixedThreadPool(1, MultiStreamsExample.class.getName() + "-test_producer"); + private static ExecutorService consumerPool = ThreadPoolFactory.createCachedThreadPool(MultiStreamsExample.class.getName() + "-test_consumer"); private static Random random = new Random(); public static void main(String[] args) { //producer producerPool.submit(new Runnable() { - @Override - public void run() { + @Override public void run() { Producer.produceInLoop("data.txt"); } }); @@ -50,8 +49,7 @@ public void run() { //consumer for (int i = 0; i < 2; i++) { consumerPool.submit(new Runnable() { - @Override - public void run() { + @Override public void run() { runOneStreamsClient(random.nextInt(100)); } }); @@ -60,13 +58,8 @@ public void run() { } private static void runOneStreamsClient(int index) { - DataStreamSource source = StreamBuilder.dataStream("namespace" + index, "pipeline" + index); - source.fromRocketmq( - RMQ_TOPIC, - RMQ_CONSUMER_GROUP_NAME, - false, - NAMESRV_ADDRESS) - .filter((message) -> { + DataStreamSource source = StreamExecutionEnvironment.getExecutionEnvironment().create("namespace" + index, "pipeline" + index); + source.fromRocketmq(RMQ_TOPIC, RMQ_CONSUMER_GROUP_NAME, false, NAMESRV_ADDRESS).filter((message) -> { try { JSONObject.parseObject((String) message); } catch (Throwable t) { @@ -76,17 +69,6 @@ private static void runOneStreamsClient(int index) { return false; }) //must convert message to json. - .map(message -> JSONObject.parseObject((String) message)) - .window(TumblingWindow.of(Time.seconds(10))) - .groupBy("ProjectName", "LogStore") - .sum("OutFlow", "OutFlow") - .sum("InFlow", "InFlow") - .count("total") - .waterMark(5) - .setLocalStorageOnly(true) - .toDataSteam() - .toPrint(1) - .with(WindowStrategy.highPerformance()) - .start(); + .map(message -> JSONObject.parseObject((String) message)).window(TumblingWindow.of(Time.seconds(10))).groupBy("ProjectName", "LogStore").sum("OutFlow", "OutFlow").sum("InFlow", "InFlow").count("total").waterMark(5).setLocalStorageOnly(true).toDataSteam().toPrint(1).with(WindowStrategy.highPerformance()).start(); } } diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/mutilconsumer/Producer.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/mutilconsumer/Producer.java index 66903b45..fe397e62 100644 --- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/mutilconsumer/Producer.java +++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/mutilconsumer/Producer.java @@ -25,11 +25,15 @@ import org.apache.rocketmq.common.message.Message; import org.apache.rocketmq.remoting.common.RemotingHelper; import org.apache.rocketmq.streams.examples.aggregate.ProducerFromFile; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static org.apache.rocketmq.streams.examples.aggregate.Constant.NAMESRV_ADDRESS; import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_TOPIC; public class Producer { + private static final Logger LOGGER = LoggerFactory.getLogger(Producer.class); + private static final AtomicInteger count = new AtomicInteger(0); /** @@ -48,7 +52,7 @@ public static void produceInLoop(String fileName) { for (int i = 0; i < 100; i++) { if (count.get() % 100 == 0) { - System.out.println("already send message: " + count.get()); + LOGGER.info("already send message: " + count.get()); } for (String str : result) { @@ -61,7 +65,7 @@ public static void produceInLoop(String fileName) { } } catch (Throwable t) { - + LOGGER.error("produce error", t); } } diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/sink/MqttSinkExample.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/sink/MqttSinkExample.java index 42fd8638..3a54653d 100644 --- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/sink/MqttSinkExample.java +++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/sink/MqttSinkExample.java @@ -16,13 +16,13 @@ */ package org.apache.rocketmq.streams.examples.sink; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.source.DataStreamSource; public class MqttSinkExample { public static void main(String[] args) { - DataStreamSource source = StreamBuilder.dataStream("namespace", "pipeline"); + DataStreamSource source = StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "pipeline"); source.fromFile("/Users/junjie.cheng/jobs/access.log", false) .map(message -> message) .toMqtt("tpc://localhost:1883", "test", "test") diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/FileSourceExample.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/FileSourceExample.java index ecaf0bca..b5f4f218 100644 --- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/FileSourceExample.java +++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/FileSourceExample.java @@ -16,13 +16,13 @@ */ package org.apache.rocketmq.streams.examples.source; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.source.DataStreamSource; public class FileSourceExample { public static void main(String[] args) { - DataStreamSource source = StreamBuilder.dataStream("namespace", "pipeline"); - source.fromFile("/Users/junjie.cheng/jobs/access.log", false) + DataStreamSource source = StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "pipeline"); + source.fromFile("/Users/junjie.cheng/logs/jmonitor.log", false) .map(message -> message) .toPrint(1) .start(); diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/MqttSourceExample.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/MqttSourceExample.java index d0dbad59..01affba5 100644 --- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/MqttSourceExample.java +++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/MqttSourceExample.java @@ -17,7 +17,7 @@ package org.apache.rocketmq.streams.examples.source; import com.alibaba.fastjson.JSONObject; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.source.DataStreamSource; import org.apache.rocketmq.streams.client.strategy.ShuffleStrategy; import org.apache.rocketmq.streams.client.transform.window.Time; @@ -26,7 +26,7 @@ public class MqttSourceExample { public static void main(String[] args) { - DataStreamSource dataStream = StreamBuilder.dataStream("test_namespace", "graph_pipeline"); + DataStreamSource dataStream = StreamExecutionEnvironment.getExecutionEnvironment().create("test_namespace", "graph_pipeline"); dataStream.fromMqtt("xxxxx", "xxxx", "xxxxxx", "", "") .flatMap(message -> ((JSONObject) message).getJSONArray("Data")) .window(TumblingWindow.of(Time.minutes(1))) diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample1.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample1.java index 9b293d5c..e770e2fb 100644 --- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample1.java +++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample1.java @@ -16,7 +16,7 @@ */ package org.apache.rocketmq.streams.examples.source; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.source.DataStreamSource; import org.apache.rocketmq.streams.examples.aggregate.ProducerFromFile; @@ -24,10 +24,9 @@ import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_CONSUMER_GROUP_NAME; import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_TOPIC; - public class RocketmqSourceExample1 { public static void main(String[] args) { - ProducerFromFile.produce("data.txt",NAMESRV_ADDRESS, RMQ_TOPIC); + ProducerFromFile.produce("data.txt", NAMESRV_ADDRESS, RMQ_TOPIC); try { Thread.sleep(1000 * 3); @@ -36,7 +35,7 @@ public static void main(String[] args) { System.out.println("begin streams code."); - DataStreamSource source = StreamBuilder.dataStream("namespace", "pipeline"); + DataStreamSource source = StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "pipeline"); /** * 1、before run this case, make sure some data has already been rocketmq. */ @@ -44,10 +43,10 @@ public static void main(String[] args) { RMQ_TOPIC, RMQ_CONSUMER_GROUP_NAME, NAMESRV_ADDRESS - ) - .map(message -> message) - .toPrint(1) - .start(); + ) + .map(message -> message) + .toPrint(1) + .start(); } } diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample2.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample2.java index f24fe6ca..8cbe793b 100644 --- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample2.java +++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample2.java @@ -17,7 +17,7 @@ package org.apache.rocketmq.streams.examples.source; import java.util.Arrays; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.source.DataStreamSource; import org.apache.rocketmq.streams.examples.aggregate.ProducerFromFile; @@ -39,7 +39,7 @@ public static void main(String[] args) { System.out.println("begin streams code."); - DataStreamSource source = StreamBuilder.dataStream("namespace", "pipeline"); + DataStreamSource source = StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "pipeline"); source.fromRocketmq( RMQ_TOPIC, RMQ_CONSUMER_GROUP_NAME, diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample3.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample3.java index 72d94c1d..ccfd976c 100644 --- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample3.java +++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample3.java @@ -22,7 +22,7 @@ import java.util.List; import java.util.Map; import java.util.Set; -import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.StreamExecutionEnvironment; import org.apache.rocketmq.streams.client.source.DataStreamSource; import org.apache.rocketmq.streams.examples.aggregate.ProducerFromFile; @@ -44,7 +44,7 @@ public static void main(String[] args) { System.out.println("begin streams code."); - DataStreamSource source = StreamBuilder.dataStream("namespace", "pipeline"); + DataStreamSource source = StreamExecutionEnvironment.getExecutionEnvironment().create("namespace", "pipeline"); source.fromRocketmq( RMQ_TOPIC, RMQ_CONSUMER_GROUP_NAME, diff --git a/rocketmq-streams-filter/pom.xml b/rocketmq-streams-filter/pom.xml index 2adf0e57..b9675164 100755 --- a/rocketmq-streams-filter/pom.xml +++ b/rocketmq-streams-filter/pom.xml @@ -15,14 +15,14 @@ See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 org.apache.rocketmq rocketmq-streams - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT rocketmq-streams-filter ROCKETMQ STREAMS :: filter diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/FilterComponent.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/FilterComponent.java index 8a28d768..44be35a4 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/FilterComponent.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/FilterComponent.java @@ -19,8 +19,6 @@ import com.alibaba.fastjson.JSONObject; import java.util.List; import java.util.Properties; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.component.AbstractComponent; import org.apache.rocketmq.streams.common.component.ComponentCreator; import org.apache.rocketmq.streams.common.component.IgnoreNameSpace; @@ -37,8 +35,6 @@ */ public class FilterComponent extends AbstractComponent implements IRuleEngineService, IgnoreNameSpace { - private static final Log LOG = LogFactory.getLog(FilterComponent.class); - private static final FilterComponent filterComponent = ComponentCreator.getComponent(null, FilterComponent.class); protected RuleEngineServiceImpl ruleEngineService; @@ -50,6 +46,13 @@ public static FilterComponent getInstance() { return filterComponent; } + public static RuleEngineServiceImpl createRuleEngineService(Properties properties) { + RuleEngineServiceImpl ruleEngineService = new RuleEngineServiceImpl(); + ContextConfigure config = new ContextConfigure(properties); + ruleEngineService.initRuleContext(config); + return ruleEngineService; + } + @Override public boolean stop() { return true; @@ -65,13 +68,6 @@ protected boolean startComponent(String namespace) { return true; } - public static RuleEngineServiceImpl createRuleEngineService(Properties properties) { - RuleEngineServiceImpl ruleEngineService = new RuleEngineServiceImpl(); - ContextConfigure config = new ContextConfigure(properties); - ruleEngineService.initRuleContext(config); - return ruleEngineService; - } - @Override protected boolean initProperties(Properties properties) { if (ruleEngineService != null) { @@ -86,7 +82,7 @@ protected boolean initProperties(Properties properties) { @Override public Rule createRule(String namespace, String ruleName, String expressionStr, String... msgMetaInfo) { RuleBuilder ruleCreator = new RuleBuilder(namespace, ruleName, expressionStr, msgMetaInfo); - Rule rule = ruleCreator.generateRule(null); + Rule rule = ruleCreator.generateRule(); return rule; } @@ -100,6 +96,11 @@ public List executeRule(IMessage message, AbstractContext context, Rule... return ruleEngineService.executeRule(message, context, rules); } + @Override + public List executeRule(IMessage message, AbstractContext context, Rule rule) { + return ruleEngineService.executeRule(message, context, rule); + } + @Override public List executeRule(IMessage message, AbstractContext context, List rules) { return ruleEngineService.executeRule(message, context, rules); } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/ExpressionBuilder.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/ExpressionBuilder.java index 6a4b7902..72a8f7e2 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/ExpressionBuilder.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/ExpressionBuilder.java @@ -46,6 +46,37 @@ * 必须先初始化组件后才可以使用 */ public class ExpressionBuilder { + protected static IRuleExpressionCreator ruleExpressionCreator = new IRuleExpressionCreator() { + @Override + public String createExpression(String namespace, String ruleName, String expresionStr, List expressions, boolean containsConstant, Map flag2ExpressionStr, NameCreator nameCreator, String relationStr) { + String[] values = ExpressionBuilder.createElement(expresionStr, containsConstant, flag2ExpressionStr); + String expressionName = nameCreator.createName(ruleName); + + SimpleExpression expression = null; + if (values.length == 3) { + String value = values[2]; + expression = new SimpleExpression(values[0], values[1], value); + + } + if (values.length == 4) { + String value = values[3]; + expression = + new SimpleExpression(values[0], values[1], DataTypeUtil.getDataType(values[2]), value); + } + if (expression != null) { + expression.setNameSpace(namespace); + expression.setName(expressionName); + if (!relationStr.contains("(")) { + relationStr = relationStr.replace(expresionStr, expressionName); + } else { + relationStr = relationStr.replace("(" + expresionStr + ")", expressionName); + } + + expressions.add(expression); + } + return relationStr; + } + }; private static ListDataType listDataType = new ListDataType(DataTypeUtil.getDataTypeFromClass(String.class)); private static ICache cache = new SoftReferenceCache<>(); @@ -107,8 +138,8 @@ public static boolean executeExecute(String namespace, String expressionStr, JSO public static boolean executeExecute(Expression expression, IMessage message, AbstractContext context) { Rule rule = null; - if (expression.getConfigureName() == null) { - expression.setConfigureName(expression.getVarName()); + if (expression.getName() == null) { + expression.setName(expression.getVarName()); } String key = MapKeyUtil.createKey(expression.getNameSpace(), expression.toString()); rule = cache.get(key); @@ -121,8 +152,8 @@ public static boolean executeExecute(Expression expression, IMessage message, Ab public static boolean executeExecute(Expression expression, JSONObject msg) { Rule rule = null; - if (expression.getConfigureName() == null) { - expression.setConfigureName(expression.getVarName()); + if (expression.getName() == null) { + expression.setName(expression.getVarName()); } String key = MapKeyUtil.createKey(expression.getNameSpace(), expression.toString()); rule = cache.get(key); @@ -144,7 +175,7 @@ public static Expression createExpression(String namespace, String ruleName, Str List relationExpressions) { String relationStr = parseExpression(namespace, ruleName, expressionStr, expressions); if (expressions != null && expressions.size() == 1) { - if (expressions.get(0).getConfigureName().equals(relationStr)) { + if (expressions.get(0).getName().equals(relationStr)) { Expression expression = expressions.get(0); ContextVar contextVar = new ContextVar(); contextVar.setFieldName(expression.getVarName()); @@ -164,11 +195,11 @@ public static Expression createExpression(String namespace, String ruleName, Str ContextVar contextVar = new ContextVar(); contextVar.setFieldName(expression.getVarName()); expression.setVar(contextVar); - map.put(expression.getConfigureName(), expression); + map.put(expression.getName(), expression); } for (RelationExpression relation : relationExpressions) { relation.setExpressionMap(map); - map.put(relation.getConfigureName(), relation); + map.put(relation.getName(), relation); } return relationExpression; } @@ -228,7 +259,6 @@ public static Rule createRule(String namespace, String ruleName, Expression expr ruleCreator.setRootExpression(expression); rule = ruleCreator.createRule(); rule.initElements(); - ruleCreator.getMetaData().toObject(ruleCreator.getMetaData().toJson());//metadata的一个bug,如果不做这步,map为空。后续版本修复后可以去掉 return rule; } @@ -271,7 +301,7 @@ private static void addConfigurable2Map(Map if (Var.class.isInstance(t)) { map.put(((Var) t).getVarName(), t); } else { - map.put(t.getConfigureName(), t); + map.put(t.getName(), t); } } @@ -367,7 +397,7 @@ protected static String parseExpression(String namespace, String ruleName, Strin return relationStr; } else { - String relationFlag = NameCreatorContext.get().createNewName("relation"); + String relationFlag = NameCreatorContext.get().createName("relation"); flag2ExpressionStr.put(relationFlag, oriStr); relationStr = relationStr.replace(oriStr, relationFlag); return parseExpression(namespace, ruleName, relationStr, expressions, flag2ExpressionStr, creator, containsContant, nameCreator, contantsFlags); @@ -377,42 +407,6 @@ protected static String parseExpression(String namespace, String ruleName, Strin return relationStr; } - interface IRuleExpressionCreator { - String createExpression(String namespace, String ruleName, String expresionStr, List expressions, boolean containsConstant, Map flag2ExpressionStr, NameCreator nameCreator, String relationStr); - } - - protected static IRuleExpressionCreator ruleExpressionCreator = new IRuleExpressionCreator() { - @Override - public String createExpression(String namespace, String ruleName, String expresionStr, List expressions, boolean containsConstant, Map flag2ExpressionStr, NameCreator nameCreator, String relationStr) { - String[] values = ExpressionBuilder.createElement(expresionStr, containsConstant, flag2ExpressionStr); - String expressionName = nameCreator.createName(ruleName); - - SimpleExpression expression = null; - if (values.length == 3) { - String value = values[2]; - expression = new SimpleExpression(values[0], values[1], value); - - } - if (values.length == 4) { - String value = values[3]; - expression = - new SimpleExpression(values[0], values[1], DataTypeUtil.getDataType(values[2]), value); - } - if (expression != null) { - expression.setNameSpace(namespace); - expression.setConfigureName(expressionName); - if (!relationStr.contains("(")) { - relationStr = relationStr.replace(expresionStr, expressionName); - } else { - relationStr = relationStr.replace("(" + expresionStr + ")", expressionName); - } - - expressions.add(expression); - } - return relationStr; - } - }; - public static String[] createElement(String expresionStr, boolean containsContant, Map flag2ExpressionStr) { String[] values = expresionStr.split(","); @@ -430,4 +424,8 @@ public static String[] createElement(String expresionStr, boolean containsContan } return values; } + + interface IRuleExpressionCreator { + String createExpression(String namespace, String ruleName, String expresionStr, List expressions, boolean containsConstant, Map flag2ExpressionStr, NameCreator nameCreator, String relationStr); + } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/RuleBuilder.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/RuleBuilder.java index 50daf3a0..f62d7294 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/RuleBuilder.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/RuleBuilder.java @@ -25,7 +25,6 @@ import java.util.Set; import org.apache.rocketmq.streams.common.channel.sink.ISink; import org.apache.rocketmq.streams.common.configurable.IConfigurable; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; import org.apache.rocketmq.streams.common.datatype.DataType; import org.apache.rocketmq.streams.common.metadata.MetaData; import org.apache.rocketmq.streams.common.metadata.MetaDataField; @@ -88,7 +87,7 @@ public RuleBuilder(String namespace, String ruleName, String expressionStr, Stri for (Expression express : expressionList) { Var var = RuleElementBuilder.createContextVar(namespace, ruleName, express.getVarName(), null, express.getVarName()); - var.setConfigureName(express.getVarName()); + var.setName(express.getVarName()); varList.add(var); } return; @@ -99,10 +98,10 @@ public RuleBuilder(String namespace, String ruleName, String expressionStr, Stri Set existVarNames = new HashSet<>();// for (MetaDataField metaDataField : metaDataFields) { Var var = RuleElementBuilder.createContextVar(namespace, ruleName, metaDataField.getFieldName(), - metaData.getConfigureName(), metaDataField.getFieldName()); - var.setConfigureName(metaDataField.getFieldName()); + metaData.getName(), metaDataField.getFieldName()); + var.setName(metaDataField.getFieldName()); varList.add(var); - existVarNames.add(var.getConfigureName()); + existVarNames.add(var.getName()); } for (Expression express : expressionList) { if (existVarNames.contains(express.getVarName())) { @@ -113,13 +112,13 @@ public RuleBuilder(String namespace, String ruleName, String expressionStr, Stri } Var var = RuleElementBuilder.createContextVar(namespace, ruleName, express.getVarName(), null, express.getVarName()); - var.setConfigureName(express.getVarName()); + var.setName(express.getVarName()); varList.add(var); } metaData.toObject(metaData.toJson()); this.metaData = metaData; if (metaData != null) { - this.rule.setMsgMetaDataName(metaData.getConfigureName()); + this.rule.setMsgMetaDataName(metaData.getName()); } metaDataList.add(this.metaData); } @@ -139,35 +138,6 @@ public Rule getRule() { return rule; } - public RuleBuilder(IConfigurableService ruleEngineConfigurableService, String namespace, String ruleName) { - Rule rule = - (Rule)ruleEngineConfigurableService.queryConfigurableByIdent(RuleElementType.RULE.getType(), ruleName); - if (rule == null) { - init(namespace, ruleName); - createChannelMetaData(); - return; - } - this.metaData = - (MetaData)ruleEngineConfigurableService.queryConfigurableByIdent(RuleElementType.METADATA.getType(), - rule.getMsgMetaDataName()); - this.rootExpression = - (Expression)ruleEngineConfigurableService.queryConfigurableByIdent(RuleElementType.EXPRESSION.getType(), - rule.getExpressionName()); - this.rule = rule; - this.ruleName = rule.getConfigureName(); - this.namespace = rule.getNameSpace(); - this.ruleTitle = rule.getRuleTitle(); - this.ruleCode = rule.getRuleCode(); - this.ruleDescription = rule.getRuleDesc(); - this.varList = ruleEngineConfigurableService.queryConfigurableByType(RuleElementType.VAR.getType()); - this.expressionList = - ruleEngineConfigurableService.queryConfigurableByType(RuleElementType.EXPRESSION.getType()); - this.metaDataList = ruleEngineConfigurableService.queryConfigurableByType(RuleElementType.METADATA.getType()); - this.actionList = ruleEngineConfigurableService.queryConfigurableByType(RuleElementType.ACTION.getType()); - this.dataSourceList = - ruleEngineConfigurableService.queryConfigurableByType(RuleElementType.DATASOURCE.getType()); - } - /** * 创建规则必须的信息。 * @@ -177,7 +147,7 @@ public RuleBuilder(IConfigurableService ruleEngineConfigurableService, String na */ private void init(String namespace, String ruleName, String... rulecode_title_description) { rule.setNameSpace(namespace); - rule.setConfigureName(ruleName); + rule.setName(ruleName); rule.setRuleStatus(3); this.namespace = namespace; this.ruleName = ruleName; @@ -232,7 +202,7 @@ public RuleBuilder addVarAndMetaDataField(String... fieldNameTypeIsRequireds) { this.metaData.getMetaDataFields().addAll(metaDataFields); for (MetaDataField metaDataField : metaDataFields) { ContextVar var = RuleElementBuilder.createContextVar(namespace, ruleName, metaDataField.getFieldName(), - metaData.getConfigureName(), metaDataField.getFieldName()); + metaData.getName(), metaDataField.getFieldName()); varList.add(var); } return this; @@ -316,7 +286,7 @@ public RuleBuilder addExpression(Expression expression) { * @return */ public RuleBuilder addExpression(String expressionName, String varName, String functionName, DataType dataType, - String value) { + String value) { Expression expression = RuleElementBuilder.createExpression(ruleName, expressionName, varName, functionName, dataType, value); expressionList.add(expression); @@ -369,38 +339,24 @@ public RuleBuilder addRelationExpression(String relationStr) { // return this; //} - - - - - - /** * 保存规则相关的对象 * - * @param ruleEngineConfigurableService * @return */ - public Rule generateRule(IConfigurableService ruleEngineConfigurableService) { + public Rule generateRule() { Rule rule = createRule(); - insertOrUpdate(ruleEngineConfigurableService); + insertOrUpdate(); rule.initElements(); return rule; } - public Rule generateRule() { - return generateRule(null); - } - - private void insertOrUpdate(IConfigurableService ruleEngineConfigurableService) { - insertOrUpdate(ruleEngineConfigurableService, metaDataList, MetaData.TYPE); - insertOrUpdate(ruleEngineConfigurableService, varList, Var.TYPE); - insertOrUpdate(ruleEngineConfigurableService, expressionList, Expression.TYPE); - insertOrUpdate(ruleEngineConfigurableService, actionList, Action.TYPE); - insertOrUpdate(ruleEngineConfigurableService, dataSourceList, ISink.TYPE); - if (ruleEngineConfigurableService != null) { - ruleEngineConfigurableService.insert(rule); - } + private void insertOrUpdate() { + insertOrUpdate(metaDataList, MetaData.TYPE); + insertOrUpdate(varList, Var.TYPE); + insertOrUpdate(expressionList, Expression.TYPE); + insertOrUpdate(actionList, Action.TYPE); + insertOrUpdate(dataSourceList, ISink.TYPE); } /** @@ -412,14 +368,15 @@ protected Rule createRule() { if (rootExpression == null && expressionList != null && expressionList.size() > 0) { rootExpression = expressionList.get(0); } - rule.setExpressionName(rootExpression.getConfigureName()); + rule.setRootExpression(rootExpression); + rule.setExpressionName(rootExpression.getName()); rule.setActionNames(convert(actionList)); rule.setVarNames(convert(varList)); rule.setRuleCode(ruleCode); rule.setRuleTitle(ruleTitle); rule.setRuleDesc(ruleDescription); if (this.metaData != null) { - rule.setMsgMetaDataName(metaData.getConfigureName()); + rule.setMsgMetaDataName(metaData.getName()); } return rule; } @@ -467,15 +424,12 @@ private String[] createMetaDataFiledStr(Map varName2FieldName) { return filedStr; } - private void insertOrUpdate(IConfigurableService ruleEngineConfigurableService, - List configurables, String type) { + private void insertOrUpdate( + List configurables, String type) { if (configurables == null) { return; } for (IConfigurable configurable : configurables) { - if (ruleEngineConfigurableService != null) { - ruleEngineConfigurableService.insert(configurable); - } rule.putConfigurableMap(configurable, type); } } @@ -483,7 +437,7 @@ private void insertOrUpdate(IConfigurableService ruleE private List convert(List configurables) { List configurableNames = new ArrayList<>(); for (T t : configurables) { - configurableNames.add(t.getConfigureName()); + configurableNames.add(t.getName()); } return configurableNames; } @@ -532,6 +486,10 @@ public String getRuleName() { return ruleName; } + public void setRuleName(String ruleName) { + this.ruleName = ruleName; + } + public Expression getRootExpression() { return rootExpression; } @@ -540,10 +498,6 @@ public void setRootExpression(Expression rootExpression) { this.rootExpression = rootExpression; } - public void setRuleName(String ruleName) { - this.ruleName = ruleName; - } - public String getRuleCode() { return ruleCode; } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/RuleElementBuilder.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/RuleElementBuilder.java index 37efcbec..c8171bfa 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/RuleElementBuilder.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/RuleElementBuilder.java @@ -39,7 +39,7 @@ public class RuleElementBuilder { public static MetaData createMetaData(String namespace, String name, String... fields) { MetaData metaData = new MetaData(); metaData.setNameSpace(namespace); - metaData.setConfigureName(name); + metaData.setName(name); if (fields == null || fields.length == 0) { metaData.toObject(metaData.toJson()); return metaData; @@ -70,7 +70,6 @@ public static MetaData createMetaData(String namespace, String name, String... f metaDataFieldList.add(metaDataField); } metaData.getMetaDataFields().addAll(metaDataFieldList); - metaData.toObject(metaData.toJson()); return metaData; } @@ -79,7 +78,7 @@ public static ContextVar createContextVar(String namespace, String ruleName, Str contextVar.setNameSpace(namespace); contextVar.setType(RuleElementType.VAR.getType()); contextVar.setVarName(varName); - contextVar.setConfigureName(MapKeyUtil.createKeyBySign("_", ruleName, varName)); + contextVar.setName(MapKeyUtil.createKeyBySign("_", ruleName, varName)); contextVar.setMetaDataName(metaDataName); contextVar.setFieldName(fieldName); return contextVar; @@ -90,18 +89,18 @@ public static ConstantVar createConstantVar(String namespace, String ruleName, S constantVar.setNameSpace(ruleName); constantVar.setDataType(dataType); constantVar.setVarName(varName); - constantVar.setConfigureName(MapKeyUtil.createKeyBySign("_", ruleName, varName)); + constantVar.setName(MapKeyUtil.createKeyBySign("_", ruleName, varName)); constantVar.setValue(dataType.getData(value)); constantVar.setType(RuleElementType.VAR.getType()); return constantVar; } public static Expression createExpression(String ruleName, String expressionName, String varName, - String functionName, DataType dataType, String value) { + String functionName, DataType dataType, String value) { Expression expression = new Expression(); expression.setNameSpace(ruleName); expression.setType(RuleElementType.EXPRESSION.getType()); - expression.setConfigureName(expressionName); + expression.setName(expressionName); expression.setVarName(varName); expression.setValue(dataType.getData(value)); expression.setDataType(dataType); @@ -110,16 +109,14 @@ public static Expression createExpression(String ruleName, String expressionName } public static RelationExpression createRelationExpression(String ruleName, String expressionName, String relation, - List expressionNames) { + List expressionNames) { RelationExpression relationExpression = new RelationExpression(); relationExpression.setNameSpace(ruleName); relationExpression.setType(RuleElementType.EXPRESSION.getType()); - relationExpression.setConfigureName(expressionName); + relationExpression.setName(expressionName); relationExpression.setRelation(relation); relationExpression.setValue(expressionNames); return relationExpression; } - - } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/context/ContextConfigure.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/context/ContextConfigure.java index e5c21458..631d3674 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/context/ContextConfigure.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/context/ContextConfigure.java @@ -20,69 +20,65 @@ public class ContextConfigure { + public static final int MONITOR_OPEN = 1; // 开启模式 + public static final int MONITOR_CLOSE = -1; // 关闭模式 + public static final int MONITOR_SAMPLE = 0; // 关闭模式 + public static final int MONITOR_STATISTICS = 2; // 只统计时间,不输出 /** * 正则表达式优化相关 */ protected volatile boolean isSupportRegexOptimization = true; // 是否支持正则表达式优化 protected volatile int regexTimeout = 1000; // 正则表达式超时时间,默认100ms - protected int actionPoolSize = 5; /** * 控制监控输出 */ protected int monitorPoolSize = 10000; // 监控对象池大小 - private volatile String rulesLogsIps = "127.0.0.1"; protected volatile boolean fetchDataInThisHost = false; // 是否在本机采样,在采样模式下使用 protected volatile int longTimeAlert = 1000; // 规则超过500ms则打印输出 protected volatile int mode = MONITOR_STATISTICS; // 0:开启模式;1,采样模式;-1:关闭模式 - public static final int MONITOR_OPEN = 1; // 开启模式 - public static final int MONITOR_CLOSE = -1; // 关闭模式 - public static final int MONITOR_SAMPLE = 0; // 关闭模式 - public static final int MONITOR_STATISTICS = 2; // 只统计时间,不输出 /** * 区分引擎运行的环境 */ protected String evnNamespace; // 区分压测环境和线上环境 - /** * 控制结果输出 */ protected boolean action2Online = true; // 是否执行在线的action,如果是false,不执行线上的action protected boolean action2Observer = false; // 观察中的规则是否输出到观察表,如果为true,则写到观察表 protected boolean actionOnline2Observer = false; // 在线规则输出到观察表,如果为true,则写到观察表,此种情况是为了测试 - /** * 控制加载规则 */ protected String loadRuleStatuses = "3"; // 规则加载方式,根据规则的状态加载 protected String diyRuleCodes = null; // 如果规则状态选择自定义,则此处列出规则的状态,用逗号分隔 + /** + * 分流用对象 + */ + protected boolean openUnRepeateScript = false; // ///** // * 分流用对象 // */ //protected transient SplitFlow splitFlow = new SplitFlow(); - - /** - * 分流用对象 - */ - protected boolean openUnRepeateScript = false; - protected boolean openObserverDatasource = true; - /** * 是否使用缓存处理正则 */ protected boolean encacheFlag = false; - /** * 升级中心 压测数据 */ protected int pressureTotalCount = 10000; - /** * 恶意IP最大加载数 */ protected int maliciousIPMaxCount = 1000000; + private volatile String rulesLogsIps = "127.0.0.1"; + + public ContextConfigure(Properties properties) { + + } /** * 是否支持监控模块输出超时日志 @@ -99,10 +95,6 @@ public boolean canMonitor() { return false; } - public ContextConfigure(Properties properties) { - - } - /** * 是否支持规则统计(统计慢规则) * diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/context/RuleContext.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/context/RuleContext.java index 865683c6..17ebb0b3 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/context/RuleContext.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/context/RuleContext.java @@ -17,7 +17,6 @@ package org.apache.rocketmq.streams.filter.context; import com.alibaba.fastjson.JSONObject; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.io.Serializable; import java.util.ArrayList; import java.util.List; @@ -26,20 +25,12 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.Message; import org.apache.rocketmq.streams.common.metadata.MetaData; import org.apache.rocketmq.streams.common.metadata.MetaDataAdapter; import org.apache.rocketmq.streams.common.monitor.IMonitor; -import org.apache.rocketmq.streams.common.monitor.TopologyFilterMonitor; import org.apache.rocketmq.streams.db.driver.JDBCDriver; import org.apache.rocketmq.streams.filter.function.expression.ExpressionFunction; import org.apache.rocketmq.streams.filter.operator.Rule; @@ -49,31 +40,24 @@ import org.apache.rocketmq.streams.filter.operator.var.Var; import org.apache.rocketmq.streams.script.function.model.FunctionConfigure; import org.apache.rocketmq.streams.script.function.service.impl.ScanFunctionService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class RuleContext extends AbstractContext implements Serializable { - private static final Log LOG = LogFactory.getLog(RuleContext.class); - /** * 观察者对应的configure */ public static final String OBSERVER_NAME = "observerDBAction"; - + private static final Logger LOGGER = LoggerFactory.getLogger(RuleContext.class); private static RuleContext superRuleContext; - + private static volatile boolean initflag = false; private ExecutorService actionExecutor = null; - - /** - * 默认的命名空间 - */ - public static final String DEFALUT_NAME_SPACE = IConfigurableService.PARENT_CHANNEL_NAME_SPACE; - /** * var的名称合值对应的map */ private ConcurrentMap varValueMap = new ConcurrentHashMap(); private ConcurrentMap expressionValueMap = new ConcurrentHashMap(); - /** * 错误信息 */ @@ -82,69 +66,33 @@ public class RuleContext extends AbstractContext implements Serializabl * 当前的命名空间 */ private String nameSpace; - /** * 配置服务 */ // private transient IConfigurableService configureService; private transient ScanFunctionService functionService = ScanFunctionService.getInstance(); - private transient RuleContext parentContext; private transient Rule rule; - /** * 监控一条信息的运行情况 */ private transient volatile IMonitor ruleMonitor; + // private transient static volatile AtomicBoolean initflag = new AtomicBoolean(false); /** * 一个规则对应的系统配置 */ private transient ContextConfigure contextConfigure; - // private transient static volatile AtomicBoolean initflag = new AtomicBoolean(false); - - private static volatile boolean initflag = false; - - - - public static void initSuperRuleContext(ContextConfigure contextConfigure) { - if (!initflag) { - synchronized (RuleContext.class) { - if (!initflag) { - RuleContext staticRuleContext = new RuleContext(DEFALUT_NAME_SPACE, contextConfigure); - ThreadFactory actionFactory = new ThreadFactoryBuilder().setNameFormat("RuleContext-Action-Poo-%d").build(); - int threadSize = contextConfigure.getActionPoolSize(); - ExecutorService actionExecutor = new ThreadPoolExecutor(threadSize, threadSize, 0L, TimeUnit.MILLISECONDS, - new LinkedBlockingQueue<>(1024), actionFactory, new ThreadPoolExecutor.AbortPolicy()); - staticRuleContext.actionExecutor = actionExecutor; - staticRuleContext.functionService.scanePackage("org.apache.rocketmq.streams.filter.function"); - superRuleContext = staticRuleContext; - initflag = true; - } - } - } - - } - private RuleContext(String pnameSpace, ContextConfigure contextConfigure) { this(pnameSpace, new JSONObject(), null, contextConfigure); } public RuleContext(String nameSpace, JSONObject pmessage, Rule rule, ContextConfigure contextConfigure) { super(new Message(pmessage)); - if (!DEFALUT_NAME_SPACE.equals(nameSpace)) { - this.parentContext = superRuleContext; - } this.nameSpace = nameSpace; this.rule = rule; this.contextConfigure = contextConfigure; - if (contextConfigure == null && parentContext != null) { - this.contextConfigure = this.parentContext.getContextConfigure(); - } - if (this.functionService == null && this.parentContext != null) { - this.functionService = this.parentContext.getFunctionService(); - } } @@ -156,16 +104,36 @@ public RuleContext(JSONObject pmessage, Rule rule) { this(rule.getNameSpace(), pmessage, rule, new ContextConfigure(null)); } + public static void addNotFireExpressionMonitor( + Object expression, AbstractContext context) { + + if (RelationExpression.class.isInstance(expression)) { + List notFireExpressionMonitor = new ArrayList<>(); + RelationExpression relationExpression = (RelationExpression) expression; + notFireExpressionMonitor.add(relationExpression.getName()); + context.setNotFireExpressionMonitor(notFireExpressionMonitor); + } else if (Expression.class.isInstance(expression)) { + Expression e = (Expression) expression; + context.getNotFireExpressionMonitor().add(e.getName()); + } else if (String.class.isInstance(expression)) { + context.getNotFireExpressionMonitor().add((String) expression); + } else { + LOGGER.warn("can not support the express " + expression); + } + + } + public String getNameSpace() { if (nameSpace != null) { return nameSpace; } - if (parentContext != null) { - return parentContext.getNameSpace(); - } return nameSpace; } + public void setNameSpace(String nameSpace) { + this.nameSpace = nameSpace; + } + /** * 获取表达式值 * @@ -230,7 +198,7 @@ public ExpressionFunction getExpressionFunction(String name, Object... objects) } return (ExpressionFunction) fc.getBean(); } catch (Exception e) { - LOG.error("RuleContext getExpressionFunction error,name is: " + name, e); + LOGGER.error("RuleContext getExpressionFunction error,name is: " + name, e); return null; } @@ -264,59 +232,6 @@ public boolean containsVarName(String varName) { return false; } - /** - * 获取变量值,内部使用,不能直接使用,获取变量的值需要用var.getVarValue() - * - * @param varName - * @return - */ - public Object getVarCacheValue(String varName) { - Object value = varValueMap.get(varName); - if (value != null) { - return value; - } - if (parentContext != null) { - value = parentContext.getVarCacheValue(varName); - } - return value; - - } - - /** - * 设置变量值 - * - * @param varName - * @param value - */ - public void putVarValue(String nameSpace, String varName, Object value) { - if (varName == null || value == null) { - return; - } - if (rule.getVarMap().containsKey(varName)) { - varValueMap.putIfAbsent(varName, value); - } - - } - - public void setContextConfigure(ContextConfigure contextConfigure) { - this.contextConfigure = contextConfigure; - } - - public ContextConfigure getContextConfigure() { - if (contextConfigure != null) { - return contextConfigure; - } - if (parentContext != null) { - return parentContext.getContextConfigure(); - } - return null; - } - - @Override - public IConfigurableService getConfigurableService() { - return rule.getConfigurableService(); - } - @Override public AbstractContext copy() { IMessage message = this.message.deepCopy(); @@ -327,24 +242,10 @@ public AbstractContext copy() { context.expressionValueMap = expressionValueMap; context.functionService = functionService; context.ruleMonitor = ruleMonitor; - context.parentContext = parentContext; context.varValueMap = varValueMap; - context.configurableService = configurableService; return context; } - public ExecutorService getActionExecutor() { - if (actionExecutor != null) { - return actionExecutor; - } else { - return parentContext.getActionExecutor(); - } - } - - public void setNameSpace(String nameSpace) { - this.nameSpace = nameSpace; - } - public ConcurrentMap getVarValueMap() { return varValueMap; } @@ -373,14 +274,6 @@ public ScanFunctionService getFunctionService() { return functionService; } - public RuleContext getParentContext() { - return parentContext; - } - - public void setParentContext(RuleContext parentContext) { - this.parentContext = parentContext; - } - public Rule getRule() { return rule; } @@ -396,28 +289,4 @@ public IMonitor getRuleMonitor() { public void setRuleMonitor(IMonitor ruleMonitor) { this.ruleMonitor = ruleMonitor; } - - public static void addNotFireExpressionMonitor( - Object expression,AbstractContext context) { - - if(RelationExpression.class.isInstance(expression)){ - List notFireExpressionMonitor=new ArrayList<>(); - RelationExpression relationExpression=(RelationExpression) expression; - for(String expressionName:notFireExpressionMonitor){ - if(!relationExpression.getValue().contains(expressionName)){ - notFireExpressionMonitor.add(expressionName); - } - } - notFireExpressionMonitor.add(relationExpression.getConfigureName()); - context.setNotFireExpressionMonitor(notFireExpressionMonitor); - }else if(Expression.class.isInstance(expression)) { - Expression e=(Expression)expression; - context.getNotFireExpressionMonitor().add(e.getConfigureName()); - }else if(String.class.isInstance(expression)){ - context.getNotFireExpressionMonitor().add((String)expression); - }else { - LOG.warn("can not support the express "+expression); - } - - } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/engine/SplitFlow.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/engine/SplitFlow.java index 6589fe15..dc94c479 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/engine/SplitFlow.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/engine/SplitFlow.java @@ -17,25 +17,23 @@ package org.apache.rocketmq.streams.filter.engine; import com.alibaba.fastjson.JSONObject; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class SplitFlow { - private static final Log LOG = LogFactory.getLog(SplitFlow.class); + private static final Logger LOGGER = LoggerFactory.getLogger(SplitFlow.class); + // 全部尾号 + private static final String NUMBERS_ALL = "all"; + // 没有尾号 + private static final String NUMBERS_NONE = "none"; private volatile String nameSpace; private JSONObject uuidNameMap = new JSONObject(); - // uuid尾号 private volatile String uuidEndNum = NUMBERS_ALL; private volatile String ageis_split_namespaces = "topic_aegis_detect_msg_proc"; private volatile String sas_split_namespaces; - // 全部尾号 - private static final String NUMBERS_ALL = "all"; - // 没有尾号 - private static final String NUMBERS_NONE = "none"; - private void paserConfigToField(String jsonValue) { JSONObject jsonObject = JSONObject.parseObject(jsonValue); this.uuidEndNum = jsonObject.getString("uuidEndNum"); @@ -50,7 +48,7 @@ private void paserConfigToField(String jsonValue) { */ public boolean aegisUuidMatch(String uuid) { if (uuidEndNum == null || "".equals(uuidEndNum)) { - LOG.warn("SplitFlow aegisUuidMatch,uuidEndNum is null"); + LOGGER.warn("SplitFlow aegisUuidMatch,uuidEndNum is null"); return true; } String numbers = uuidEndNum; @@ -86,7 +84,7 @@ public void setUuidNameJson(String uuidNameJson) { try { this.uuidNameMap = JSONObject.parseObject(uuidNameJson); } catch (Exception e) { - LOG.error("SplitFlow setUuidNameJson error" + uuidNameJson, e); + LOGGER.error("SplitFlow setUuidNameJson error" + uuidNameJson, e); } } @@ -134,7 +132,7 @@ public void setSplitFlowJson(String splitFlowJson) { try { paserConfigToField(splitFlowJson); } catch (Exception e) { - LOG.error("SplitFlow setSplitFlowJson error,splitFlowJson is:" + splitFlowJson, e); + LOGGER.error("SplitFlow setSplitFlowJson error,splitFlowJson is:" + splitFlowJson, e); } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/engine/impl/DefaultRuleEngine.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/engine/impl/DefaultRuleEngine.java index 28948b3d..c5b1c0e1 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/engine/impl/DefaultRuleEngine.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/engine/impl/DefaultRuleEngine.java @@ -17,35 +17,21 @@ package org.apache.rocketmq.streams.filter.engine.impl; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.Map; -import java.util.Set; -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.IMessage; import org.apache.rocketmq.streams.common.monitor.IMonitor; -import org.apache.rocketmq.streams.common.monitor.TopologyFilterMonitor; import org.apache.rocketmq.streams.common.monitor.group.MonitorCommander; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; -import org.apache.rocketmq.streams.common.topology.metric.NotFireReason; -import org.apache.rocketmq.streams.common.utils.TraceUtil; import org.apache.rocketmq.streams.filter.context.RuleContext; import org.apache.rocketmq.streams.filter.engine.IRuleEngine; import org.apache.rocketmq.streams.filter.operator.Rule; import org.apache.rocketmq.streams.filter.operator.action.Action; -import org.apache.rocketmq.streams.filter.operator.expression.Expression; -import org.apache.rocketmq.streams.filter.operator.expression.RelationExpression; -import org.apache.rocketmq.streams.filter.optimization.dependency.CommonExpression; -import org.apache.rocketmq.streams.filter.optimization.dependency.SimplePipelineTree; -import org.apache.rocketmq.streams.filter.optimization.dependency.StateLessDependencyTree; -import org.apache.rocketmq.streams.script.service.IScriptExpression; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class DefaultRuleEngine implements IRuleEngine { - private static final Log LOG = LogFactory.getLog(DefaultRuleEngine.class); - private static final Log RULEENGINE_MESSAGE_LOG = LogFactory.getLog("ruleengine_message"); + private static final Logger LOGGER = LoggerFactory.getLogger(DefaultRuleEngine.class); @SuppressWarnings({"rawtypes", "unchecked"}) @Override @@ -90,11 +76,11 @@ protected List executeRule(AbstractContext context, IMessage message, List } } } catch (Exception e) { - LOG.error("DefaultRuleEngine executeRule error,excuteRules size is " + excuteRules.size() + LOGGER.error("DefaultRuleEngine executeRule error,excuteRules size is " + excuteRules.size() + " ,fireRules size is :" + fireRules.size(), e); } } catch (Exception e) { - LOG.error( + LOGGER.error( "DefaultRuleEngine executeRule error: fireRules size is: " + fireRules.size() + "excuteRules size is : " + excuteRules.size(), e); @@ -116,7 +102,7 @@ public List executeRuleWithoutAction(IMessage message, List rules) { @SuppressWarnings("rawtypes") private void fireAction(IMessage message, AbstractContext context, Rule rule) { if (rule == null) { - LOG.error("DefaultRuleEngine fireAction error: rules is null!"); + LOGGER.error("DefaultRuleEngine fireAction error: rules is null!"); return; } try { @@ -136,11 +122,11 @@ private void fireAction(IMessage message, AbstractContext context, Rule rule) { doAction(message, context, action, rule); } } catch (Exception e) { - LOG.error("DefaultRuleEngine fire atciton error: ruleName is" + rule.getConfigureName(), e); + LOGGER.error("DefaultRuleEngine fire atciton error: ruleName is" + rule.getName(), e); } } else { - Action action = getAction(RuleContext.OBSERVER_NAME,rule); + Action action = getAction(RuleContext.OBSERVER_NAME, rule); if (action == null) { return; } @@ -148,7 +134,7 @@ private void fireAction(IMessage message, AbstractContext context, Rule rule) { } } catch (Exception e) { - LOG.error("DefaultRuleEngine fireAction error: ruleName is" + rule.getConfigureName(), e); + LOGGER.error("DefaultRuleEngine fireAction error: ruleName is" + rule.getName(), e); } } @@ -161,7 +147,7 @@ private void fireAction(IMessage message, AbstractContext context, Rule rule) { */ @SuppressWarnings("rawtypes") protected void doAction(final IMessage message, AbstractContext context, final Action action, final Rule rule) { - action.doMessage(message,context); + action.doMessage(message, context); } public Action getAction(String name, Rule rule) { diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/AbstractExpressionFunction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/AbstractExpressionFunction.java index 0a07fbe1..8471ffc0 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/AbstractExpressionFunction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/AbstractExpressionFunction.java @@ -16,8 +16,6 @@ */ package org.apache.rocketmq.streams.filter.function.expression; -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.IMessage; import org.apache.rocketmq.streams.filter.exception.RegexTimeoutException; @@ -25,17 +23,14 @@ public abstract class AbstractExpressionFunction implements ExpressionFunction { - private static final Log LOG = LogFactory.getLog(AbstractExpressionFunction.class); - private static final Log RULEENGINE_MESSAGE_LOG = LogFactory.getLog("ruleengine_message"); - @Override public Boolean doFunction(IMessage message, AbstractContext context, Expression expression) { try { - Boolean result = doExpressionFunction(message, context,expression); + Boolean result = doExpressionFunction(message, context, expression); return result; } catch (RegexTimeoutException e) { - e.printStackTrace(); - throw e; + e.printStackTrace(); + throw e; } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/CompareFunction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/CompareFunction.java index aa7941a4..31104c34 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/CompareFunction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/CompareFunction.java @@ -16,21 +16,20 @@ */ package org.apache.rocketmq.streams.filter.function.expression; -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.IMessage; import org.apache.rocketmq.streams.common.datatype.DataType; import org.apache.rocketmq.streams.common.utils.DataTypeUtil; import org.apache.rocketmq.streams.common.utils.ReflectUtil; -import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.filter.operator.expression.Expression; import org.apache.rocketmq.streams.filter.operator.var.Var; import org.apache.rocketmq.streams.script.utils.FunctionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public abstract class CompareFunction extends AbstractExpressionFunction { - public static String VAR_PREFIX="##*^%$#@!*";//标识一个表达式的值式一个变量,是一个特殊处理。这个标识会放到值的前面。如变量式uuid,值会变成&&&&##$$%^*uuid - private static final Log LOG = LogFactory.getLog(CompareFunction.class); + private static final Logger LOGGER = LoggerFactory.getLogger(CompareFunction.class); + public static String VAR_PREFIX = "##*^%$#@!*";//标识一个表达式的值式一个变量,是一个特殊处理。这个标识会放到值的前面。如变量式uuid,值会变成&&&&##$$%^*uuid @Override public Boolean doExpressionFunction(IMessage message, AbstractContext context, Expression expression) { @@ -39,12 +38,10 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, E } Object varValue = null; String varName = expression.getVarName(); - Var var =expression.getVar(); + Var var = expression.getVar(); varValue = var.doMessage(message, context); - /** - * 两个数字比较的情况 - */ + //两个数字比较的情况 if ((FunctionUtils.isNumber(varName) || FunctionUtils.isConstant(varName)) && varValue == null) { varValue = varName; } @@ -52,34 +49,34 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, E if (varValue == null || expression.getValue() == null) { return false; } - if(StringUtil.isEmpty(varValue.toString())||StringUtil.isEmpty(expression.getValue().toString())){ + if (expression.getValue() == null) { return false; } Object basicVarValue = expression.getDataType().getData(varValue.toString()); Object basicValue = expression.getDataType().getData(expression.getValue().toString()); - if (varValue == null || expression.getValue() == null) { + if (expression.getValue() == null) { return false; } boolean match = false; if (basicValue == null || basicVarValue == null) { return false; } - DataType dataType=expression.getDataType(); - if(String.class.isInstance(basicValue)&&basicValue.toString().startsWith(VAR_PREFIX)){ - String valueVarName=basicValue.toString().replace(VAR_PREFIX,""); - basicValue=message.getMessageBody().get(valueVarName); - dataType= DataTypeUtil.getDataTypeFromClass(varValue.getClass()); - basicVarValue=dataType.getData(varValue.toString()); + DataType dataType = expression.getDataType(); + if (basicValue instanceof String && basicValue.toString().startsWith(VAR_PREFIX)) { + String valueVarName = basicValue.toString().replace(VAR_PREFIX, ""); + basicValue = message.getMessageBody().get(valueVarName); + dataType = DataTypeUtil.getDataTypeFromClass(varValue.getClass()); + basicVarValue = dataType.getData(varValue.toString()); } - Class varClass = basicVarValue == null ? dataType.getDataClass() : basicVarValue.getClass(); - Class valueClass = basicValue == null ? dataType.getDataClass() : basicValue.getClass(); + Class varClass = basicVarValue == null ? dataType.getDataClass() : basicVarValue.getClass(); + Class valueClass = basicValue == null ? dataType.getDataClass() : basicValue.getClass(); try { match = (Boolean) ReflectUtil.invoke(this, "compare", new Class[] {varClass, valueClass}, new Object[] {basicVarValue, basicValue}); } catch (Exception e) { - LOG.error("CompareFunction doFunction ReflectUtil.invoke error: ", e); + LOGGER.error("CompareFunction doFunction ReflectUtil.invoke error: ", e); } return match; diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/ContainsCaseInsensitiveFunction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/ContainsCaseInsensitiveFunction.java index 335791e8..011e468d 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/ContainsCaseInsensitiveFunction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/ContainsCaseInsensitiveFunction.java @@ -16,8 +16,6 @@ */ package org.apache.rocketmq.streams.filter.function.expression; -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.IMessage; import org.apache.rocketmq.streams.filter.operator.expression.Expression; @@ -25,11 +23,13 @@ import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; import org.apache.rocketmq.streams.script.annotation.FunctionMethodAilas; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @Function public class ContainsCaseInsensitiveFunction extends AbstractExpressionFunction { - private static final Log LOG = LogFactory.getLog(ContainsCaseInsensitiveFunction.class); + private static final Logger LOGGER = LoggerFactory.getLogger(ContainsCaseInsensitiveFunction.class); @Override @FunctionMethod("containsCaseInsensitiveFunction") @@ -47,7 +47,7 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, E } Object varObject = null; Object valueObject = null; - varObject = var.doMessage(message,context); + varObject = var.doMessage(message, context); valueObject = expression.getValue(); if (varObject == null || valueObject == null) { @@ -65,7 +65,7 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, E return false; } } catch (Exception e) { - LOG.error("ContainsCaseInsensitiveFunction error: rule name is: " + expression.getConfigureName(), e); + LOGGER.error("ContainsCaseInsensitiveFunction error: rule name is: " + expression.getName(), e); return false; } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/ContainsFunction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/ContainsFunction.java index ee02affc..0d94088a 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/ContainsFunction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/ContainsFunction.java @@ -41,7 +41,7 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, E } Object varObject = null; Object valueObject = null; - varObject = var.doMessage(message,context); + varObject = var.doMessage(message, context); valueObject = expression.getValue(); if (varObject == null || valueObject == null) { diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/Equals.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/Equals.java index efa44e8d..73548439 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/Equals.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/Equals.java @@ -45,11 +45,34 @@ public static boolean isEqualFunction(String functionName) { return false; } + public static void main(String args[]) { + Equals equals = new Equals(); + + String a = "/usr/sbin/sshd"; + String b = "/usr/sbin/zabbix_agentd"; + + long startTime1 = System.currentTimeMillis(); + for (int i = 0; i < 10000; i++) { + a.equals(b); + } + + long endTime1 = System.currentTimeMillis() - startTime1; + System.out.println("time is:" + endTime1); + + long startTime2 = System.currentTimeMillis(); + for (int i = 0; i < 10000; i++) { + a.trim().equals(b.trim()); + } + long endTime2 = System.currentTimeMillis() - startTime2; + System.out.println("time is:" + endTime2); + + } + @FunctionMethod(value = "=", alias = "==") @FunctionMethodAilas("等于") @Override public Boolean doFunction(IMessage message, AbstractContext context, Expression expression) { - return super.doFunction(message,context,expression); + return super.doFunction(message, context, expression); } public boolean compare(int a, int b) { @@ -139,27 +162,4 @@ public boolean compare(Date a, Date b) { String bd = dateDataType.toDataJson(b); return compare(ad, bd); } - - public static void main(String args[]) { - Equals equals = new Equals(); - - String a = "/usr/sbin/sshd"; - String b = "/usr/sbin/zabbix_agentd"; - - long startTime1 = System.currentTimeMillis(); - for (int i = 0; i < 10000; i++) { - a.equals(b); - } - - long endTime1 = System.currentTimeMillis() - startTime1; - System.out.println("time is:" + endTime1); - - long startTime2 = System.currentTimeMillis(); - for (int i = 0; i < 10000; i++) { - a.trim().equals(b.trim()); - } - long endTime2 = System.currentTimeMillis() - startTime2; - System.out.println("time is:" + endTime2); - - } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/GreaterEquals.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/GreaterEquals.java index 82e445e5..a22fc49e 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/GreaterEquals.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/GreaterEquals.java @@ -35,7 +35,7 @@ public class GreaterEquals extends CompareFunction { @FunctionMethodAilas("大于等于") @Override public Boolean doFunction(IMessage message, AbstractContext context, Expression expression) { - return super.doFunction(message,context,expression); + return super.doFunction(message, context, expression); } public boolean compare(int a, int b) { diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/GreaterThan.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/GreaterThan.java index 63416e98..75a78ed0 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/GreaterThan.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/GreaterThan.java @@ -32,6 +32,11 @@ public class GreaterThan extends CompareFunction { private static final DateDataType dateDataType = new DateDataType(Date.class); + public static void main(String args[]) { + GreaterThan gt = new GreaterThan(); + System.out.println(gt.compare("1201", "1200")); + } + @FunctionMethod(value = ">", alias = ">") @FunctionMethodAilas("大于") @Override @@ -123,9 +128,4 @@ public boolean compare(Date a, Date b) { String bd = dateDataType.toDataJson(b); return compare(ad, bd); } - - public static void main(String args[]) { - GreaterThan gt = new GreaterThan(); - System.out.println(gt.compare("1201", "1200")); - } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/InFunction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/InFunction.java index fa425c34..47ddad9c 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/InFunction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/InFunction.java @@ -38,8 +38,8 @@ public class InFunction extends AbstractExpressionFunction { */ private ICache> cache = new SoftReferenceCache(); - public static boolean matchFunction(String functionName){ - return "in".equals(functionName)||"~in".equals(functionName); + public static boolean matchFunction(String functionName) { + return "in".equals(functionName) || "~in".equals(functionName); } @Override @@ -53,7 +53,7 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, E } Object varObject = null; Object valueObject = null; - varObject = var.doMessage(message,context); + varObject = var.doMessage(message, context); valueObject = expression.getValue(); if (varObject == null || valueObject == null) { diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/InMetaData.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/InMetaData.java index da78f378..18ed4e3e 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/InMetaData.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/InMetaData.java @@ -41,7 +41,7 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, E } Object varObject = null; Object valueObject = null; - varObject = var.doMessage(message,context); + varObject = var.doMessage(message, context); valueObject = expression.getValue(); if (varObject == null || valueObject == null) { diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/IpContainsFunction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/IpContainsFunction.java index 788f8b3a..395da179 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/IpContainsFunction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/IpContainsFunction.java @@ -16,8 +16,6 @@ */ package org.apache.rocketmq.streams.filter.function.expression; -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.IMessage; import org.apache.rocketmq.streams.common.utils.StringUtil; @@ -27,11 +25,13 @@ import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; import org.apache.rocketmq.streams.script.annotation.FunctionMethodAilas; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @Function public class IpContainsFunction extends AbstractExpressionFunction { - private static final Log LOG = LogFactory.getLog(IpContainsFunction.class); + private static final Logger LOGGER = LoggerFactory.getLogger(IpContainsFunction.class); private String ip; private long start; @@ -55,7 +55,7 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, E } Object varObject = null; Object valueObject = null; - varObject = var.doMessage(message,context ); + varObject = var.doMessage(message, context); valueObject = expression.getValue(); if (varObject == null || valueObject == null) { @@ -73,7 +73,7 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, E parseRegexIp(regex); return contains(varString); } catch (Exception e) { - LOG.error("IpContainsFunction doExpressionFunction error", e); + LOGGER.error("IpContainsFunction doExpressionFunction error", e); return false; } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/IsNotNull.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/IsNotNull.java index 9b9f51d1..2d1b4e1c 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/IsNotNull.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/IsNotNull.java @@ -27,8 +27,8 @@ @Function public class IsNotNull extends AbstractExpressionFunction { - public static boolean matchFunction(String functionName){ - return "isNotNull".equals(functionName)||"notNull".equals(functionName); + public static boolean matchFunction(String functionName) { + return "isNotNull".equals(functionName) || "notNull".equals(functionName); } @Override @@ -40,7 +40,7 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, E return false; } Object varObject = null; - varObject = var.doMessage(message,context); + varObject = var.doMessage(message, context); if (varObject == null) { return false; } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/IsNull.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/IsNull.java index c5bbdf7b..368f37a6 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/IsNull.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/IsNull.java @@ -26,9 +26,10 @@ @Function public class IsNull extends AbstractExpressionFunction { - public static boolean matchFunction(String functionName){ - return "isNull".equals(functionName)||"null".equals(functionName); + public static boolean matchFunction(String functionName) { + return "isNull".equals(functionName) || "null".equals(functionName); } + @Override @FunctionMethod(value = "isNull", alias = "null") @FunctionMethodAilas("为空") @@ -38,7 +39,7 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, E return false; } Object varObject = null; - varObject = var.doMessage(message,context); + varObject = var.doMessage(message, context); if (varObject == null) { return true; } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/LessEquals.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/LessEquals.java index d5ef2829..86b9eafa 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/LessEquals.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/LessEquals.java @@ -34,7 +34,7 @@ public class LessEquals extends CompareFunction { @FunctionMethodAilas("小于等于") @Override public Boolean doFunction(IMessage message, AbstractContext context, Expression expression) { - return super.doFunction(message,context,expression); + return super.doFunction(message, context, expression); } public boolean compare(int a, int b) { diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/LessThan.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/LessThan.java index 0bcd61b9..e3c8473f 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/LessThan.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/LessThan.java @@ -35,10 +35,9 @@ public class LessThan extends CompareFunction { @FunctionMethod(value = "<", alias = "<") @FunctionMethodAilas("小于") public Boolean doFunction(IMessage message, AbstractContext context, Expression expression) { - return super.doFunction(message,context,expression); + return super.doFunction(message, context, expression); } - public boolean compare(int a, int b) { return a < b; } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/LikeFunction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/LikeFunction.java index 6052c758..42880ebd 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/LikeFunction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/LikeFunction.java @@ -31,26 +31,65 @@ @Function public class LikeFunction extends AbstractExpressionFunction { + protected static String[] regexSpecialWords = {"\\", "$", "(", ")", "*", "+", ".", "[", "]", "?", "^", "{", "}", "|"}; private transient ICache likeCache = new SoftReferenceCache<>(); - private transient ICache cache = new SoftReferenceCache<>(); public static boolean isLikeFunciton(String functionName) { return "like".equals(functionName); } - private class LikeCache { - public LikeCache(String containStr, String regexStr, boolean isPrefix) { - this.containStr = containStr; - this.regexStr = regexStr; - this.isPrefix = isPrefix; - + /** + * 把like中一定包含的字符串抽取出来,再正则匹配前,先做预处理 + * + * @param likeStr like的语句 + * @return 如果能够抽取,返回抽取的字符串,否则返回null + */ + protected static String parseContainStr(String likeStr) { + String tmp = likeStr; + if (tmp.startsWith("%")) { + tmp = tmp.substring(1); + } + if (tmp.endsWith("%")) { + tmp = tmp.substring(0, tmp.length() - 1); + } + for (String word : regexSpecialWords) { + if (tmp.contains(word)) { + return null; + } } + return null; + } - protected String containStr;//预处理like中一定包含的字符串 - protected String regexStr;//把like转换成regex对应的字符串 - protected boolean isPrefix = false;//是否是前缀匹配 + /** + * regexWord.add("."); regexWord.add("*"); regexWord.add("?"); regexWord.add("{"); regexWord.add("}"); // regexWord.add("\\d"); regexWord.add(")"); // regexWord.add("("); regexWord.add("+"); regexWord.add("\\"); regexWord.add("["); regexWord.add("]"); regexWord.add("^"); regexWord.add("$"); + * + * @param likeStr + * @return + */ + protected static String convertRegex(String likeStr) { + likeStr = likeStr.replace("\\", "\\\\"); + likeStr = likeStr.replace("[!", "[^"); + likeStr = likeStr.replace(".", "\\."); + likeStr = likeStr.replace("*", "\\*"); + likeStr = likeStr.replace("_", "."); + likeStr = likeStr.replace("+", "\\+"); + + if (!likeStr.trim().startsWith("%")) { + likeStr = "^" + likeStr; + } + if (!likeStr.trim().endsWith("%")) { + likeStr = likeStr + "$"; + } + likeStr = likeStr.replace("%", ".*"); + return likeStr; + } + + public static void main(String[] args) { + String valueString = convertRegex("%/php%"); + System.out.println(valueString); + System.out.println(StringUtil.matchRegex("/www/server/php/70/sbin/php-fpm", valueString)); } @Override @@ -63,7 +102,7 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, E } Object varObject = null; Object valueObject = null; - varObject = var.doMessage(message,context); + varObject = var.doMessage(message, context); valueObject = expression.getValue(); if (varObject == null || valueObject == null) { @@ -107,58 +146,17 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, E //return result; } - protected static String[] regexSpecialWords = {"\\", "$", "(", ")", "*", "+", ".", "[", "]", "?", "^", "{", "}", "|"}; - - /** - * 把like中一定包含的字符串抽取出来,再正则匹配前,先做预处理 - * - * @param likeStr like的语句 - * @return 如果能够抽取,返回抽取的字符串,否则返回null - */ - protected static String parseContainStr(String likeStr) { - String tmp = likeStr; - if (tmp.startsWith("%")) { - tmp = tmp.substring(1); - } - if (tmp.endsWith("%")) { - tmp = tmp.substring(0, tmp.length() - 1); - } - for (String word : regexSpecialWords) { - if (tmp.contains(word)) { - return null; - } - } - return null; - } - - /** - * regexWord.add("."); regexWord.add("*"); regexWord.add("?"); regexWord.add("{"); regexWord.add("}"); // regexWord.add("\\d"); regexWord.add(")"); // regexWord.add("("); regexWord.add("+"); regexWord.add("\\"); regexWord.add("["); regexWord.add("]"); regexWord.add("^"); regexWord.add("$"); - * - * @param likeStr - * @return - */ + private class LikeCache { + protected String containStr;//预处理like中一定包含的字符串 + protected String regexStr;//把like转换成regex对应的字符串 + protected boolean isPrefix = false;//是否是前缀匹配 - protected static String convertRegex(String likeStr) { - likeStr = likeStr.replace("\\", "\\\\"); - likeStr = likeStr.replace("[!", "[^"); - likeStr = likeStr.replace(".", "\\."); - likeStr = likeStr.replace("*", "\\*"); - likeStr = likeStr.replace("_", "."); - likeStr = likeStr.replace("+", "\\+"); + public LikeCache(String containStr, String regexStr, boolean isPrefix) { + this.containStr = containStr; + this.regexStr = regexStr; + this.isPrefix = isPrefix; - if (!likeStr.trim().startsWith("%")) { - likeStr = "^" + likeStr; - } - if (!likeStr.trim().endsWith("%")) { - likeStr = likeStr + "$"; } - likeStr = likeStr.replace("%", ".*"); - return likeStr; - } - public static void main(String[] args) { - String valueString = convertRegex("%/php%"); - System.out.println(valueString); - System.out.println(StringUtil.matchRegex("/www/server/php/70/sbin/php-fpm", valueString)); } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotContainsCaseInsensitiveFunction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotContainsCaseInsensitiveFunction.java index 21f93210..20443b1e 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotContainsCaseInsensitiveFunction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotContainsCaseInsensitiveFunction.java @@ -16,8 +16,6 @@ */ package org.apache.rocketmq.streams.filter.function.expression; -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.IMessage; import org.apache.rocketmq.streams.filter.operator.expression.Expression; @@ -25,12 +23,14 @@ import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; import org.apache.rocketmq.streams.script.annotation.FunctionMethodAilas; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @Function public class NotContainsCaseInsensitiveFunction extends AbstractExpressionFunction { - private static final Log LOG = LogFactory.getLog(NotContainsCaseInsensitiveFunction.class); + private static final Logger LOGGER = LoggerFactory.getLogger(NotContainsCaseInsensitiveFunction.class); @Override @FunctionMethod(value = "notContainsCaseInsensitiveFunction", alias = "~notContains") @@ -48,7 +48,7 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, E } Object varObject = null; Object valueObject = null; - varObject = var.doMessage(message,context); + varObject = var.doMessage(message, context); valueObject = expression.getValue(); if (varObject == null) { return true; @@ -67,7 +67,7 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, E return true; } } catch (Exception e) { - LOG.error("ContainsCaseInsensitiveFunction error: rule name is: " + expression.getConfigureName(), e); + LOGGER.error("ContainsCaseInsensitiveFunction error: rule name is: " + expression.getName(), e); return false; } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotContainsFunction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotContainsFunction.java index c5f97cfe..769afacf 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotContainsFunction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotContainsFunction.java @@ -41,7 +41,7 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, E } Object varObject = null; Object valueObject = null; - varObject = var.doMessage(message,context); + varObject = var.doMessage(message, context); valueObject = expression.getValue(); if (varObject == null) { return true; diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotEquals.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotEquals.java index 8c2c4cf5..fe52598c 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotEquals.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotEquals.java @@ -33,11 +33,18 @@ public class NotEquals extends CompareFunction { private final static double MIN_VALUE = 0.000001; private static final DateDataType dateDataType = new DateDataType(Date.class); + public static void main(String[] args) { + Boolean a = true; + Boolean b = false; + NotEquals not = new NotEquals(); + System.out.println(not.compare(a, b)); + } + @FunctionMethod(value = "!=", alias = "<>") @FunctionMethodAilas("不等于") @Override public Boolean doFunction(IMessage message, AbstractContext context, Expression expression) { - return super.doFunction(message,context,expression); + return super.doFunction(message, context, expression); } public boolean compare(int a, int b) { @@ -158,11 +165,4 @@ public boolean compare(Date a, Date b) { String bd = dateDataType.toDataJson(b); return compare(ad, bd); } - - public static void main(String[] args) { - Boolean a = true; - Boolean b = false; - NotEquals not = new NotEquals(); - System.out.println(not.compare(a, b)); - } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotInFunction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotInFunction.java index 96ea1480..c19c9b7d 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotInFunction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotInFunction.java @@ -30,6 +30,6 @@ public class NotInFunction extends InFunction { @FunctionMethod(value = "!in", alias = "~!in") @FunctionMethodAilas("包含") public Boolean doExpressionFunction(IMessage message, AbstractContext context, Expression expression) { - return !super.doExpressionFunction(message,context,expression); + return !super.doExpressionFunction(message, context, expression); } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotLikeFunction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotLikeFunction.java index 5311bb94..171c10f8 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotLikeFunction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotLikeFunction.java @@ -30,7 +30,7 @@ public class NotLikeFunction extends LikeFunction { @FunctionMethod(value = "notLike", alias = "!like") @FunctionMethodAilas("not like") public Boolean doExpressionFunction(IMessage message, AbstractContext context, Expression expression) { - return !super.doExpressionFunction(message,context,expression); + return !super.doExpressionFunction(message, context, expression); } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotRegexCaseInsensitiveFunction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotRegexCaseInsensitiveFunction.java index 85294a38..c7daf724 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotRegexCaseInsensitiveFunction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotRegexCaseInsensitiveFunction.java @@ -31,6 +31,6 @@ public class NotRegexCaseInsensitiveFunction extends RegexCaseInsensitiveFunctio @FunctionMethod(value = "notRegexCaseInsensitive", alias = "~!regex") @FunctionMethodAilas("正则不匹配(忽略大小写)") public Boolean doExpressionFunction(IMessage message, AbstractContext context, Expression expression) { - return !super.doExpressionFunction(message,context,expression); + return !super.doExpressionFunction(message, context, expression); } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotRegexFunction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotRegexFunction.java index 89f1e5dc..708aa5af 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotRegexFunction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/NotRegexFunction.java @@ -31,6 +31,6 @@ public class NotRegexFunction extends RegexFunction { @FunctionMethod(value = "notRegex", alias = "!regex") @FunctionMethodAilas("正则不匹配") public Boolean doExpressionFunction(IMessage message, AbstractContext context, Expression expression) { - return !super.doExpressionFunction(message,context,expression); + return !super.doExpressionFunction(message, context, expression); } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/RegexCaseInsensitiveFunction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/RegexCaseInsensitiveFunction.java index b42ff999..2a7c261a 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/RegexCaseInsensitiveFunction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/RegexCaseInsensitiveFunction.java @@ -31,7 +31,7 @@ public class RegexCaseInsensitiveFunction extends RegexFunction { @FunctionMethod(value = "regexCaseInsensitive", alias = "~regex") @FunctionMethodAilas("正则匹配(忽略大小写)") public Boolean doExpressionFunction(IMessage message, AbstractContext context, Expression expression) { - return super.doExpressionFunction(message,context,expression); + return super.doExpressionFunction(message, context, expression); } @Override diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/RegexFunction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/RegexFunction.java index 2b84b771..2b5f305e 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/RegexFunction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/RegexFunction.java @@ -16,8 +16,6 @@ */ package org.apache.rocketmq.streams.filter.function.expression; -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.IMessage; import org.apache.rocketmq.streams.common.utils.StringUtil; @@ -27,12 +25,14 @@ import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; import org.apache.rocketmq.streams.script.annotation.FunctionMethodAilas; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @Function public class RegexFunction extends AbstractExpressionFunction { - private static final Log LOG = LogFactory.getLog(RegexFunction.class); + private static final Logger LOGGER = LoggerFactory.getLogger(RegexFunction.class); private static final int REGEX_TIME_OUT = -1; public static boolean isRegex(String functionName) { @@ -53,8 +53,7 @@ public static boolean isNotRegex(String functionName) { @Override @FunctionMethod("regex") @FunctionMethodAilas("正则匹配") - public Boolean doExpressionFunction(IMessage message, AbstractContext context, Expression expression) { - + public Boolean doExpressionFunction(IMessage message, AbstractContext context, Expression expression) { Var var = expression.getVar(); if (var == null) { @@ -62,7 +61,7 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, } Object varObject = null; Object valueObject = null; - varObject = var.doMessage(message,context); + varObject = var.doMessage(message, context); valueObject = expression.getValue(); @@ -75,7 +74,6 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, varString = String.valueOf(varObject); regex = String.valueOf(valueObject); - boolean value = false; if (caseInsensitive()) { value = StringUtil.matchRegexCaseInsensitive(varString, regex); @@ -100,7 +98,7 @@ protected boolean doPreProcess(RuleContext context, Expression expression, Strin return result; } catch (Exception e) { - LOG.error("AbstractExpressionFunction doPreProcess error", e); + LOGGER.error("AbstractExpressionFunction doPreProcess error", e); return true; } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/ScriptFunction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/ScriptFunction.java index d35dfdd1..41ea5418 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/ScriptFunction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/ScriptFunction.java @@ -65,7 +65,7 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, E throw new RuntimeException("execute scriptFunction error, expect return boolean value ,real is null"); } if (Boolean.class.isInstance(object)) { - return (Boolean)object; + return (Boolean) object; } throw new RuntimeException("execute scriptFunction error, expect return boolean value ,real is " + object.getClass().getName()); } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/script/CaseFunction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/script/CaseFunction.java index 08ffe23d..86bb203c 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/script/CaseFunction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/script/CaseFunction.java @@ -48,6 +48,9 @@ public Boolean match(IMessage message, FunctionContext context, String value) { if (value == null) { value = tmp; } + if (value == null) { + return false; + } if (value.length() <= 5) { String lowValue = value.trim().toLowerCase(); if (lowValue.equals("true") || lowValue.equals("false")) { diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/script/CaseDependentParser.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/script/GroupScriptOptimization.java similarity index 51% rename from rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/script/CaseDependentParser.java rename to rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/script/GroupScriptOptimization.java index eb3d2524..561a83ca 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/script/CaseDependentParser.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/script/GroupScriptOptimization.java @@ -21,13 +21,16 @@ import org.apache.rocketmq.streams.filter.builder.ExpressionBuilder; import org.apache.rocketmq.streams.filter.operator.Rule; import org.apache.rocketmq.streams.filter.operator.RuleExpression; -import org.apache.rocketmq.streams.script.operator.expression.ICaseDependentParser; +import org.apache.rocketmq.streams.script.operator.expression.GroupScriptExpression; +import org.apache.rocketmq.streams.script.operator.expression.IGroupScriptOptimization; +import org.apache.rocketmq.streams.script.operator.expression.ScriptParameter; import org.apache.rocketmq.streams.script.service.IScriptExpression; import org.apache.rocketmq.streams.script.service.IScriptParamter; import org.apache.rocketmq.streams.script.utils.FunctionUtils; -@AutoService(ICaseDependentParser.class) -public class CaseDependentParser implements ICaseDependentParser { +@AutoService(IGroupScriptOptimization.class) +public class GroupScriptOptimization implements IGroupScriptOptimization { + @Override public Set getDependentFields(IScriptExpression scriptExpression) { if (RuleExpression.class.isInstance(scriptExpression)) { return ((RuleExpression) scriptExpression).getRule().getDependentFields(); @@ -43,4 +46,30 @@ public class CaseDependentParser implements ICaseDependentParser { } return CaseFunction.isCaseFunction(scriptExpression.getFunctionName()); } + + @Override public void compile(GroupScriptExpression groupScriptExpression) { + IScriptExpression ifScriptExpression = groupScriptExpression.getIfExpresssion(); + if (CaseFunction.isCaseFunction(ifScriptExpression.getFunctionName())) { + if (ifScriptExpression.getScriptParamters() != null && ifScriptExpression.getScriptParamters().size() == 1) { + IScriptParamter scriptParamter = (IScriptParamter) ifScriptExpression.getScriptParamters().get(0); + if (ScriptParameter.class.isInstance(scriptParamter)) { + ScriptParameter valueParameter = (ScriptParameter) scriptParamter; + if (valueParameter.getFunctionName() == null && valueParameter.getRigthVarName() == null && valueParameter.getNewFieldNames() == null) { + if (valueParameter.getLeftVarName().startsWith("(") && valueParameter.getLeftVarName().endsWith(")")) { + String expressionStr = FunctionUtils.getConstant(valueParameter.getLeftVarName()); + Rule rule = ExpressionBuilder.createRule("tmp", "tmp", expressionStr); + rule.optimize(); + groupScriptExpression.setRule(rule); + } else { + groupScriptExpression.setBoolVar(valueParameter.getLeftVarName()); + } + } + } + } + for (GroupScriptExpression group : groupScriptExpression.getElseIfExpressions()) { + compile(group); + } + + } + } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/monitor/rule/MessageMonitor.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/monitor/rule/MessageMonitor.java index 2da52b07..0016d107 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/monitor/rule/MessageMonitor.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/monitor/rule/MessageMonitor.java @@ -82,7 +82,7 @@ public Monitor getRuleMonitor(String ruleName) { if (monitor == null) { return null; } - return (RuleMonitor)monitor; + return (RuleMonitor) monitor; } @Override @@ -98,7 +98,7 @@ protected String resultToString() { } else { sb.append(","); } - sb.append(rule.getConfigureName()); + sb.append(rule.getName()); } return sb.toString(); } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/FilterOperator.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/FilterOperator.java index 792f93b2..980d3ee1 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/FilterOperator.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/FilterOperator.java @@ -22,12 +22,11 @@ public FilterOperator(String namespace, String name, String expression) { super(namespace, name, expression); } - public FilterOperator() {} + public FilterOperator() { + } public FilterOperator(String expression) { super(expression); } - - } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/Rule.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/Rule.java index 5ae02236..9dec0254 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/Rule.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/Rule.java @@ -19,30 +19,23 @@ import com.alibaba.fastjson.JSONArray; import com.alibaba.fastjson.JSONObject; import java.util.ArrayList; -import java.util.Collection; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Set; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.channel.sink.ISink; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; import org.apache.rocketmq.streams.common.configurable.IConfigurable; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; 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.metadata.MetaData; import org.apache.rocketmq.streams.common.metadata.MetaDataField; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; -import org.apache.rocketmq.streams.common.topology.ChainStage; -import org.apache.rocketmq.streams.common.topology.builder.IStageBuilder; +import org.apache.rocketmq.streams.common.topology.IStageBuilder; import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; -import org.apache.rocketmq.streams.common.topology.metric.NotFireReason; +import org.apache.rocketmq.streams.common.topology.model.AbstractChainStage; import org.apache.rocketmq.streams.common.topology.model.AbstractRule; import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; import org.apache.rocketmq.streams.common.utils.TraceUtil; @@ -55,16 +48,17 @@ import org.apache.rocketmq.streams.filter.operator.expression.GroupExpression; import org.apache.rocketmq.streams.filter.operator.expression.RelationExpression; import org.apache.rocketmq.streams.filter.operator.var.ContextVar; -import org.apache.rocketmq.streams.filter.operator.var.InnerVar; import org.apache.rocketmq.streams.filter.operator.var.Var; import org.apache.rocketmq.streams.filter.optimization.ExpressionOptimization; -import org.apache.rocketmq.streams.filter.optimization.dependency.CommonExpression; -import org.apache.rocketmq.streams.filter.optimization.dependency.StateLessDependencyTree; -import org.apache.rocketmq.streams.script.service.IScriptExpression; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class Rule extends AbstractRule implements IAfterConfigurableRefreshListener, - IStageBuilder { - private static final Log LOG = LogFactory.getLog(Rule.class); +public class Rule extends AbstractRule implements + IStageBuilder> { + public static final String FIRE_RULES = "fireRules"; + private static final Logger LOGGER = LoggerFactory.getLogger(Rule.class); + private transient static FilterComponent filterComponent = FilterComponent.getInstance(); + protected transient Expression rootExpression; private transient volatile Map varMap = new HashMap<>(); private transient volatile Map expressionMap = new HashMap<>(); @Deprecated @@ -72,8 +66,6 @@ public class Rule extends AbstractRule implements IAfterConfigurableRefreshListe private transient Map metaDataMap = new HashMap<>(); private transient volatile Map dataSourceMap = new HashMap<>(); private String expressionStr;//表达式 - protected transient Expression rootExpression; - /** * 如果已经完成varmap和expressionmap的初始化,主要是用于兼容老版本规则数据,新规则可以忽略这个字段,值设置为true */ @@ -95,6 +87,8 @@ public Rule(String namespace, String name, String expression) { this.setExpressionStr(rule.getExpressionStr()); this.setVarNames(rule.getVarNames()); this.setExpressionName(rule.getExpressionName()); + this.rootExpression = this.expressionMap.get(getExpressionName()); + initElements(); } public Rule(String expression) { @@ -105,7 +99,7 @@ public Rule copy() { Rule rule = new Rule(); rule.setNameSpace(getNameSpace()); rule.setType(getType()); - rule.setConfigureName(getConfigureName()); + rule.setName(getName()); rule.varMap = varMap; rule.expressionMap = expressionMap; rule.actionMap = actionMap; @@ -120,29 +114,12 @@ public Rule copy() { rule.setRuleStatus(ruleStatus); rule.setRuleTitle(ruleTitle); rule.setRuleStatus(ruleStatus); - rule.setConfigurableService(configurableService); - rule.setPrivateDatas(privateDatas); rule.rootExpression = rootExpression; return rule; } - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - - if (isFinishVarAndExpression == false) { - this.dataSourceMap = configurableService.queryConfigurableMapByType(ISink.TYPE); - - } - initVar(configurableService); - initExpression(configurableService); - initAction(configurableService); - initMetaData(configurableService); - - initElements(); - } - public void initElements() { - this.rootExpression = this.expressionMap.get(getExpressionName()); + for (Expression expression : this.expressionMap.values()) { if (RelationExpression.class.isInstance(expression)) { RelationExpression relationExpression = (RelationExpression) expression; @@ -159,42 +136,8 @@ public void initElements() { } public void addAction(SinkAction action) { - actionMap.put(action.getConfigureName(), action); - this.getActionNames().add(action.getConfigureName()); - } - - private void initVar(IConfigurableService configurableService) { - Map varMap = new HashMap<>(); - if (isFinishVarAndExpression == false) { - varMap = configurableService.queryConfigurableMapByType(Var.TYPE); - } - if (expressionMap != null) { - Iterator> it = expressionMap.entrySet().iterator(); - while (it.hasNext()) { - Entry entry = it.next(); - Expression expression = entry.getValue(); - if (RelationExpression.class.isInstance(expression)) { - continue; - } - String varName = expression.getVarName(); - Var var = varMap.get(varName); - if (var == null) { - ContextVar contextVar = new ContextVar(); - contextVar.setNameSpace(expression.getNameSpace()); - contextVar.setConfigureName(varName); - contextVar.setVarName(varName); - contextVar.setFieldName(varName); - varMap.put(varName, contextVar); - var = contextVar; - } - this.varMap.put(varName, var); - } - } - InnerVar innerVar = new InnerVar(); - innerVar.setNameSpace(getNameSpace()); - innerVar.setConfigureName(innerVar.getClass().getSimpleName()); - innerVar.setVarName(InnerVar.ORIG_MESSAGE); - this.varMap.put(innerVar.getVarName(), innerVar); + actionMap.put(action.getName(), action); + this.getActionNames().add(action.getName()); } @Override @@ -203,15 +146,6 @@ public Set getDependentFields() { return expression.getDependentFields(expressionMap); } - protected void initExpression(IConfigurableService configurableService) { - Map expressionMap = new HashMap<>(); - if (isFinishVarAndExpression == false) { - expressionMap = configurableService.queryConfigurableMapByType(Expression.TYPE); - } - String expressionName = getExpressionName(); - fetchExpression(expressionName, expressionMap); - } - /** * 递归获取所有依赖的expression * @@ -223,7 +157,7 @@ protected void fetchExpression(String expressionName, Map ex if (expression == null) { return; } - this.expressionMap.put(expression.getConfigureName(), expression); + this.expressionMap.put(expression.getName(), expression); if (!RelationExpression.class.isInstance(expression)) { return; } @@ -236,59 +170,26 @@ protected void fetchExpression(String expressionName, Map ex } } - protected void initMetaData(IConfigurableService configurableService) { - Map metaDataMap = new HashMap<>(); - this.metaDataMap = configurableService.queryConfigurableMapByType(MetaData.TYPE); - } - protected Var createVar(String varName) { ContextVar contextVar = new ContextVar(); contextVar.setNameSpace(getNameSpace()); - contextVar.setConfigureName(varName); + contextVar.setName(varName); contextVar.setVarName(varName); contextVar.setFieldName(varName); return contextVar; } - /** - * 赋值action给rule - * - * @param configurableService - */ - protected void initAction(IConfigurableService configurableService) { - Map actionMap = new HashMap<>(); - actionMap = configurableService.queryConfigurableMapByType(Action.TYPE); - if (actionNames != null) { - for (String actionName : actionNames) { - Action action = actionMap.get(actionName); - if (action != null) { - this.actionMap.put(actionName, action); - } - } - } - } - public void putConfigurableMap(IConfigurable configurable, String type) { if (Var.TYPE.equals(type)) { - varMap.put(configurable.getConfigureName(), (Var) configurable); + varMap.put(configurable.getName(), (Var) configurable); } else if (Expression.TYPE.equals(type)) { - expressionMap.put(configurable.getConfigureName(), (Expression) configurable); + expressionMap.put(configurable.getName(), (Expression) configurable); } else if (Action.TYPE.equals(type)) { - actionMap.put(configurable.getConfigureName(), (Action) configurable); + actionMap.put(configurable.getName(), (Action) configurable); } else if (MetaData.TYPE.equals(type)) { - metaDataMap.put(configurable.getConfigureName(), (MetaData) configurable); + metaDataMap.put(configurable.getName(), (MetaData) configurable); } else if (ISink.TYPE.equals(type)) { - dataSourceMap.put(configurable.getConfigureName(), (JDBCDriver) configurable); - } - } - - private void insertOrUpdate(IConfigurableService ruleEngineConfigurableService, - Collection configurables) { - if (configurables == null) { - return; - } - for (IConfigurable configurable : configurables) { - ruleEngineConfigurableService.insert(configurable); + dataSourceMap.put(configurable.getName(), (JDBCDriver) configurable); } } @@ -352,64 +253,15 @@ public void setDataSourceMap(Map dataSourceMap) { this.dataSourceMap = dataSourceMap; } - private transient static FilterComponent filterComponent = FilterComponent.getInstance(); - @Override public Boolean doMessage(IMessage message, AbstractContext context) { boolean isTrace = TraceUtil.hit(message.getHeader().getTraceId()); context.setNotFireExpressionMonitor(new ArrayList<>()); boolean isFireRule = processExpress(message, context, isTrace); - - if (isFireRule == false && isTrace) { - NotFireReason notFireReason= context.getNotFireReason(); - if(notFireReason!=null){ - ChainPipeline chainPipeline=notFireReason.getPipeline(); - StateLessDependencyTree stateLessDependencyTree=new StateLessDependencyTree(chainPipeline); - List commonExpressions=stateLessDependencyTree.parseTopology(chainPipeline); - Map> filterFieldName2ETLScriptList=new HashMap<>(); - Map filterFieldName2OriFieldName=new HashMap<>(); - for(CommonExpression commonExpression:commonExpressions){ - String filterFieldName=commonExpression.getVarName(); - String origFieldName=commonExpression.getSourceVarName(); - filterFieldName2OriFieldName.put(filterFieldName,origFieldName); - List etlScript=filterFieldName2ETLScriptList.get(filterFieldName); - if(etlScript==null){ - etlScript=new ArrayList<>(); - filterFieldName2ETLScriptList.put(filterFieldName,etlScript); - } - for(IScriptExpression scriptExpression:commonExpression.getScriptExpressions()){ - if(!etlScript.contains(scriptExpression.toString())){ - etlScript.add(scriptExpression.toString()); - } - } - } - List filterFieldNames=new ArrayList<>(); - List expressions=new ArrayList<>(); - List expressionNames=context.getNotFireExpressionMonitor(); - for(String expressionName:expressionNames){ - Expression expression=this.getExpressionMap().get(expressionName); - if(expression==null){ - expressions.add(expressionName); - }else if(RelationExpression.class.isInstance(expression)){ - expressions.add(expression.toExpressionString(this.getExpressionMap())); - filterFieldNames.addAll(expression.getDependentFields(this.expressionMap)); - }else { - filterFieldNames.add(expression.getVarName()); - expressions.add(expression.toExpressionString(this.getExpressionMap())); - } - } - notFireReason.analysis(message,filterFieldName2ETLScriptList,filterFieldName2OriFieldName,expressions,filterFieldNames); - context.setNotFireReason(notFireReason); - } - - } - return isFireRule; } - public static final String FIRE_RULES = "fireRules"; - public boolean execute(JSONObject msg) { Message message = new Message(msg); AbstractContext context = new Context(message); @@ -422,7 +274,7 @@ private JSONArray createJsonArray(List fireRules) { return jsonArray; } for (Rule rule : fireRules) { - jsonArray.add(rule.getNameSpace() + ":" + rule.getConfigureName()); + jsonArray.add(rule.getNameSpace() + ":" + rule.getName()); } return jsonArray; } @@ -476,12 +328,12 @@ public void setExpressionStr(String expressionStr) { } @Override - public ChainStage createStageChain(PipelineBuilder pipelineBuilder) { + public AbstractChainStage createStageChain(PipelineBuilder pipelineBuilder) { FilterChainStage filterChainStage = new FilterChainStage(); pipelineBuilder.addConfigurables(this); filterChainStage.setRule(this); filterChainStage.setEntityName("filter"); - filterChainStage.setLabel(this.getConfigureName()); + filterChainStage.setLabel(this.getName()); return filterChainStage; } @@ -506,10 +358,10 @@ public void addConfigurables(PipelineBuilder pipelineBuilder) { */ public void optimize() { Expression root = createOptimizationRule(); - if (!RelationExpression.class.isInstance(root)) { - return; - } - groupByChildrenExpression((RelationExpression) root); +// if (!RelationExpression.class.isInstance(root)) { +// return; +// } +// groupByChildrenExpression((RelationExpression) root); } /** @@ -558,8 +410,8 @@ protected void groupByChildrenExpression(RelationExpression root) { if (groupExpression.size() < 2) { newExpressionNames.addAll(groupExpression.getAllExpressionNames()); } else { - expressionMap.put(groupExpression.getConfigureName(), groupExpression); - newExpressionNames.add(groupExpression.getConfigureName()); + expressionMap.put(groupExpression.getName(), groupExpression); + newExpressionNames.add(groupExpression.getName()); } } root.setValue(newExpressionNames); @@ -590,7 +442,7 @@ protected Expression createOptimizationRule() { expressions.clear(); relationExpressions.clear(); for (Expression express : list) { - expressionMap.put(express.getConfigureName(), express); + expressionMap.put(express.getName(), express); if (RelationExpression.class.isInstance(express)) { relationExpressions.add((RelationExpression) express); } else { @@ -619,15 +471,15 @@ private boolean processExpress(IMessage message, AbstractContext context, boolea } boolean match = expression.doMessage(message, context); - if(!RelationExpression.class.isInstance(expression)){ - RuleContext.addNotFireExpressionMonitor(expression,context); + if (!RelationExpression.class.isInstance(expression)) { + RuleContext.addNotFireExpressionMonitor(expression, context); } if (!match) { return false; } } catch (Exception e) { e.printStackTrace(); - LOG.error("DefaultRuleEngine processExpress error,rule is: " + getConfigureName(), e); + LOGGER.error("DefaultRuleEngine processExpress error,rule is: " + getName(), e); return false; } return true; diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/action/Action.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/action/Action.java index 8cd0402d..8c3ab90f 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/action/Action.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/action/Action.java @@ -30,6 +30,7 @@ public abstract class Action extends BasedConfigurable implements IConfigurab private transient Map metaDataMap = new HashMap<>(); private transient volatile Map dataSourceMap = new HashMap<>(); private transient volatile Map varMap = new HashMap<>(); + public Action() { setType(TYPE); } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/action/IConfigurableAction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/action/IConfigurableAction.java index 581b4d69..18e09817 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/action/IConfigurableAction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/action/IConfigurableAction.java @@ -19,7 +19,6 @@ import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; -public interface IConfigurableAction extends IStreamOperator { - +public interface IConfigurableAction extends IStreamOperator { } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/action/impl/SinkAction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/action/impl/SinkAction.java index 88536c1e..7b5d20cd 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/action/impl/SinkAction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/action/impl/SinkAction.java @@ -16,27 +16,21 @@ */ package org.apache.rocketmq.streams.filter.operator.action.impl; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.channel.IChannel; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.configurable.annotation.ConfigurableReference; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.filter.operator.action.Action; -public class SinkAction extends Action implements IAfterConfigurableRefreshListener { +public class SinkAction extends Action { - private static final Log LOG = LogFactory.getLog(SinkAction.class); - protected String channelName; - protected transient IChannel channel; + @ConfigurableReference protected IChannel channel; public SinkAction() { setType(Action.TYPE); } public SinkAction setChannel(IChannel channel) { - this.channelName = channel.getConfigureName(); this.channel = channel; return this; } @@ -47,18 +41,4 @@ public Boolean doMessage(IMessage message, AbstractContext context) { return true; } - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - IChannel channel = configurableService.queryConfigurable(IChannel.TYPE, channelName); - channel.openAutoFlush(); - this.channel = channel; - } - - public String getChannelName() { - return channelName; - } - - public void setChannelName(String channelName) { - this.channelName = channelName; - } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/Expression.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/Expression.java index 58cbf577..e441866e 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/Expression.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/Expression.java @@ -22,13 +22,9 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.component.ComponentCreator; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; import org.apache.rocketmq.streams.common.configurable.IConfigurable; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.datatype.DataType; @@ -57,67 +53,62 @@ import org.apache.rocketmq.streams.script.ScriptComponent; import org.apache.rocketmq.streams.script.function.model.FunctionConfigure; import org.apache.rocketmq.streams.script.utils.FunctionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class Expression extends BasedConfigurable - implements IConfigurable, IConfigurableAction, Serializable, IStreamOperator { +public class Expression extends BasedConfigurable implements IConfigurable, IConfigurableAction, Serializable, IStreamOperator { public static final String TYPE = "express"; private static final long serialVersionUID = 4610495074511059465L; - private static final Log LOG = LogFactory.getLog(Expression.class); + private static final Logger LOGGER = LoggerFactory.getLogger(Expression.class); private static final String FIELD_COMPARE = "field"; private static final String AES_KEY = "baicheng.cbc"; + protected static RegexFunction regexFunction = new RegexFunction(); + protected static LikeFunction likeFunction = new LikeFunction(); + protected static InFunction inFunction = new InFunction(); + protected static IsNotNull isNotNull = new IsNotNull(); + protected static IsNull isNull = new IsNull(); + protected static transient Map cache = new HashMap<>(); + protected transient HomologousVar homologousVar; + protected T value; + protected transient ExpressionFunction expressionFunction; + protected transient Var var; private String varName; private String functionName; - protected transient HomologousVar homologousVar; @SuppressWarnings("rawtypes") private DataType dataType = new StringDataType(String.class); - protected T value; - - public Expression copy() { - Expression expression = new Expression(); - copy(expression); - return expression; - } - - public void copy(Expression expression) { - expression.setNameSpace(getNameSpace()); - expression.setType(getType()); - expression.setConfigureName(getConfigureName()); - expression.setVarName(varName); - expression.setFunctionName(functionName); - expression.setDataType(dataType); - expression.setValue(value); - expression.setPrivateDatas(privateDatas); - expression.setAesFlag(aesFlag); - } - // 前端处理使用 private String dataTypestr; - // 表达式中的关键词 正则之前进行字符串判断 private String keyword; - // 表达式的值是否加密 0未加密 1加密 private int aesFlag = 0; - /** * 表达式是否是字段之间的比较,dataTypestr为field时为true */ private boolean fieldFlag = false; private volatile boolean initFlag = false; - - protected transient ExpressionFunction expressionFunction; - protected transient Var var; + private transient LikeRegex likeRegex; public Expression() { setType(TYPE); } - protected static RegexFunction regexFunction = new RegexFunction(); - protected static LikeFunction likeFunction = new LikeFunction(); - protected static InFunction inFunction = new InFunction(); - protected static IsNotNull isNotNull = new IsNotNull(); - protected static IsNull isNull = new IsNull(); - protected static transient Map cache = new HashMap<>(); + public Expression copy() { + Expression expression = new Expression(); + copy(expression); + return expression; + } + + public void copy(Expression expression) { + expression.setNameSpace(getNameSpace()); + expression.setType(getType()); + expression.setName(getName()); + expression.setVarName(varName); + expression.setFunctionName(functionName); + expression.setDataType(dataType); + expression.setValue(value); + expression.setAesFlag(aesFlag); + } @Override public Boolean doMessage(IMessage message, AbstractContext context) { @@ -138,21 +129,19 @@ public Boolean doMessage(IMessage message, AbstractContext context) { isMatch = executeFunctionDirectly(message, context); long cost = System.currentTimeMillis() - startTime; long timeout = 10; - if (ComponentCreator.getProperties().getProperty(ConfigureFileKey.MONITOR_SLOW_TIMEOUT) != null) { - timeout = Long.valueOf(ComponentCreator.getProperties().getProperty(ConfigureFileKey.MONITOR_SLOW_TIMEOUT)); + if (getConfiguration() != null && getConfiguration().getProperty(ConfigurationKey.MONITOR_SLOW_TIMEOUT) != null) { + timeout = Long.parseLong(getConfiguration().getProperty(ConfigurationKey.MONITOR_SLOW_TIMEOUT)); } if (cost > timeout) { - LOG.warn("SLOW-" + cost + "----" + this.toString() + PrintUtil.LINE + "the var value is " + message.getMessageBody().getString(varName)); + LOGGER.debug("SLOW-" + cost + "----" + this.toString() + PrintUtil.LINE + "the var value is " + message.getMessageBody().getString(varName)); } return isMatch; } catch (Exception e) { - LOG.error("SLOW-" + this.toString() + PrintUtil.LINE + "the var value is " + message.getMessageBody().getString(varName), e); + LOGGER.error("SLOW-" + this.toString() + PrintUtil.LINE + "the var value is " + message.getMessageBody().getString(varName), e); throw e; } } - private transient LikeRegex likeRegex; - @SuppressWarnings("unchecked") protected Boolean executeFunctionDirectly(IMessage message, AbstractContext context) { @@ -194,9 +183,9 @@ protected Boolean executeFunctionDirectly(IMessage message, AbstractContext cont } result = function.doFunction(message, context, this); } catch (Exception e) { - LOG.error( - "Expression doAction function.doFunction error,rule is: " + getConfigureName() + " ,express is:" - + getConfigureName(), e); + LOGGER.error( + "Expression doAction function.doFunction error,rule is: " + getName() + " ,express is:" + + getName(), e); throw e; } return result; @@ -208,12 +197,12 @@ public String toJson() { String valueStr = String.valueOf(value); try { // 适配fastjson的bug,parseArray时会对"-"进行分割 - if (ListDataType.class.isInstance(dataType) && valueStr.contains("-") && valueStr.contains(",")) { + if (dataType instanceof ListDataType && valueStr.contains("-") && valueStr.contains(",")) { this.value = (T) value; } else { // value很多时候会是String类型,这样dataType.toDataJson会报错,所以先转为dataType类型, // list类型JsonArray.tojson会报错,不做处理([pfile_path_rule,proc_vul_exploit_rce_rule]) - if (!ListDataType.class.isInstance(dataType)) { + if (!(dataType instanceof ListDataType)) { this.value = (T) dataType.getData(valueStr); } } @@ -221,7 +210,7 @@ public String toJson() { e.printStackTrace(); } JSONObject jsonObject = new JSONObject(); - jsonObject.put(IConfigurableService.CLASS_NAME, this.getClass().getName()); + jsonObject.put(IConfigurable.CLASS_NAME, this.getClass().getName()); jsonObject.put("varName", varName); jsonObject.put("functionName", functionName); @@ -231,7 +220,7 @@ public String toJson() { // 如果是String,且不是字段比较,则加密 if (!fieldFlag) { - if (StringDataType.class.isInstance(dataType)) { + if (dataType instanceof StringDataType) { try { jsonObject.put("value", AESUtil.aesEncrypt(valueStr, AES_KEY)); jsonObject.put("aesFlag", 1); @@ -275,7 +264,7 @@ public void toObject(String jsonString) { } if (!fieldFlag) { - if (StringDataType.class.isInstance(dataType)) { + if (dataType instanceof StringDataType) { // value经过加密 需要解密 if (aesFlag == 1) { try { @@ -361,7 +350,7 @@ public boolean init() { */ public boolean supportQuickMatch(Expression expression, RuleContext context, Rule rule) { String varName = expression.getVarName(); - Var var = context.getVar(rule.getConfigureName(), varName); + Var var = context.getVar(rule.getName(), varName); if (var == null || var.canLazyLoad()) { return false; } @@ -374,10 +363,7 @@ public boolean supportQuickMatch(Expression expression, RuleContext context, Rul * @return */ public boolean volidate() { - if (StringUtil.isEmpty(varName) || StringUtil.isEmpty(functionName) || dataType == null) { - return false; - } - return true; + return !StringUtil.isEmpty(varName) && !StringUtil.isEmpty(functionName) && dataType != null; } public String getDataTypestr() { @@ -390,8 +376,7 @@ public void setDataTypestr(String dataTypestr) { fieldFlag = true; } - DataType dt = MetaDataField.getDataTypeByStr(dataTypestr); - this.dataType = dt; + this.dataType = MetaDataField.getDataTypeByStr(dataTypestr); } public String getKeyword() { @@ -413,19 +398,19 @@ public void setAesFlag(int aesFlag) { this.aesFlag = aesFlag; } + public String toExpressionString(Map name2Expression, + String... expressionNamePrefixs) { + return toExpressionString(name2Expression, 0, expressionNamePrefixs); + } + /** * 返回expressCode * * @return */ @SuppressWarnings("rawtypes") - public String toExpressionString(Map name2Expression, + public String toExpressionString(Map name2Expression, int blackCount, String... expressionNamePrefixs) { - return toString(); - } - - @Override - public String toString() { String dataType = null; String result = "(" + varName + "," + getFunctionName() + ","; if (!getDataType().matchClass(String.class)) { @@ -433,19 +418,36 @@ public String toString() { result += dataType + ","; } String value = getDataType().toDataJson(getValue()); - if (getDataType().matchClass(String.class) && needContants(value)) { + if (getDataType().matchClass(String.class) && needContains(value)) { value = "'" + value + "'"; } result += value + ")"; +// for(int i=0;i expressionPerformances = new ArrayList<>(); - protected transient volatile List expressionNames = new ArrayList<>(); - protected transient Map expressionStatisticMap = new HashMap<>(); - protected transient long lastTime = System.currentTimeMillis();//最后一次的优化时间 - protected transient final List values; - - private static ScheduledExecutorService scheduledExecutorService = new ScheduledThreadPoolExecutor(10, new BasicThreadFactory.Builder().namingPattern("ExpressionPerformance-Performance-%d").build()); - static { - scheduledExecutorService.scheduleWithFixedDelay(() -> { + ScheduleFactory.getInstance().execute(ExpressionPerformance.class.getName() + "-schedule", () -> { for (ExpressionPerformance expressionPerformance : expressionPerformances) { expressionPerformance.run(); } }, 10, 3, TimeUnit.SECONDS); } + protected transient final List values; + protected transient volatile List expressionNames = new ArrayList<>(); + protected transient Map expressionStatisticMap = new HashMap<>(); + protected transient long lastTime = System.currentTimeMillis();//最后一次的优化时间 + + public ExpressionPerformance(List values) { + this.expressionNames = values; + this.values = values; + for (String name : expressionNames) { + ExpressionStatistic expressionStatistic = new ExpressionStatistic(); + expressionStatistic.name = name; + expressionStatistic.count.set(0); + expressionStatisticMap.put(name, expressionStatistic); + } + expressionPerformances.add(this); + } + @Override public void run() { List newExpressionNames = new ArrayList<>(); @@ -65,23 +73,6 @@ public int compare(ExpressionStatistic o1, ExpressionStatistic o2) { this.lastTime = System.currentTimeMillis(); } - protected static class ExpressionStatistic { - protected String name; - protected AtomicInteger count = new AtomicInteger(0); - } - - public ExpressionPerformance(List values) { - this.expressionNames = values; - this.values = values; - for (String name : expressionNames) { - ExpressionStatistic expressionStatistic = new ExpressionStatistic(); - expressionStatistic.name = name; - expressionStatistic.count.set(0); - expressionStatisticMap.put(name, expressionStatistic); - } - expressionPerformances.add(this); - } - public Boolean optimizate(String expressionName, Boolean value) { ExpressionStatistic expressionStatistic = expressionStatisticMap.get(expressionName); expressionStatistic.count.incrementAndGet(); @@ -94,4 +85,9 @@ public Iterator iterator() { } return expressionNames.iterator();//expressionNames 会做优化,会给快速失效的表达式,加权中 } + + protected static class ExpressionStatistic { + protected String name; + protected AtomicInteger count = new AtomicInteger(0); + } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/ExpressionRelationParser.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/ExpressionRelationParser.java index 506a88e9..56b327e4 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/ExpressionRelationParser.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/ExpressionRelationParser.java @@ -48,7 +48,7 @@ public static RelationExpression createRelations(String namespace, String ruleNa } String expression = str.substring(startIndex, endIndex); RelationExpression relationExpression = createMixRelation(namespace, ruleName, expression, groupList); - str = str.replace("(" + expression + ")", relationExpression.getConfigureName()); + str = str.replace("(" + expression + ")", relationExpression.getName()); return createRelations(namespace, ruleName, str, groupList); } @@ -69,7 +69,7 @@ private static RelationExpression createMixRelation(String namespace, String rul String sign = AND; if (value.indexOf(sign) != -1) { RelationExpression relationExpression = createSignleRelation(namespace, ruleName, value, sign, groupList); - str = str.replace(value, relationExpression.getConfigureName()); + str = str.replace(value, relationExpression.getName()); } } str = str.replace(" ", ""); @@ -91,7 +91,7 @@ private static RelationExpression createSignleRelation(String namespace, String relationExpression.setNameSpace(namespace); relationExpression.setType(Expression.TYPE); relationExpression.setValue(new ArrayList()); - relationExpression.setConfigureName(expressionName); + relationExpression.setName(expressionName); relationExpression.setRelation(OR.equals(sign) ? "or" : "and"); String[] values = str.split("\\" + sign); diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/GroupExpression.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/GroupExpression.java index 5c0b9531..074d9408 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/GroupExpression.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/GroupExpression.java @@ -29,7 +29,6 @@ import org.apache.rocketmq.streams.common.cache.compress.BitSetCache; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.model.NameCreator; import org.apache.rocketmq.streams.common.model.NameCreatorContext; import org.apache.rocketmq.streams.common.optimization.RegexEngine; import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintCache; @@ -52,18 +51,29 @@ public class GroupExpression extends Expression> { protected List notRegexExpression = new ArrayList<>(); protected AtomicBoolean hasCompile = new AtomicBoolean(false); protected transient Var var; + protected transient String fingerpringtNamespace; public GroupExpression(Rule rule, Var var, boolean isOrRelation) { this.rule = rule; this.var = var; this.varName = var.getVarName(); this.isOrRelation = isOrRelation; - this.setConfigureName(NameCreatorContext.get().createNewName("expression.group")); + this.setName(NameCreatorContext.get().createName("expression.group")); value = new ArrayList<>(); this.setNameSpace(rule.getNameSpace()); fingerprintCache = FingerprintCache.getInstance(); } + public static void main(String[] args) { +// String content = "abdfdfd"; +// String regex = "ab.*fd"; +// System.out.println(StringUtil.matchRegex(content, regex)); + + BitSetCache.BitSet bitset = new BitSetCache.BitSet(1); +// bitset.set(0); + System.out.println(bitset.getBytes().length); + } + public void compile() { if (!hasCompile.compareAndSet(false, true)) { return; @@ -71,7 +81,7 @@ public void compile() { regexEngine = new RegexEngine(); for (Expression expression : getValue()) { if (SimpleExpression.class.isInstance(expression) && (RegexFunction.isRegex(expression.getFunctionName()))) { - regexEngine.addRegex((String) expression.getValue(), expression.getConfigureName()); + regexEngine.addRegex((String) expression.getValue(), expression.getName()); } else { notRegexExpression.add(expression); } @@ -108,11 +118,9 @@ public Set getDependentFields(Map expressionMap) { return set; } - protected transient String fingerpringtNamespace; - protected String getFingerprintNamespace() { if (fingerpringtNamespace == null) { - return MapKeyUtil.createKey(getNameSpace(), rule.getConfigureName(), getConfigureName()); + return MapKeyUtil.createKey(getNameSpace(), rule.getName(), getName()); } return fingerpringtNamespace; } @@ -153,7 +161,7 @@ private Boolean executeMatch(IMessage message, AbstractContext context) { public void addExpressionName(Expression expression) { if (RegexFunction.isRegex(expression.getFunctionName())) { - regexExpressionNameSet.add(expression.getConfigureName()); + regexExpressionNameSet.add(expression.getName()); } // if(LikeFunction.isLikeFunciton(expression.getFunctionName())){ // regexExpressionNameSet.add(expression.getConfigureName()); @@ -181,18 +189,8 @@ public int size() { public Collection getAllExpressionNames() { Set names = new HashSet<>(); for (Expression expression : getValue()) { - names.add(expression.getConfigureName()); + names.add(expression.getName()); } return names; } - - public static void main(String[] args) { -// String content = "abdfdfd"; -// String regex = "ab.*fd"; -// System.out.println(StringUtil.matchRegex(content, regex)); - - BitSetCache.BitSet bitset = new BitSetCache.BitSet(1); -// bitset.set(0); - System.out.println(bitset.getBytes().length); - } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/GroupExpressionManager.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/GroupExpressionManager.java index 5e278f75..ae21001d 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/GroupExpressionManager.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/GroupExpressionManager.java @@ -52,7 +52,7 @@ public void compile() { regexEngine = new RegexEngine(); regexEngineMap.put(varName, regexEngine); } - regexEngine.addRegex((String) expression.getValue(), expression.getConfigureName()); + regexEngine.addRegex((String) expression.getValue(), expression.getName()); // if(LikeFunction.isLikeFunciton(expression.getFunctionName())){ // String like=(String)expression.getValue(); // LikeRegex likeRegex=new LikeRegex(like); diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/RelationExpression.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/RelationExpression.java index 9dd3e28d..87402e52 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/RelationExpression.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/RelationExpression.java @@ -23,26 +23,29 @@ import java.util.List; import java.util.Map; import java.util.Set; -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.IMessage; -import org.apache.rocketmq.streams.common.monitor.TopologyFilterMonitor; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.common.utils.TraceUtil; import org.apache.rocketmq.streams.filter.context.RuleContext; import org.apache.rocketmq.streams.filter.operator.Rule; import org.apache.rocketmq.streams.script.utils.FunctionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class RelationExpression extends Expression> { private static final long serialVersionUID = -3213091464347965570L; - private static final Log LOG = LogFactory.getLog(RelationExpression.class); + private static final Logger LOGGER = LoggerFactory.getLogger(RelationExpression.class); + protected volatile ExpressionPerformance expressionPerformance; + protected transient Map expressionMap; private String relation = "and"; // and or // 前端处理使用 private String expressions; // @隔开的表达式名称 - protected volatile ExpressionPerformance expressionPerformance; + public RelationExpression() { + + } @Override protected void getJsonValue(JSONObject jsonObject) { @@ -54,13 +57,6 @@ protected void setJsonValue(JSONObject jsonObject) { this.relation = jsonObject.getString("relation"); } - protected transient Map expressionMap; - - - public RelationExpression() { - - } - @Override protected boolean initConfigurable() { boolean success = super.initConfigurable(); @@ -125,7 +121,7 @@ public Boolean doMessage(IMessage message, AbstractContext context) { if (StringUtil.isEmpty(relation)) { return false; } - boolean isTrace= TraceUtil.hit(message.getHeader().getTraceId()); + boolean isTrace = TraceUtil.hit(message.getHeader().getTraceId()); if ("and".equals(this.relation)) { while (it.hasNext()) { if (flag) { @@ -135,8 +131,8 @@ public Boolean doMessage(IMessage message, AbstractContext context) { Boolean result = message.getMessageBody().getBoolean(expressionName); if (result != null) { if (result == false) { - if(isTrace){ - RuleContext.addNotFireExpressionMonitor(expressionName,context); + if (isTrace) { + RuleContext.addNotFireExpressionMonitor(expressionName, context); } return optimizate(expressionName, false); @@ -144,30 +140,31 @@ public Boolean doMessage(IMessage message, AbstractContext context) { continue; } } else { - throw new RuntimeException("expect exist expression, but not " + expressionName); + return false; +// throw new RuntimeException("expect exist expression, but not " + expressionName); } } if (RelationExpression.class.isInstance(exp)) { - Boolean foreachResult = exp.doMessage(message,context); + Boolean foreachResult = exp.doMessage(message, context); if (foreachResult != null && !foreachResult) { // expressions.add(exp.toString()); return optimizate(expressionName, false); } } else { try { - flag =exp.doMessage(message,context); + flag = exp.doMessage(message, context); } catch (Exception e) { - LOG.error("RelationExpression and function.doFunction error,rule is: " - + getConfigureName() + " ,express is: " + exp.getConfigureName(), e); + LOGGER.error("RelationExpression and function.doFunction error,rule is: " + + getName() + " ,express is: " + exp.getName(), e); e.printStackTrace(); return false; } if (!flag) { - if(isTrace){ - RuleContext.addNotFireExpressionMonitor(exp,context); + if (isTrace) { + RuleContext.addNotFireExpressionMonitor(exp, context); } return optimizate(expressionName, false); @@ -179,7 +176,6 @@ public Boolean doMessage(IMessage message, AbstractContext context) { } else {// or flag = false; - while (it.hasNext()) { String expressionName = it.next(); Expression exp = getExpression(expressionName); @@ -192,11 +188,12 @@ public Boolean doMessage(IMessage message, AbstractContext context) { continue; } } else { - throw new RuntimeException("expect exist expression, but not " + expressionName); + return false; +// throw new RuntimeException("expect exist expression, but not " + expressionName); } } if (RelationExpression.class.isInstance(exp)) { - Boolean foreachResult = exp.doMessage(message,context); + Boolean foreachResult = exp.doMessage(message, context); if (foreachResult != null && foreachResult) { // expressions.add(exp.toString()); return optimizate(expressionName, true); @@ -207,11 +204,11 @@ public Boolean doMessage(IMessage message, AbstractContext context) { } else { try { - flag =exp.doMessage(message,context); + flag = exp.doMessage(message, context); } catch (Exception e) { - LOG.error( - "RelationExpression or function.doFunction error,rule is: " + getConfigureName() - + " ,express is: " + exp.getConfigureName(), e); + LOGGER.error( + "RelationExpression or function.doFunction error,rule is: " + getName() + + " ,express is: " + exp.getName(), e); } if (flag) { return optimizate(expressionName, true); @@ -219,8 +216,8 @@ public Boolean doMessage(IMessage message, AbstractContext context) { } } - if(isTrace){ - RuleContext.addNotFireExpressionMonitor(this,context); + if (isTrace) { + RuleContext.addNotFireExpressionMonitor(this, context); } return false; @@ -229,7 +226,7 @@ public Boolean doMessage(IMessage message, AbstractContext context) { } private Expression getExpression(String name) { - return this.expressionMap.get(name); + return this.expressionMap.get(name); } public Boolean optimizate(String exprssionName, Boolean value) { @@ -277,7 +274,7 @@ public void setExpressions(String expressions) { if (exps.length > 0) { for (String express : exps) { // 防止表达式组里依赖自己,造成死循环 - if (this.getConfigureName().equals(express)) { + if (this.getName().equals(express)) { continue; } this.addExpression(express.trim()); @@ -320,10 +317,20 @@ public void setExpressionsNotValue(String expressions) { @Override public String toExpressionString(Map name2Expression, - String... expressionNamePrefixs) { + String... expressionNamePrefixs) { + return toExpressionString(name2Expression, 0, expressionNamePrefixs); + } + + @Override + public String toExpressionString(Map name2Expression, int blackCount, + String... expressionNamePrefixs) { String sign = "and".equals(relation) ? "&" : "|"; StringBuilder sb = new StringBuilder(); sb.append("("); +// if(blackCount>0){ +// sb.append("

    "); +// } + boolean isFirst = true; for (String expressionName : this.getValue()) { String expressKey = expressionName; @@ -340,20 +347,68 @@ public String toExpressionString(Map name2Expression, expression = name2Expression.get(expressKey); } - if (expression == null) { - continue; - } if (isFirst) { isFirst = false; } else { +// for(int i=0;i   "); +// +// } sb.append(sign); +// if(blackCount>0){ +// sb.append("
    "); +// } + + } + if (expression == null) { + sb.append(expressKey); + continue; } - sb.append( - expression.toExpressionString(name2Expression, expressionNamePrefixs)); +// if(blackCount>0){ +// sb.append(" "); +// } + sb.append(expression.toExpressionString(name2Expression, (blackCount + 1), expressionNamePrefixs)); +// if(blackCount>0){ +// sb.append(" "); +// } } +// if(blackCount>0){ +// sb.append("
    "); +// } + sb.append(")"); return sb.toString(); +// String sign = relation; +// StringBuilder sb = new StringBuilder(); +// sb.append("(
    "); +// boolean isFirst = true; +// for (String expressionName : this.getValue()) { +// String expressKey = expressionName; +// Expression expression = null; +// if (expressionNamePrefixs != null && expressionNamePrefixs.length > 0) { +// for (String prefix : expressionNamePrefixs) { +// expressKey = prefix + expressionName; +// expression = name2Expression.get(expressKey); +// if (expression != null) { +// break; +// } +// } +// } else { +// expression = name2Expression.get(expressKey); +// } +// +// if (isFirst) { +// isFirst = false; +// } else { +// sb.append(" "+sign+" "); +// } +// sb.append( +// expression==null?expressionName: expression.toExpressionString(name2Expression, expressionNamePrefixs)+ "
    "); +// } +// sb.append(")"); +// return sb.toString(); } + public Map getExpressionMap() { return expressionMap; } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/SimpleExpression.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/SimpleExpression.java index fed21e57..8184d933 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/SimpleExpression.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/SimpleExpression.java @@ -44,6 +44,7 @@ public SimpleExpression(String msgFieldName, String functionName, DataType dataT setValue(dataType.getData(value)); setDataType(dataType); } + public boolean doExecute(JSONObject msg) { return ExpressionBuilder.executeExecute(this, msg); } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/var/ConstantVar.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/var/ConstantVar.java index a5ba6f95..6e3ce553 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/var/ConstantVar.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/var/ConstantVar.java @@ -38,16 +38,15 @@ public T doMessage(IMessage message, AbstractContext context) { return value; } - @SuppressWarnings("unchecked") @Override protected void getJsonObject(JSONObject jsonObject) { // value很多时候会是String类型,这样dataType.toDataJson会报错,所以先转为dataType类型 if (dataType == null) { - dataType = (DataType)new StringDataType(String.class); + dataType = (DataType) new StringDataType(String.class); } try { - this.value = (T)dataType.getData(String.valueOf(value)); + this.value = (T) dataType.getData(String.valueOf(value)); } catch (Exception e) { e.printStackTrace(); } @@ -62,7 +61,7 @@ protected void setJsonObject(JSONObject jsonObject) { String dataTypeJson = jsonObject.getString("dataType"); dataType = (DataType) DataTypeUtil.createDataType(dataTypeJson); String valueString = jsonObject.getString("value"); - this.value = (T)dataType.getData(valueString); + this.value = (T) dataType.getData(valueString); // 前端显示用 String dataTypestr = ""; diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/var/ContextVar.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/var/ContextVar.java index 28f11eb2..09efa83d 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/var/ContextVar.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/var/ContextVar.java @@ -26,9 +26,10 @@ public class ContextVar extends Var { private static final long serialVersionUID = -7025012350292140132L; + protected transient MetaData metaData; private String fieldName; private String metaDataName; // 消息队列对应的meta信息 - protected transient MetaData metaData; + @SuppressWarnings("unchecked") @Override public T doMessage(IMessage message, AbstractContext context) { @@ -38,10 +39,10 @@ public T doMessage(IMessage message, AbstractContext context) { Object object = null; if (metaData == null) { object = message.getMessageBody().get(fieldName); - if (object == null&& !JSONObject.class.isInstance(message.getMessageBody())&&fieldName.indexOf(".")!=-1) { + if (object == null && !JSONObject.class.isInstance(message.getMessageBody()) && fieldName.indexOf(".") != -1) { object = ReflectUtil.getBeanFieldOrJsonValue(message, String.class, fieldName); } - return (T)object; + return (T) object; } Class dataClass = String.class; MetaDataField field = metaData.getMetaDataField(fieldName); @@ -52,7 +53,7 @@ public T doMessage(IMessage message, AbstractContext context) { if (object == null) { return null; } - return (T)object; + return (T) object; } public String getFieldName() { diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/var/InnerVar.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/var/InnerVar.java index c82975a2..b7669cfa 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/var/InnerVar.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/var/InnerVar.java @@ -22,8 +22,8 @@ @SuppressWarnings("rawtypes") public class InnerVar extends Var { - private static final long serialVersionUID = -166963014761276615L; public static final String ORIG_MESSAGE = "inner_message"; + private static final long serialVersionUID = -166963014761276615L; @Override public Object doMessage(IMessage message, AbstractContext context) { diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/var/Var.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/var/Var.java index b306a7d3..97f64119 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/var/Var.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/var/Var.java @@ -17,16 +17,12 @@ package org.apache.rocketmq.streams.filter.operator.var; import java.io.Serializable; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; import org.apache.rocketmq.streams.common.configurable.IConfigurable; import org.apache.rocketmq.streams.filter.operator.action.IConfigurableAction; public abstract class Var extends BasedConfigurable implements IConfigurable, IConfigurableAction, Serializable { - private static final Log LOG = LogFactory.getLog(Var.class); - public static final String TYPE = "var"; protected String varName; @@ -43,7 +39,6 @@ public void setVarName(String varName) { this.varName = varName; } - @Override public boolean init() { return true; diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/ExpressionOptimization.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/ExpressionOptimization.java index 364213a3..d1e80f8f 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/ExpressionOptimization.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/ExpressionOptimization.java @@ -43,6 +43,11 @@ public ExpressionOptimization(Expression rootExpression, Map this.expressionMap = expressionMap; } + public static void main(String[] args) { + String expressionName = "((a,=,c)&((d,=,e)|(f,=,d)))&(label|label2)"; + ExpressionBuilder.executeExecute("reew", expressionName, null); + } + /** * 去除不必要的关系,生成只需要的表达式和关系 * @@ -53,7 +58,7 @@ public List optimizate() { if (!RelationExpression.class.isInstance(rootExpression)) { return new ArrayList<>(); } - RelationExpression relationExpression = (RelationExpression)rootExpression; + RelationExpression relationExpression = (RelationExpression) rootExpression; optimizate(relationExpression); List expressions = new ArrayList<>(); createExpressionList(relationExpression, expressions); @@ -73,7 +78,7 @@ protected void createExpressionList(Expression expression, List expr if (RelationExpression.class.isInstance(expression) == false) { return; } - RelationExpression relationExpression = (RelationExpression)expression; + RelationExpression relationExpression = (RelationExpression) expression; for (String name : relationExpression.getValue()) { Expression subExpression = getExpression(name); if (subExpression == null) { @@ -93,12 +98,12 @@ protected void createExpressionList(Expression expression, List expr protected void initExpressionMap(List simpleExpressions, List relationExpressions) { if (simpleExpressions != null) { for (Expression expression : simpleExpressions) { - this.expressionMap.put(expression.getConfigureName(), expression); + this.expressionMap.put(expression.getName(), expression); } } if (relationExpressions != null) { for (RelationExpression relationExpression : relationExpressions) { - this.expressionMap.put(relationExpression.getConfigureName(), relationExpression); + this.expressionMap.put(relationExpression.getName(), relationExpression); } } } @@ -121,10 +126,10 @@ public void optimizate(RelationExpression parentExpression) { names.add(expressionName); continue; } - RelationExpression relationExpression = (RelationExpression)children; + RelationExpression relationExpression = (RelationExpression) children; optimizate(relationExpression); if (!relationExpression.getRelation().equals(parentRelation)) { - names.add(relationExpression.getConfigureName()); + names.add(relationExpression.getName()); } else { names.addAll(relationExpression.getValue()); } @@ -135,9 +140,4 @@ public void optimizate(RelationExpression parentExpression) { private Expression getExpression(String expressionName) { return this.expressionMap.get(expressionName); } - - public static void main(String[] args) { - String expressionName = "((a,=,c)&((d,=,e)|(f,=,d)))&(label|label2)"; - ExpressionBuilder.executeExecute("reew", expressionName, null); - } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/PiplineLogFingerprintAnalysis.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/PiplineLogFingerprintAnalysis.java index 0271ba9f..7fe7b97d 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/PiplineLogFingerprintAnalysis.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/PiplineLogFingerprintAnalysis.java @@ -24,9 +24,9 @@ import java.util.Set; import org.apache.rocketmq.streams.common.metadata.MetaData; import org.apache.rocketmq.streams.common.metadata.MetaDataField; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; import org.apache.rocketmq.streams.common.topology.model.AbstractRule; import org.apache.rocketmq.streams.common.topology.model.AbstractStage; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; import org.apache.rocketmq.streams.common.topology.stages.ScriptChainStage; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; @@ -44,10 +44,12 @@ public class PiplineLogFingerprintAnalysis { public PiplineLogFingerprintAnalysis(ChainPipeline pipline) { this.pipline = pipline; } - public PiplineLogFingerprintAnalysis(ChainPipeline pipline,int filterIndex) { + + public PiplineLogFingerprintAnalysis(ChainPipeline pipline, int filterIndex) { this.pipline = pipline; - this.filterIndex=filterIndex; + this.filterIndex = filterIndex; } + /** * 通过分析pipline找到可以过滤的日志指纹,插入到合适的stage中。如最顶部的stage中 * @@ -66,7 +68,7 @@ public List analysisPipline() { AbstractStage rootStage = stageMap.get(label); //处理一个分支,目前只支持最顶层的分支,后续再有分支暂不支持 AbstractStage stage = analysisStage(rootStage, logFingerFieldNames, null, 0); - logFingerprintFliterChainStage = (FilterChainStage)stage; + logFingerprintFliterChainStage = (FilterChainStage) stage; stageName = stage.getLabel(); } @@ -74,7 +76,7 @@ public List analysisPipline() { List stages = pipline.getStages(); AbstractStage rootStage = stages.get(0); AbstractStage stage = analysisStage(rootStage, logFingerFieldNames, null, 0); - logFingerprintFliterChainStage = (FilterChainStage)getPrewScriptChainStage(stage, FilterChainStage.class); + logFingerprintFliterChainStage = (FilterChainStage) getPrewScriptChainStage(stage, FilterChainStage.class); int i = 0; for (AbstractStage stage1 : stages) { if (stage == stage1) { @@ -91,7 +93,7 @@ public List analysisPipline() { List logFingerList = new ArrayList<>(); logFingerList.addAll(logFingerFieldNames); Collections.sort(logFingerList); - String key = MapKeyUtil.createKeyBySign(".", pipline.getNameSpace(), pipline.getConfigureName(), stageName); + String key = MapKeyUtil.createKeyBySign(".", pipline.getNameSpace(), pipline.getName(), stageName); String value = MapKeyUtil.createKeyFromCollection(",", logFingerList); System.out.println(key + "=" + value); return logFingerList; @@ -105,8 +107,8 @@ public List analysisPipline() { * @return */ private AbstractStage analysisStage(AbstractStage currentStage, Set logFingerFieldNames, - FilterChainStage prewFilterStage, int filterStageIndex) { - ChainPipeline pipline = (ChainPipeline)currentStage.getPipeline(); + FilterChainStage prewFilterStage, int filterStageIndex) { + ChainPipeline pipline = (ChainPipeline) currentStage.getPipeline(); if (currentStage == null) { return prewFilterStage; } @@ -117,7 +119,7 @@ private AbstractStage analysisStage(AbstractStage currentStage, Set logF return prewFilterStage; } if (FilterChainStage.class.isInstance(currentStage)) { - FilterChainStage filterChainStage = (FilterChainStage)currentStage; + FilterChainStage filterChainStage = (FilterChainStage) currentStage; if (filterStageIndex >= filterIndex) { return prewFilterStage; } @@ -130,12 +132,12 @@ private AbstractStage analysisStage(AbstractStage currentStage, Set logF } logFingerFieldNames.addAll(filterLogFingerprints); - prewFilterStage = (FilterChainStage)currentStage; + prewFilterStage = (FilterChainStage) currentStage; filterStageIndex++; } else if (ScriptChainStage.class.isInstance(currentStage)) { //continue - } else { + } else { return prewFilterStage; } AbstractStage nextStage = getNextStage(currentStage); @@ -162,24 +164,22 @@ protected boolean hasExcludeFunctionNames(AbstractRule[] rules) { * @return */ private Set fetchLogFingerprint(FilterChainStage filterChainStage) { - ChainPipeline pipline = (ChainPipeline)filterChainStage.getPipeline(); - List rules=filterChainStage.getRules(); - if (rules == null) { + ChainPipeline pipline = (ChainPipeline) filterChainStage.getPipeline(); + AbstractRule rule = filterChainStage.getRule(); + if (rule == null) { return null; } /** * 规则依赖的所有字段 */ Set dependentFields = new HashSet<>(); - for (AbstractRule rule : rules) { - Set set = rule.getDependentFields(); - dependentFields.addAll(set); - } + Set set = rule.getDependentFields(); + dependentFields.addAll(set); Set logFingerFieldNames = new HashSet<>(); /** * 获取最近的script stage */ - ScriptChainStage scriptChainStage = (ScriptChainStage)getPrewScriptChainStage(filterChainStage, ScriptChainStage.class); + ScriptChainStage scriptChainStage = (ScriptChainStage) getPrewScriptChainStage(filterChainStage, ScriptChainStage.class); for (String field : dependentFields) { Set metaDataFields = getChannelMetaField(field, pipline.getChannelMetaData(), scriptChainStage); if (metaDataFields == null) { @@ -264,7 +264,7 @@ public Set getChannelMetaField(String fieldName, MetaData metaData, Scri if (middleFieldNames.size() == 0) { return channelMetaFields; } - ScriptChainStage prewScriptChainStage = (ScriptChainStage)getPrewScriptChainStage(scriptChainStage, ScriptChainStage.class); + ScriptChainStage prewScriptChainStage = (ScriptChainStage) getPrewScriptChainStage(scriptChainStage, ScriptChainStage.class); if (prewScriptChainStage == null) { return null; } @@ -286,7 +286,7 @@ public Set getChannelMetaField(String fieldName, MetaData metaData, Scri * @return */ protected AbstractStage getPrewScriptChainStage(AbstractStage currentStage, Class stageClass) { - ChainPipeline pipline = (ChainPipeline)currentStage.getPipeline(); + ChainPipeline pipline = (ChainPipeline) currentStage.getPipeline(); /** * 如果是拓扑,则通过拓扑寻找,只支持单链模式。如果有两个来源,则不支持 */ @@ -388,7 +388,7 @@ protected String getMetaDataFieldName(MetaData metaData, String fieldName) { * @return */ protected AbstractStage getNextStage(AbstractStage currentStage) { - ChainPipeline pipline = (ChainPipeline)currentStage.getPipeline(); + ChainPipeline pipline = (ChainPipeline) currentStage.getPipeline(); if (pipline.isTopology()) { List nextLables = currentStage.getNextStageLabels(); if (nextLables == null || nextLables.size() != 1) { @@ -414,7 +414,7 @@ protected AbstractStage getNextStage(AbstractStage currentStage) { public String createLogFingerprint(ChainPipeline pipline) { List logFingerList = analysisPipline(); - return pipline.getNameSpace() + ">>" + pipline.getConfigureName() + "=" + MapKeyUtil.createKey(",", logFingerList); + return pipline.getNameSpace() + ">>" + pipline.getName() + "=" + MapKeyUtil.createKey(",", logFingerList); } public int getFilterIndex() { diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/AbstractWhenExpression.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/AbstractWhenExpression.java index bde74741..edd789c6 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/AbstractWhenExpression.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/AbstractWhenExpression.java @@ -39,68 +39,65 @@ public abstract class AbstractWhenExpression implements IScriptExpression { protected String namespace; protected String name; //group by varname, if has mutil varname, join by ; - protected Map varNames2GroupByVarCaseWhen=new HashMap<>(); + protected Map varNames2GroupByVarCaseWhen = new HashMap<>(); //all case when element - protected List allCaseWhenElement=new ArrayList<>(); + protected List allCaseWhenElement = new ArrayList<>(); //can not cache CaseWhenElement - protected List notCacheCaseWhenElement=new ArrayList<>(); + protected List notCacheCaseWhenElement = new ArrayList<>(); //key:varnames join by ; value:varname list - protected Map>varNames=new HashMap<>(); + protected Map> varNames = new HashMap<>(); //key:varnames join by ; value:index in allCaseWhenElement - protected Map varName2Indexs=new HashMap<>(); + protected Map varName2Indexs = new HashMap<>(); //key:allCaseWhenElement value:index in allCaseWhenElement - protected Map caseWhenElementIndexMap=new HashMap<>(); + protected Map caseWhenElementIndexMap = new HashMap<>(); //create index - protected AtomicInteger groupIndex=new AtomicInteger(0); + protected AtomicInteger groupIndex = new AtomicInteger(0); - public AbstractWhenExpression(String namespace,String name){ - this.name=name; - this.namespace=namespace; + public AbstractWhenExpression(String namespace, String name) { + this.name = name; + this.namespace = namespace; } - @Override public abstract Object executeExpression(IMessage message, FunctionContext context) ; + @Override public abstract Object executeExpression(IMessage message, FunctionContext context); - - - public void registe(CaseWhenElement caseWhenElement, Set varNames){ - List varNameList=new ArrayList<>(); + public void registe(CaseWhenElement caseWhenElement, Set varNames) { + List varNameList = new ArrayList<>(); varNameList.addAll(varNames); Collections.sort(varNameList); - String key= MapKeyUtil.createKey(varNameList); - GroupByVarCaseWhen groupByVarCaseWhen=varNames2GroupByVarCaseWhen.get(key); - if(groupByVarCaseWhen==null){ - groupByVarCaseWhen=new GroupByVarCaseWhen(groupIndex.incrementAndGet()); - varNames2GroupByVarCaseWhen.put(key,groupByVarCaseWhen); + String key = MapKeyUtil.createKey(varNameList); + GroupByVarCaseWhen groupByVarCaseWhen = varNames2GroupByVarCaseWhen.get(key); + if (groupByVarCaseWhen == null) { + groupByVarCaseWhen = new GroupByVarCaseWhen(groupIndex.incrementAndGet()); + varNames2GroupByVarCaseWhen.put(key, groupByVarCaseWhen); } groupByVarCaseWhen.registe(caseWhenElement); allCaseWhenElement.add(caseWhenElement); - this.varNames.put(key,varNameList); - this.varName2Indexs.put(key,allCaseWhenElement.size()-1); - caseWhenElementIndexMap.put(caseWhenElement,allCaseWhenElement.size()-1); + this.varNames.put(key, varNameList); + this.varName2Indexs.put(key, allCaseWhenElement.size() - 1); + caseWhenElementIndexMap.put(caseWhenElement, allCaseWhenElement.size() - 1); } - public void compile(){ - Map groupByVarCaseWhenMap=new HashMap<>(); - for(String key:varNames2GroupByVarCaseWhen.keySet()){ - GroupByVarCaseWhen groupByVarCaseWhen=varNames2GroupByVarCaseWhen.get(key); - if(groupByVarCaseWhen.size()<5||varNames.get(key).size()>5){ + public void compile() { + Map groupByVarCaseWhenMap = new HashMap<>(); + for (String key : varNames2GroupByVarCaseWhen.keySet()) { + GroupByVarCaseWhen groupByVarCaseWhen = varNames2GroupByVarCaseWhen.get(key); + if (groupByVarCaseWhen.size() < 5 || varNames.get(key).size() > 5) { notCacheCaseWhenElement.addAll(groupByVarCaseWhen.getAllCaseWhenElements()); - }else { - groupByVarCaseWhenMap.put(key,groupByVarCaseWhen); + } else { + groupByVarCaseWhenMap.put(key, groupByVarCaseWhen); } //最大个数不超过1个字节能表示的数量 - if(groupByVarCaseWhen.size()>254){ - List removeElements=groupByVarCaseWhen.removeUtilSize(255); - if(removeElements!=null){ + if (groupByVarCaseWhen.size() > 254) { + List removeElements = groupByVarCaseWhen.removeUtilSize(255); + if (removeElements != null) { this.notCacheCaseWhenElement.addAll(removeElements); } } } - this.varNames2GroupByVarCaseWhen=groupByVarCaseWhenMap; + this.varNames2GroupByVarCaseWhen = groupByVarCaseWhenMap; } - @Override public List getScriptParamters() { return null; } @@ -108,6 +105,7 @@ public void compile(){ @Override public String getFunctionName() { return "condition"; } + @Override public String getExpressionDescription() { return null; } @@ -121,41 +119,39 @@ public void compile(){ } @Override public List getDependentFields() { - Set varNames=new HashSet<>(); - for(CaseWhenElement caseWhenElement:this.allCaseWhenElement){ + Set varNames = new HashSet<>(); + for (CaseWhenElement caseWhenElement : this.allCaseWhenElement) { varNames.addAll(caseWhenElement.getDependentFields()); } - List varNameList=new ArrayList<>(); + List varNameList = new ArrayList<>(); varNameList.addAll(varNames); Collections.sort(varNameList); return varNameList; } @Override public Set getNewFieldNames() { - Set varNames=new HashSet<>(); - for(CaseWhenElement caseWhenElement:this.allCaseWhenElement){ + Set varNames = new HashSet<>(); + for (CaseWhenElement caseWhenElement : this.allCaseWhenElement) { varNames.addAll(caseWhenElement.getNewFieldNames()); } return varNames; } - - - protected List executeGroupByVarCaseWhen(String key,GroupByVarCaseWhen groupByVarCaseWhen, IMessage message, FunctionContext context) { - List varList=varNames.get(key); - String varValue=createVarValue(varList,message); - String cacheKey=MapKeyUtil.createKey(namespace,name,groupByVarCaseWhen.index+""); - BitSetCache.BitSet bitSet = FingerprintCache.getInstance().getLogFingerprint(cacheKey,varValue); - List matchCaseWhenElements=new ArrayList<>(); - if(bitSet==null){ - List matchIndexs=groupByVarCaseWhen.executeCase(message,context,executeThenDirectly(),matchCaseWhenElements); - bitSet=new BitSetCache.BitSet(createBytes(matchIndexs)); - FingerprintCache.getInstance().addLogFingerprint(cacheKey,varValue,bitSet); + protected List executeGroupByVarCaseWhen(String key, GroupByVarCaseWhen groupByVarCaseWhen, IMessage message, FunctionContext context) { + List varList = varNames.get(key); + String varValue = createVarValue(varList, message); + String cacheKey = MapKeyUtil.createKey(namespace, name, groupByVarCaseWhen.index + ""); + BitSetCache.BitSet bitSet = FingerprintCache.getInstance().getLogFingerprint(cacheKey, varValue); + List matchCaseWhenElements = new ArrayList<>(); + if (bitSet == null) { + List matchIndexs = groupByVarCaseWhen.executeCase(message, context, executeThenDirectly(), matchCaseWhenElements); + bitSet = new BitSetCache.BitSet(createBytes(matchIndexs)); + FingerprintCache.getInstance().addLogFingerprint(cacheKey, varValue, bitSet); return matchCaseWhenElements; - }else { - byte[] bytes=bitSet.getBytes(); - List matchIndexs=createMatchIndex(bytes); - groupByVarCaseWhen.executeByResult(message,context,matchIndexs,executeThenDirectly(),matchCaseWhenElements); + } else { + byte[] bytes = bitSet.getBytes(); + List matchIndexs = createMatchIndex(bytes); + groupByVarCaseWhen.executeByResult(message, context, matchIndexs, executeThenDirectly(), matchCaseWhenElements); return matchCaseWhenElements; } @@ -163,52 +159,52 @@ protected List executeGroupByVarCaseWhen(String key,GroupByVarC protected abstract boolean executeThenDirectly(); - protected byte[] createBytes(List indexs) { - if(CollectionUtil.isEmpty(indexs)){ - byte[] bytes=new byte[1]; - bytes[0]=NumberUtils.toByte(0)[0]; + if (CollectionUtil.isEmpty(indexs)) { + byte[] bytes = new byte[1]; + bytes[0] = NumberUtils.toByte(0)[0]; return bytes; } - byte[] bytes=new byte[indexs.size()]; - for(int i=0;i createMatchIndex(byte[] bytes) { - if(bytes.length==1&&NumberUtils.toInt(bytes)==0){ + if (bytes.length == 1 && NumberUtils.toInt(bytes) == 0) { return null; } - List matchIndexs=new ArrayList<>(); - for(byte b:bytes){ - Integer index= NumberUtils.toInt(b); - matchIndexs.add(index-1); + List matchIndexs = new ArrayList<>(); + for (byte b : bytes) { + Integer index = NumberUtils.toInt(b); + matchIndexs.add(index - 1); } return matchIndexs; } + /** - * var name to var message value + * var name to var message value + * * @param varNames var name list * @param message * @return */ protected String createVarValue(List varNames, IMessage message) { - StringBuilder stringBuilder=new StringBuilder(); - for(String varName:varNames){ - String varValue=message.getMessageBody().getString(varName); - stringBuilder.append(varValue+";"); + StringBuilder stringBuilder = new StringBuilder(); + for (String varName : varNames) { + String varValue = message.getMessageBody().getString(varName); + stringBuilder.append(varValue + ";"); } return stringBuilder.toString(); } public List getIfExpressions() { - List ruleScripts=new ArrayList<>(); - for(CaseWhenElement caseWhenElement:allCaseWhenElement){ + List ruleScripts = new ArrayList<>(); + for (CaseWhenElement caseWhenElement : allCaseWhenElement) { ruleScripts.add(new RuleExpression(caseWhenElement.getRule())); } return ruleScripts; diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/CaseWhenBuilder.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/CaseWhenBuilder.java index b276804d..fe610782 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/CaseWhenBuilder.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/CaseWhenBuilder.java @@ -55,6 +55,38 @@ public static boolean canSupportSingleCaseWhen(GroupScriptExpression groupScript return true; } + public static boolean canSupportIFCaseWhen(GroupScriptExpression groupScriptExpression) { + if (CollectionUtil.isNotEmpty(groupScriptExpression.getElseIfExpressions())) { + return false; + } + if (CollectionUtil.isEmpty(groupScriptExpression.getElseExpressions())) { + return false; + } + if (CollectionUtil.isEmpty(groupScriptExpression.getThenExpresssions())) { + return false; + } + if (groupScriptExpression.getThenExpresssions().size() == 1 && groupScriptExpression.getElseExpressions().size() == 1) { + IScriptExpression thenExpression = groupScriptExpression.getThenExpresssions().get(0); + IScriptExpression elseExpression = groupScriptExpression.getElseExpressions().get(0); + if (isIFExpression(thenExpression) && isIFExpression(elseExpression)) { + return true; + } + return false; + } else { + return false; + } + } + + private static boolean isIFExpression(IScriptExpression expression) { + if (expression.getFunctionName() == null && expression.getScriptParamters() != null && expression.getScriptParamters().size() == 1) { + IScriptParamter scriptParamter = (IScriptParamter) expression.getScriptParamters().get(0); + if (scriptParamter.getScriptParameterStr().toLowerCase().equals("true") || scriptParamter.getScriptParameterStr().toLowerCase().equals("false")) { + return true; + } + } + return false; + } + public static List compile(String namespace, String name, List expressions) { expressions = mergeGroupScriptExpression(expressions); expressions = groupGroupScriptExpression(namespace, name, expressions); @@ -78,9 +110,9 @@ protected static List mergeGroupScriptExpression(List afterGroup = findAfterExpression(groupScriptExpression, expressions, i, skpitIndex); groupScriptExpression.setBeforeExpressions(beforeGroup); if (afterGroup != null) { @@ -129,6 +161,8 @@ public static List groupGroupScriptExpression(String namespac map.putAll(groupScriptExpression.getBeforeDependents()); SingleCaseWhenExpression singleCaseWhenExpression = registeSingleCaseWhen(namespace, name, groupScriptExpression, map); scriptExpressions.add(singleCaseWhenExpression); + } else if (canSupportIFCaseWhen(groupScriptExpression)) { + scriptExpressions.add(new IFCaseWhenExpression(groupScriptExpression)); } else { scriptExpressions.add(groupScriptExpression); } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/CaseWhenElement.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/CaseWhenElement.java index 0f112dfc..6b426125 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/CaseWhenElement.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/CaseWhenElement.java @@ -38,95 +38,93 @@ public class CaseWhenElement { protected Rule rule; - protected List thenScriptExpressionList; + protected List thenScriptExpressionList; protected List beforeExpressions; - protected transient Map varName2Scripts=new HashMap<>(); - protected transient Map> varName2DependentFields=new HashMap<>(); + protected transient Map varName2Scripts = new HashMap<>(); + protected transient Map> varName2DependentFields = new HashMap<>(); - public CaseWhenElement(GroupScriptExpression groupScriptExpression){ - this(groupScriptExpression,false); + public CaseWhenElement(GroupScriptExpression groupScriptExpression) { + this(groupScriptExpression, false); } - public CaseWhenElement(GroupScriptExpression groupScriptExpression,boolean isCompressBeforeExpression){ - if(groupScriptExpression.getElseIfExpressions()!=null&&groupScriptExpression.getElseIfExpressions().size()>0){ + + public CaseWhenElement(GroupScriptExpression groupScriptExpression, boolean isCompressBeforeExpression) { + if (groupScriptExpression.getElseIfExpressions() != null && groupScriptExpression.getElseIfExpressions().size() > 0) { throw new RuntimeException("can not support has elseif groupScriptExpression, please use CaseWhenFactory"); } - this.thenScriptExpressionList=groupScriptExpression.getThenExpresssions(); - if(groupScriptExpression.getAfterExpressions()!=null){ + this.thenScriptExpressionList = groupScriptExpression.getThenExpresssions(); + if (groupScriptExpression.getAfterExpressions() != null) { this.thenScriptExpressionList.addAll(groupScriptExpression.getAfterExpressions()); } - - IScriptExpression ifExpression=groupScriptExpression.getIfExpresssion(); - if(RuleExpression.class.isInstance(ifExpression)){ - this.rule=((RuleExpression)ifExpression).getRule(); - }else { - if(!CaseFunction.isCaseFunction(ifExpression.getFunctionName())){ + IScriptExpression ifExpression = groupScriptExpression.getIfExpresssion(); + if (RuleExpression.class.isInstance(ifExpression)) { + this.rule = ((RuleExpression) ifExpression).getRule(); + } else { + if (!CaseFunction.isCaseFunction(ifExpression.getFunctionName())) { throw new RuntimeException("can not support not casefunction's ifExpression"); } - String ifStr= IScriptOptimization.getParameterValue((IScriptParamter) ifExpression.getScriptParamters().get(0)); - this.rule= ExpressionBuilder.createRule("tmp","tmp",ifStr); + String ifStr = IScriptOptimization.getParameterValue((IScriptParamter) ifExpression.getScriptParamters().get(0)); + this.rule = ExpressionBuilder.createRule("tmp", "tmp", ifStr); } - - if(isCompressBeforeExpression){ - this.beforeExpressions=addBeforeExpressions(groupScriptExpression.getBeforeExpressions()); - }else { - this.beforeExpressions=groupScriptExpression.getBeforeExpressions(); + if (isCompressBeforeExpression) { + this.beforeExpressions = addBeforeExpressions(groupScriptExpression.getBeforeExpressions()); + } else { + this.beforeExpressions = groupScriptExpression.getBeforeExpressions(); } } - - - public boolean executeCase(IMessage message, AbstractContext context){ - if(beforeExpressions!=null){ - for(IScriptExpression scriptExpression:beforeExpressions){ - scriptExpression.executeExpression(message,(FunctionContext) context); + public boolean executeCase(IMessage message, AbstractContext context) { + if (beforeExpressions != null) { + for (IScriptExpression scriptExpression : beforeExpressions) { + scriptExpression.executeExpression(message, (FunctionContext) context); } } return rule.doMessage(message, context); } - public boolean executeDirectly(IMessage message, AbstractContext context){ - boolean isFire=executeCase(message,context); - if(isFire){ - executeThen(message,context); + public boolean executeDirectly(IMessage message, AbstractContext context) { + boolean isFire = executeCase(message, context); + if (isFire) { + executeThen(message, context); } return isFire; } - public Object executeThen(IMessage message, AbstractContext context){ - if(thenScriptExpressionList!=null){ - for(IScriptExpression scriptExpression:thenScriptExpressionList){ - scriptExpression.executeExpression(message,(FunctionContext) context); + public Object executeThen(IMessage message, AbstractContext context) { + if (thenScriptExpressionList != null) { + for (IScriptExpression scriptExpression : thenScriptExpressionList) { + scriptExpression.executeExpression(message, (FunctionContext) context); } } return null; } + protected List addBeforeExpressions(List expressions) { - for(IScriptExpression expression:expressions){ - if(expression.getNewFieldNames()!=null&&expression.getNewFieldNames().size()==1){ - String newFieldName=expression.getNewFieldNames().iterator().next(); - varName2Scripts.put(newFieldName,expression); - varName2DependentFields.put(newFieldName,expression.getDependentFields()); + for (IScriptExpression expression : expressions) { + if (expression.getNewFieldNames() != null && expression.getNewFieldNames().size() == 1) { + String newFieldName = expression.getNewFieldNames().iterator().next(); + varName2Scripts.put(newFieldName, expression); + varName2DependentFields.put(newFieldName, expression.getDependentFields()); } } - List beforeExpressions=new ArrayList<>(); - Set varNames=getDependentFields(); - for(String varName:varNames){ - beforeExpressions.addAll( addBeforeExpression(varName)); + List beforeExpressions = new ArrayList<>(); + Set varNames = getDependentFields(); + for (String varName : varNames) { + beforeExpressions.addAll(addBeforeExpression(varName)); } Collections.sort(beforeExpressions, new Comparator() { @Override public int compare(IScriptExpression o1, IScriptExpression o2) { - List varNames1= o1.getDependentFields(); - List varNames2=o2.getDependentFields(); - for(String varName:varNames1){ - if(o2.getNewFieldNames()!=null&&o2.getNewFieldNames().contains(varName)){ + List varNames1 = o1.getDependentFields(); + List varNames2 = o2.getDependentFields(); + for (String varName : varNames1) { + if (o2.getNewFieldNames() != null && o2.getNewFieldNames().contains(varName)) { return 1; } } - for(String varName:varNames2){ - if(o1.getNewFieldNames()!=null&&o1.getNewFieldNames().contains(varName)){ + for (String varName : varNames2) { + if (o1.getNewFieldNames() != null && o1.getNewFieldNames().contains(varName)) { return -1; } } @@ -137,34 +135,34 @@ protected List addBeforeExpressions(List e } protected List addBeforeExpression(String varName) { - IScriptExpression scriptExpression= this.varName2Scripts.get(varName); - if(scriptExpression==null){ + IScriptExpression scriptExpression = this.varName2Scripts.get(varName); + if (scriptExpression == null) { return new ArrayList<>(); } - List list=new ArrayList<>(); + List list = new ArrayList<>(); list.add(scriptExpression); //this.beforeScriptExpressions.add(scriptExpression); - List fields=scriptExpression.getDependentFields(); - if(fields!=null){ - for(String fieldName:fields){ - List dependents=addBeforeExpression(fieldName); - if(dependents==null){ + List fields = scriptExpression.getDependentFields(); + if (fields != null) { + for (String fieldName : fields) { + List dependents = addBeforeExpression(fieldName); + if (dependents == null) { return null; - }else { + } else { list.addAll(dependents); } } } return list; } - public Set getDependentFields(){ - return rule.getDependentFields(); - } + public Set getDependentFields() { + return rule.getDependentFields(); + } - public Set getNewFieldNames(){ - Set newFieldNames=new HashSet<>(); - for(IScriptExpression then:thenScriptExpressionList){ + public Set getNewFieldNames() { + Set newFieldNames = new HashSet<>(); + for (IScriptExpression then : thenScriptExpressionList) { newFieldNames.addAll(then.getNewFieldNames()); } return newFieldNames; @@ -178,6 +176,4 @@ public List getThenScriptExpressionList() { return thenScriptExpressionList; } - - } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/GroupByVarCaseWhen.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/GroupByVarCaseWhen.java index a23f9961..8afd1535 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/GroupByVarCaseWhen.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/GroupByVarCaseWhen.java @@ -27,96 +27,99 @@ //group by dependency varName list public class GroupByVarCaseWhen { - private List allCaseWhenElements=new ArrayList<>(); - protected int index; - //key: index in allCaseWhenElement value index in AbstractWhenExpression's allCaseWhenElement - protected Map index2CaseWhenElement=new HashMap<>(); + protected int index; + //key: index in allCaseWhenElement value index in AbstractWhenExpression's allCaseWhenElement + protected Map index2CaseWhenElement = new HashMap<>(); + private List allCaseWhenElements = new ArrayList<>(); - public GroupByVarCaseWhen(int index){ - this.index=index; - } - public void registe(CaseWhenElement caseWhenElement){ - this.allCaseWhenElements.add(caseWhenElement); - this.index2CaseWhenElement.put(this.getAllCaseWhenElements().size()-1, caseWhenElement); - } + public GroupByVarCaseWhen(int index) { + this.index = index; + } - public Object executeByResult(IMessage message, AbstractContext context,List matchIndexs,boolean executeThen,List caseWhenElements){ - if(matchIndexs==null){ - return null; - } - for(Integer index:matchIndexs){ - CaseWhenElement caseWhenElement=allCaseWhenElements.get(index); - if(caseWhenElements!=null){ - caseWhenElements.add(caseWhenElement); - } - if(executeThen){ - caseWhenElement.executeThen(message,context); - } - } - return null; - } + public void registe(CaseWhenElement caseWhenElement) { + this.allCaseWhenElements.add(caseWhenElement); + this.index2CaseWhenElement.put(this.getAllCaseWhenElements().size() - 1, caseWhenElement); + } - public List executeCase(IMessage message, AbstractContext context, boolean isExecuteThen,List caseWhenElements){ - List matchIndexs=new ArrayList<>(); - int index=0; - for(CaseWhenElement caseWhenElement:this.allCaseWhenElements){ - boolean isMatch=false; - if(isExecuteThen){ - isMatch=caseWhenElement.executeDirectly(message,context); - }else { - isMatch=caseWhenElement.executeCase(message,context); - } - if(isMatch){ - if(caseWhenElements!=null){ - caseWhenElements.add(this.index2CaseWhenElement.get(index)); + public Object executeByResult(IMessage message, AbstractContext context, List matchIndexs, boolean executeThen, List caseWhenElements) { + if (matchIndexs == null) { + return null; + } + for (Integer index : matchIndexs) { + CaseWhenElement caseWhenElement = allCaseWhenElements.get(index); + if (caseWhenElements != null) { + caseWhenElements.add(caseWhenElement); } - matchIndexs.add(index); - return matchIndexs; - } - index++; - } - return matchIndexs; - } + if (executeThen) { + caseWhenElement.executeThen(message, context); + } + } + return null; + } + + public List executeCase(IMessage message, AbstractContext context, boolean isExecuteThen, List caseWhenElements) { + List matchIndexs = new ArrayList<>(); + int index = 0; + for (CaseWhenElement caseWhenElement : this.allCaseWhenElements) { + boolean isMatch = false; + if (isExecuteThen) { + isMatch = caseWhenElement.executeDirectly(message, context); + } else { + isMatch = caseWhenElement.executeCase(message, context); + } + if (isMatch) { + if (caseWhenElements != null) { + caseWhenElements.add(this.index2CaseWhenElement.get(index)); + } + matchIndexs.add(index); + return matchIndexs; + } + index++; + } + return matchIndexs; + } + + public List executeDirectly(IMessage message, AbstractContext context) { + return executeCase(message, context, true, null); + } - public List executeDirectly(IMessage message, AbstractContext context){ - return executeCase(message,context,true,null); - } - public Set getDependentFields(){ - Set varNames=new HashSet<>(); - for(CaseWhenElement caseWhenElement:this.allCaseWhenElements){ - varNames.addAll(caseWhenElement.getDependentFields()); - } - return varNames; - } + public Set getDependentFields() { + Set varNames = new HashSet<>(); + for (CaseWhenElement caseWhenElement : this.allCaseWhenElements) { + varNames.addAll(caseWhenElement.getDependentFields()); + } + return varNames; + } - /** - * can supprot 254 element ,if the size >254, need remove elements - * @param size - * @return - */ - public List removeUtilSize(int size){ - if(size()<=size){ - return null; - } - int currentSize=size(); - List all=new ArrayList<>(); - all.addAll(this.getAllCaseWhenElements()); - List removes=new ArrayList<>(); - for(int i=size;i254, need remove elements + * + * @param size + * @return + */ + public List removeUtilSize(int size) { + if (size() <= size) { + return null; + } + int currentSize = size(); + List all = new ArrayList<>(); + all.addAll(this.getAllCaseWhenElements()); + List removes = new ArrayList<>(); + for (int i = size; i < currentSize; i--) { + removes.add(this.allCaseWhenElements.get(i)); + } + for (int i = 0; i < size; i++) { + all.add(this.allCaseWhenElements.get(i)); + } + this.allCaseWhenElements = all; + return removes; + } - public int size(){ - return this.allCaseWhenElements.size(); - } + public int size() { + return this.allCaseWhenElements.size(); + } - public List getAllCaseWhenElements() { - return allCaseWhenElements; - } + public List getAllCaseWhenElements() { + return allCaseWhenElements; + } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/IFCaseWhenExpression.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/IFCaseWhenExpression.java new file mode 100644 index 00000000..1865494c --- /dev/null +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/IFCaseWhenExpression.java @@ -0,0 +1,121 @@ +/* + * 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.filter.optimization.casewhen; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.rocketmq.streams.common.cache.compress.BitSetCache; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.filter.optimization.dependency.ScriptDependent; +import org.apache.rocketmq.streams.script.context.FunctionContext; +import org.apache.rocketmq.streams.script.operator.expression.GroupScriptExpression; +import org.apache.rocketmq.streams.script.service.IScriptExpression; +import org.apache.rocketmq.streams.script.service.IScriptParamter; + +public class IFCaseWhenExpression implements IScriptExpression { + protected String asName; + protected GroupScriptExpression groupScriptExpression; + protected int index; + protected BitSetCache.BitSet bitSet; + protected boolean isReverse = false; + + public IFCaseWhenExpression(GroupScriptExpression groupScriptExpression) { + this.groupScriptExpression = groupScriptExpression; + this.asName = groupScriptExpression.getAfterExpressions().get(0).getNewFieldNames().iterator().next(); + IScriptExpression elseExpression = groupScriptExpression.getElseExpressions().get(0); + IScriptParamter scriptParamter = (IScriptParamter) elseExpression.getScriptParamters().get(0); + if (scriptParamter.getScriptParameterStr().toLowerCase().equals("true")) { + isReverse = true; + } + } + + public void executeBefore(IMessage message, FunctionContext context) { + groupScriptExpression.executeBeforeExpression(message, context); + } + + @Override public Object executeExpression(IMessage message, FunctionContext context) { + return groupScriptExpression.executeExpression(message, context); + } + + @Override public List getScriptParamters() { + return groupScriptExpression.getScriptParamters(); + } + + @Override public String getFunctionName() { + return groupScriptExpression.getFunctionName(); + } + + @Override public String getExpressionDescription() { + return groupScriptExpression.getExpressionDescription(); + } + + @Override public Object getScriptParamter(IMessage message, FunctionContext context) { + return groupScriptExpression.getScriptParamter(message, context); + } + + @Override public String getScriptParameterStr() { + return groupScriptExpression.getScriptParameterStr(); + } + + @Override public List getDependentFields() { + ScriptDependent scriptDependent = new ScriptDependent(groupScriptExpression.getBeforeExpressions()); + Set fieldNames = groupScriptExpression.getIFDependentFields(); + Set dependentFields = new HashSet<>(); + if (fieldNames != null) { + for (String fieldName : fieldNames) { + dependentFields.addAll(scriptDependent.traceaField(fieldName, new AtomicBoolean(false), new ArrayList<>())); + } + return new ArrayList<>(dependentFields); + } + return groupScriptExpression.getDependentFields(); + } + + @Override public Set getNewFieldNames() { + return groupScriptExpression.getNewFieldNames(); + } + + public int getIndex() { + return index; + } + + public void setIndex(int index) { + this.index = index; + } + + public BitSetCache.BitSet getBitSet() { + return bitSet; + } + + public void setBitSet(BitSetCache.BitSet bitSet) { + this.bitSet = bitSet; + } + + public String getAsName() { + return asName; + } + + public GroupScriptExpression getGroupScriptExpression() { + return groupScriptExpression; + } + + public boolean isReverse() { + return isReverse; + } +} diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/IFCaseWhenExpressionGroup.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/IFCaseWhenExpressionGroup.java new file mode 100644 index 00000000..a38c184d --- /dev/null +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/IFCaseWhenExpressionGroup.java @@ -0,0 +1,179 @@ +/* + * 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.filter.optimization.casewhen; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.rocketmq.streams.common.cache.compress.BitSetCache; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintCache; +import org.apache.rocketmq.streams.common.utils.CollectionUtil; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.script.context.FunctionContext; +import org.apache.rocketmq.streams.script.function.impl.string.ToLowerFunction; +import org.apache.rocketmq.streams.script.function.impl.string.TrimFunction; +import org.apache.rocketmq.streams.script.service.IScriptExpression; +import org.apache.rocketmq.streams.script.service.IScriptParamter; + +public class IFCaseWhenExpressionGroup implements IScriptExpression { + protected Map> varName2ReturnNameIndexs = new HashMap<>(); + protected FingerprintCache cache = FingerprintCache.getInstance(); + protected Set dependentFields = new HashSet<>(); + protected List beforeExpressions = new ArrayList<>(); + + protected int size = 0; + + protected String namespace; + + public IFCaseWhenExpressionGroup(String namespace) { + this.namespace = namespace; + } + + @Override public Object executeExpression(IMessage message, FunctionContext context) { + + if (CollectionUtil.isNotEmpty(beforeExpressions)) { + for (IScriptExpression expression : beforeExpressions) { + expression.executeExpression(message, context); + } + } + for (String fieldNames : varName2ReturnNameIndexs.keySet()) { + if (StringUtil.isEmpty(fieldNames)) { + continue; + } + String fieldValue = null; + if (fieldNames.indexOf(MapKeyUtil.SIGN) == -1) { + fieldValue = getMsgValue(message, fieldNames); + + } else { + String[] varNames = MapKeyUtil.splitKey(MapKeyUtil.SIGN); + List values = new ArrayList<>(); + for (String varName : varNames) { + String value = getMsgValue(message, varName); + values.add(value); + + } + fieldValue = MapKeyUtil.createKey(values); + } + fieldValue = fieldNames + ";" + fieldValue; + BitSetCache.BitSet bitSet = cache.getLogFingerprint(namespace, fieldValue); + List iFCaseWhenExpressionIndexs = varName2ReturnNameIndexs.get(fieldNames); + if (bitSet == null) { + bitSet = new BitSetCache.BitSet(iFCaseWhenExpressionIndexs.size()); + int index = 0; + for (IFCaseWhenExpression ifCaseWhenExpression : iFCaseWhenExpressionIndexs) { + ifCaseWhenExpression.executeBefore(message, context); + boolean isMatch = (Boolean) ifCaseWhenExpression.getGroupScriptExpression().getIfExpresssion().executeExpression(message, context); + if (ifCaseWhenExpression.isReverse) { + isMatch = !isMatch; + } + if (isMatch) { + bitSet.set(index); + } + message.getMessageBody().put(ifCaseWhenExpression.getAsName(), isMatch); + index++; + } + cache.addLogFingerprint(namespace, fieldValue, bitSet); + } else { + int index = 0; + for (IFCaseWhenExpression ifCaseWhenExpression : iFCaseWhenExpressionIndexs) { + message.getMessageBody().put(ifCaseWhenExpression.getAsName(), bitSet.get(index)); + index++; + } + } + + } + return null; + } + + protected String getMsgValue(IMessage message, String fieldName) { + String value = message.getMessageBody().getString(fieldName); + if (StringUtil.isNotEmpty(value)) { + return value; + } + return "N/A"; + } + + @Override public List getScriptParamters() { + return null; + } + + @Override public String getFunctionName() { + return null; + } + + @Override public String getExpressionDescription() { + return null; + } + + @Override public Object getScriptParamter(IMessage message, FunctionContext context) { + return null; + } + + @Override public String getScriptParameterStr() { + return null; + } + + @Override public List getDependentFields() { + return new ArrayList<>(dependentFields); + } + + @Override public Set getNewFieldNames() { + return null; + } + + public int addIFCaseWhewnExpression(IFCaseWhenExpression ifCaseWhenExpression) { + size++; + Set varNames = new HashSet<>(ifCaseWhenExpression.getDependentFields()); + dependentFields.addAll(varNames); + List varNamesOrder = new ArrayList<>(varNames); + Collections.sort(varNamesOrder); + String fieldNames = MapKeyUtil.createKey(varNamesOrder); + List ifCaseWhenExpressions = varName2ReturnNameIndexs.get(fieldNames); + if (ifCaseWhenExpressions == null) { + ifCaseWhenExpressions = new ArrayList<>(); + varName2ReturnNameIndexs.put(fieldNames, ifCaseWhenExpressions); + } + ifCaseWhenExpressions.add(ifCaseWhenExpression); + extraBeforeExpressions(ifCaseWhenExpression); + return ifCaseWhenExpressions.size() - 1; + } + + private void extraBeforeExpressions(IFCaseWhenExpression ifCaseWhenExpression) { + List beforeExpressions = ifCaseWhenExpression.groupScriptExpression.getBeforeExpressions(); + if (CollectionUtil.isNotEmpty(beforeExpressions)) { + List beforeExpressionList = new ArrayList<>(); + for (IScriptExpression scriptExpression : beforeExpressions) { + if (TrimFunction.isTrimFunction(scriptExpression.getFunctionName()) || ToLowerFunction.isLowFunction(scriptExpression.getFunctionName())) { + this.beforeExpressions.add(scriptExpression); + } else { + beforeExpressionList.add(scriptExpression); + } + } + ifCaseWhenExpression.getGroupScriptExpression().setBeforeExpressions(beforeExpressionList); + } + } + + public int size() { + return size; + } +} diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/IFExpressionOptimization.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/IFExpressionOptimization.java new file mode 100644 index 00000000..643d1912 --- /dev/null +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/IFExpressionOptimization.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.filter.optimization.casewhen; + +import java.util.ArrayList; +import java.util.List; +import org.apache.rocketmq.streams.common.context.AbstractContext; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.optimization.FilterResultCache; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; +import org.apache.rocketmq.streams.script.optimization.performance.IScriptOptimization; +import org.apache.rocketmq.streams.script.service.IScriptExpression; + +public class IFExpressionOptimization implements IScriptOptimization.IOptimizationCompiler { + + protected List oriScriptExpressions; + protected List optimizationScriptExpressions; + + public IFExpressionOptimization(String namespace, String name, List scriptExpressions) { + this.oriScriptExpressions = scriptExpressions; + this.optimizationScriptExpressions = this.oriScriptExpressions; + List scriptExpressionList = CaseWhenBuilder.compile(namespace, name, scriptExpressions); + List optimizationScriptExpressions = new ArrayList<>(); + boolean startIfScript = false; + boolean startOtherScript = false; + boolean canOptimize = true; + IFCaseWhenExpressionGroup ifCaseWhenExpressionGroup = new IFCaseWhenExpressionGroup(MapKeyUtil.createKey(namespace, name)); + for (IScriptExpression scriptExpression : scriptExpressionList) { + if (IFCaseWhenExpression.class.isInstance(scriptExpression)) { + if (startIfScript == false) { + startIfScript = true; + optimizationScriptExpressions.add(ifCaseWhenExpressionGroup); + } + if (startOtherScript) { + canOptimize = false; + break; + } + IFCaseWhenExpression ifCaseWhenExpression = (IFCaseWhenExpression) scriptExpression; + int index = ifCaseWhenExpressionGroup.addIFCaseWhewnExpression(ifCaseWhenExpression); + ifCaseWhenExpression.setIndex(index); + } else { + if (startIfScript && !startOtherScript == false) { + startOtherScript = true; + } + optimizationScriptExpressions.add(scriptExpression); + + } + } + if (!canOptimize) { + this.optimizationScriptExpressions = oriScriptExpressions; + } + if (ifCaseWhenExpressionGroup.size() > 50) { + this.optimizationScriptExpressions = optimizationScriptExpressions; + } + } + + @Override public FilterResultCache execute(IMessage message, AbstractContext context) { + throw new RuntimeException("can not support this method"); + } + + @Override public List getOptimizationExpressionList() { + return optimizationScriptExpressions; + } +} diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/MutilCaseWhenExpression.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/MutilCaseWhenExpression.java index b5e5bdce..b6a9d990 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/MutilCaseWhenExpression.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/MutilCaseWhenExpression.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.filter.optimization.casewhen; - import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.context.FunctionContext; @@ -25,20 +24,18 @@ public MutilCaseWhenExpression(String namespace, String name) { super(namespace, name); } - @Override public Object executeExpression(IMessage message, FunctionContext context) { - for(String key:varNames2GroupByVarCaseWhen.keySet()){ - executeGroupByVarCaseWhen(key,varNames2GroupByVarCaseWhen.get(key),message,context); + for (String key : varNames2GroupByVarCaseWhen.keySet()) { + executeGroupByVarCaseWhen(key, varNames2GroupByVarCaseWhen.get(key), message, context); } - if(notCacheCaseWhenElement!=null){ - for(CaseWhenElement caseWhenElement:this.notCacheCaseWhenElement){ - caseWhenElement.executeDirectly(message,context); + if (notCacheCaseWhenElement != null) { + for (CaseWhenElement caseWhenElement : this.notCacheCaseWhenElement) { + caseWhenElement.executeDirectly(message, context); } } return null; } - @Override protected boolean executeThenDirectly() { return true; } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/SingleCaseWhenExpression.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/SingleCaseWhenExpression.java index 5d892db8..e372c3c8 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/SingleCaseWhenExpression.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/casewhen/SingleCaseWhenExpression.java @@ -26,64 +26,65 @@ public class SingleCaseWhenExpression extends AbstractWhenExpression { protected List elseScriptExpressions; protected List afterScriptExpressions; - public SingleCaseWhenExpression(String namespace, String name,List elseScriptExpressions,List afterScriptExpressions) { + + public SingleCaseWhenExpression(String namespace, String name, List elseScriptExpressions, List afterScriptExpressions) { super(namespace, name); - this.elseScriptExpressions=elseScriptExpressions; - this.afterScriptExpressions=afterScriptExpressions; + this.elseScriptExpressions = elseScriptExpressions; + this.afterScriptExpressions = afterScriptExpressions; } + @Override public Object executeExpression(IMessage message, FunctionContext context) { - int minIndex=100000; + int minIndex = 100000; - CaseWhenElement firstMatchCaseWhenElement=null; - for(String key:varNames2GroupByVarCaseWhen.keySet()){ - List caseWhenElements= executeGroupByVarCaseWhen(key,varNames2GroupByVarCaseWhen.get(key),message,context); - if(caseWhenElements!=null){ - for(CaseWhenElement caseWhenElement:caseWhenElements){ - Integer index=this.caseWhenElementIndexMap.get(caseWhenElement); - if(index caseWhenElements = executeGroupByVarCaseWhen(key, varNames2GroupByVarCaseWhen.get(key), message, context); + if (caseWhenElements != null) { + for (CaseWhenElement caseWhenElement : caseWhenElements) { + Integer index = this.caseWhenElementIndexMap.get(caseWhenElement); + if (index < minIndex) { + minIndex = index; + firstMatchCaseWhenElement = caseWhenElement; } } } } - if(notCacheCaseWhenElement!=null){ - for(CaseWhenElement caseWhenElement:this.notCacheCaseWhenElement){ - boolean success=caseWhenElement.executeCase(message,context); - if(success){ - Integer index=this.caseWhenElementIndexMap.get(caseWhenElement); - if(index getNewFieldNames() { - Set varNames=new HashSet<>(); - for(IScriptExpression scriptExpression:this.afterScriptExpressions){ - if(scriptExpression.getNewFieldNames()==null){ + Set varNames = new HashSet<>(); + for (IScriptExpression scriptExpression : this.afterScriptExpressions) { + if (scriptExpression.getNewFieldNames() == null) { continue; } varNames.addAll(scriptExpression.getNewFieldNames()); @@ -91,7 +92,6 @@ public SingleCaseWhenExpression(String namespace, String name,List> tmpWhiteListOnline = new HashMap(); protected Map> tmpWhiteListOb = new HashMap(); protected String functionName; + private transient AtomicInteger COUNT = new AtomicInteger(0); public BlinkRuleV2Expression() { } @@ -66,7 +62,23 @@ public BlinkRuleV2Expression(String fullClassName, String functionName) { parserExpressions(); } - private transient AtomicInteger COUNT = new AtomicInteger(0); + public static boolean isBlinkRuleV2Parser(IScriptExpression expression) { + return false; + +// String functionName = expression.getFunctionName(); +// if (functionName == null) { +// return false; +// } +// +// String name = MapKeyUtil.createKey("com.lyra.xs.udf.ext.sas_black_rule_v2", functionName).toLowerCase(); +// +// UDFScript udfScript = configurableService.queryConfigurable(UDFScript.TYPE, name); +// +// if (udfScript != null && "com.lyra.xs.udf.ext.sas_black_rule_v2".equals(udfScript.getFullClassName())) { +// return true; +// } +// return false; + } public RuleSetGroup getDependentFields(String[] varNames) { RuleSetGroup ruleSetGroup = new RuleSetGroup(); @@ -95,6 +107,45 @@ public RuleSetGroup getDependentFields(String[] varNames) { return ruleSetGroup; } + protected JSONObject createMsg(String[] kv) { + JSONObject msg = null; + if (kv.length % 2 == 0) { + msg = new JSONObject(); + for (int i = 0; i < kv.length; i += 2) { + msg.put(FunctionUtils.getConstant(kv[i]), kv[i + 1]); + } + } + return msg; + } + + public void parserExpressions() { + InputStream inputStream = ReflectUtil.forClass(className).getResourceAsStream("/data_4_sas_black_rule_v2.json"); + List rules = FileUtil.loadFileLine(inputStream); + String line = rules.get(0); + JSONArray allRules = JSON.parseArray(line); + for (Object object : allRules) { + JSONObject rule_json = JSON.parseObject(object.toString()); + int status = rule_json.getInteger("status"); + int ruleId = rule_json.getInteger("id"); + String modelId = rule_json.getString("model_id"); + String ruleValue = rule_json.getString("content"); + JSONObject ruleJson = JSONObject.parseObject(ruleValue); + BlinkRule blinkRule = new BlinkRule(ruleJson, ruleId); + if (status == 0) { + if (!tmpWhiteListOnline.containsKey(modelId)) { + tmpWhiteListOnline.put(modelId, new ArrayList()); + } + + ((List) tmpWhiteListOnline.get(modelId)).add(blinkRule); + } + + if (!tmpWhiteListOb.containsKey(modelId)) { + tmpWhiteListOb.put(modelId, new ArrayList()); + } + ((List) tmpWhiteListOb.get(modelId)).add(blinkRule); + } + } + public static class RuleSetGroup { protected List ruleSets = new ArrayList<>(); protected Set varNames = new HashSet<>(); @@ -189,77 +240,16 @@ public Rule createRuleExpression() { RelationExpression relationExpression = new RelationExpression(); relationExpression.setValue(new ArrayList()); relationExpression.setRelation("and"); - relationExpression.setConfigureName(NameCreatorContext.get().createNewName("_blink_rule_v2", "relation")); + relationExpression.setName(NameCreatorContext.get().createName("_blink_rule_v2", "relation")); for (Expression expression : expressions) { - String name = NameCreatorContext.get().createNewName("_blink_rule_v2", expression.getVarName()); - expression.setConfigureName(name); + String name = NameCreatorContext.get().createName("_blink_rule_v2", expression.getVarName()); + expression.setName(name); expression.setNameSpace("tmp"); - relationExpression.getValue().add(expression.getConfigureName()); + relationExpression.getValue().add(expression.getName()); } Rule rule = ExpressionBuilder.createRule("tmp", "tmp", relationExpression, expressions); return rule; } } - - protected JSONObject createMsg(String[] kv) { - JSONObject msg = null; - if (kv.length % 2 == 0) { - msg = new JSONObject(); - for (int i = 0; i < kv.length; i += 2) { - msg.put(FunctionUtils.getConstant(kv[i]), kv[i + 1]); - } - } - return msg; - } - - public void parserExpressions() { - InputStream inputStream = ReflectUtil.forClass(className).getResourceAsStream("/data_4_sas_black_rule_v2.json"); - List rules = FileUtil.loadFileLine(inputStream); - String line = rules.get(0); - JSONArray allRules = JSON.parseArray(line); - Iterator iterator = allRules.iterator(); - while (iterator.hasNext()) { - Object object = iterator.next(); - JSONObject rule_json = JSON.parseObject(object.toString()); - int status = rule_json.getInteger("status"); - int ruleId = rule_json.getInteger("id"); - String modelId = rule_json.getString("model_id"); - String ruleValue = rule_json.getString("content"); - JSONObject ruleJson = JSONObject.parseObject(ruleValue); - BlinkRule blinkRule = new BlinkRule(ruleJson, ruleId); - if (status == 0) { - if (!tmpWhiteListOnline.containsKey(modelId)) { - tmpWhiteListOnline.put(modelId, new ArrayList()); - } - - ((List) tmpWhiteListOnline.get(modelId)).add(blinkRule); - } - - if (!tmpWhiteListOb.containsKey(modelId)) { - tmpWhiteListOb.put(modelId, new ArrayList()); - } - ((List) tmpWhiteListOb.get(modelId)).add(blinkRule); - } - } - - public static boolean isBlinkRuleV2Parser(IScriptExpression expression, IConfigurableService configurableService) { - if (configurableService == null) { - return false; - } - - String functionName = expression.getFunctionName(); - if (functionName == null) { - return false; - } - - String name = MapKeyUtil.createKey("com.lyra.xs.udf.ext.sas_black_rule_v2", functionName).toLowerCase(); - - UDFScript udfScript = configurableService.queryConfigurable(UDFScript.TYPE, name); - - if (udfScript != null && "com.lyra.xs.udf.ext.sas_black_rule_v2".equals(udfScript.getFullClassName())) { - return true; - } - return false; - } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/DependencyTree.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/DependencyTree.java index 4881bdfd..9daebb3d 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/DependencyTree.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/DependencyTree.java @@ -17,16 +17,14 @@ package org.apache.rocketmq.streams.filter.optimization.dependency; import java.util.ArrayList; -import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; -import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintCache; import org.apache.rocketmq.streams.common.optimization.fingerprint.PreFingerprint; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; import org.apache.rocketmq.streams.common.topology.model.AbstractStage; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; import org.apache.rocketmq.streams.common.topology.stages.ScriptChainStage; import org.apache.rocketmq.streams.common.utils.CollectionUtil; @@ -36,12 +34,12 @@ */ public class DependencyTree { - - protected ChainPipeline chainPipeline; - protected FingerprintCache fingerprintCache; - public DependencyTree(ChainPipeline pipeline, FingerprintCache fingerprintCache) { - this.fingerprintCache = fingerprintCache; + protected List commonExpressions; + protected transient Map> preFingerprintExecutor = new HashMap<>(); +// protected FingerprintCache fingerprintCache; + + public DependencyTree(ChainPipeline pipeline) { this.chainPipeline = pipeline; } @@ -55,12 +53,7 @@ public List parse() { } else { return null; } - /** - * Create prefix fingerprint objects by branch and Merge branch - * - */ - Collection> preFingerprintMap = chainPipeline.getPreFingerprintExecutor().values(); - for (Map fingerprintMap : preFingerprintMap) { + for (Map fingerprintMap : preFingerprintExecutor.values()) { for (PreFingerprint fingerprint : fingerprintMap.values()) { fingerprint.getFilterChainStage().setPreFingerprint(fingerprint); for (AbstractStage previewFilterChainStage : fingerprint.getAllPreviewFilterChainStage()) { @@ -76,6 +69,7 @@ public List parse() { initSuccessCommonExpressions.add(commonExpression); } } + this.commonExpressions = initSuccessCommonExpressions; // System.out.println("finish homologous optimization"); return initSuccessCommonExpressions; } @@ -86,18 +80,16 @@ public List parse() { * @param pipeline */ public List parseTopology(ChainPipeline pipeline) { - if(StateLessDependencyTree.cache.containsKey(pipeline)){ + if (StateLessDependencyTree.cache.containsKey(pipeline)) { return StateLessDependencyTree.cache.get(pipeline); } List nextLalbes = pipeline.getChannelNextStageLabel(); List commonExpressions = new ArrayList<>(); parseTree(null, nextLalbes, pipeline, commonExpressions); - StateLessDependencyTree.cache.put(chainPipeline,commonExpressions); + StateLessDependencyTree.cache.put(chainPipeline, commonExpressions); return commonExpressions; } - - /** * @param parentTreeNode * @param nextLables @@ -122,7 +114,7 @@ protected void parseTree(TreeNode parentTreeNode, List nextLables, Chain } } else if (FilterChainStage.class.isInstance(stage)) { FilterTreeNode filterTreeNode = new FilterTreeNode(pipeline, (FilterChainStage) stage, parentTreeNode); - PreFingerprint preFingerprint = filterTreeNode.createPreFingerprint(this.fingerprintCache); + PreFingerprint preFingerprint = filterTreeNode.createPreFingerprint(); if (preFingerprint == null) { continue; } @@ -156,7 +148,7 @@ protected boolean mergePreFingerprint(PreFingerprint fingerprint, ChainPipeline if (sourceLable == null) { sourceLable = pipeline.getChannelName(); } - Map> preFingerprintExecutor = pipeline.getPreFingerprintExecutor(); + Map> preFingerprintExecutor = this.preFingerprintExecutor; Map preFingerprintMap = preFingerprintExecutor.get(sourceLable); if (preFingerprintMap == null) { preFingerprintMap = new HashMap<>(); @@ -226,8 +218,20 @@ protected Set loadLogFingerFieldNames(PreFingerprint preFingerprint) { return logFingerFieldNameSet; } + public Map> getPreFingerprintExecutor() { + return preFingerprintExecutor; + } + + public void setPreFingerprintExecutor( + Map> preFingerprintExecutor) { + this.preFingerprintExecutor = preFingerprintExecutor; + } + protected List parseSimple(ChainPipeline pipeline) { throw new RuntimeException("can not support this method"); } + public List getCommonExpressions() { + return commonExpressions; + } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/EmptyTreeNode.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/EmptyTreeNode.java new file mode 100644 index 00000000..a2548283 --- /dev/null +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/EmptyTreeNode.java @@ -0,0 +1,44 @@ +/* + * 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.filter.optimization.dependency; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; +import org.apache.rocketmq.streams.common.topology.stages.EmptyChainStage; +import org.apache.rocketmq.streams.script.service.IScriptExpression; + +public class EmptyTreeNode extends TreeNode { + public EmptyTreeNode(ChainPipeline pipeline, + EmptyChainStage stage, TreeNode parent) { + super(pipeline, stage, parent); + } + + @Override + public Set traceaField(String varName, AtomicBoolean isBreak, List scriptExpressions) { + Set result = new HashSet<>(); + result.add(varName); + return result; + } + + @Override public List traceDepenentToSource() { + return new ArrayList<>(); + } +} diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/FilterTreeNode.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/FilterTreeNode.java index ba8a62c3..06157500 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/FilterTreeNode.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/FilterTreeNode.java @@ -23,10 +23,9 @@ import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintCache; import org.apache.rocketmq.streams.common.optimization.fingerprint.PreFingerprint; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; import org.apache.rocketmq.streams.common.topology.model.AbstractStage; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; import org.apache.rocketmq.streams.common.utils.CollectionUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; @@ -44,15 +43,12 @@ public class FilterTreeNode extends TreeNode { public FilterTreeNode(ChainPipeline pipeline, FilterChainStage stage, TreeNode parent) { super(pipeline, stage, parent); - if (stage.getRules().size() > 1) { - throw new RuntimeException("can not optimizate mutil rule stages " + stage.getLabel()); - } - rule = (Rule) stage.getRules().get(0); + rule = (Rule) stage.getRule(); dependentFieldNames = rule.getDependentFields(); } - public PreFingerprint createPreFingerprint(FingerprintCache fingerprintCache) { - String filterStageIdentification = MapKeyUtil.createKey(rule.getNameSpace(), this.pipeline.getConfigureName(), rule.getConfigureName()); + public PreFingerprint createPreFingerprint() { + String filterStageIdentification = MapKeyUtil.createKey(rule.getNameSpace(), this.pipeline.getName(), rule.getName()); List parents = this.getStage().getPrevStageLabels(); if (parents == null || this.getParents() == null || this.getParents().size() == 0) { @@ -60,7 +56,7 @@ public PreFingerprint createPreFingerprint(FingerprintCache fingerprintCache) { if (containsDimField) { return null; } - PreFingerprint preFingerprint = new PreFingerprint(createFingerprint(dependentFieldNames), filterStageIdentification, pipeline.getChannelName(), this.stage.getLabel(), getExpressionCount(), stage, fingerprintCache); + PreFingerprint preFingerprint = new PreFingerprint(createFingerprint(dependentFieldNames), filterStageIdentification, pipeline.getChannelName(), this.stage.getLabel(), getExpressionCount(), stage); return preFingerprint; } if (parents.size() > 1) { @@ -75,7 +71,7 @@ public PreFingerprint createPreFingerprint(FingerprintCache fingerprintCache) { if (denpendentFields == null) { return null; } - PreFingerprint preFingerprint = new PreFingerprint(createFingerprint(denpendentFields), filterStageIdentification, sourceStage == null ? null : sourceStage.getLabel(), nextStage.getLabel(), getExpressionCount(), stage, fingerprintCache); + PreFingerprint preFingerprint = new PreFingerprint(createFingerprint(denpendentFields), filterStageIdentification, sourceStage == null ? null : sourceStage.getLabel(), nextStage.getLabel(), getExpressionCount(), stage); return preFingerprint; } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/PipelineTree.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/PipelineTree.java index 0a32bc93..a8d93724 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/PipelineTree.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/PipelineTree.java @@ -18,8 +18,8 @@ import java.util.List; import java.util.Map; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; import org.apache.rocketmq.streams.common.topology.model.AbstractStage; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; import org.apache.rocketmq.streams.common.topology.stages.ScriptChainStage; @@ -33,10 +33,6 @@ public PipelineTree(ChainPipeline pipeline) { stageMap = pipeline.getStageMap(); } - protected class Context { - - } - public void registePreFingerprint(Context context, TreeNode parent, List nextLabels) { TreeNode current = null; if (nextLabels != null) { @@ -54,4 +50,8 @@ public void registePreFingerprint(Context context, TreeNode parent, List } } + protected class Context { + + } + } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/ScriptDependent.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/ScriptDependent.java index 20030e89..47a8b6d1 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/ScriptDependent.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/ScriptDependent.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.filter.optimization.dependency; - import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; @@ -37,32 +36,41 @@ import org.apache.rocketmq.streams.script.utils.FunctionUtils; public class ScriptDependent { + private static Set udtfNames = new HashSet<>(); + + static { + for (int i = 0; i < 100; i++) { + udtfNames.add(FunctionType.UDTF.getName() + "" + i); + } + } + protected transient Map varName2Scripts = new HashMap<>(); protected transient Map> varName2DependentFields = new HashMap<>(); protected transient String namespace; - protected transient boolean hasExecFunction=false; - public ScriptDependent(String namespace,String scriptValue) { - this.namespace=namespace; - FunctionScript functionScript=new FunctionScript(scriptValue); + protected transient boolean hasExecFunction = false; + + public ScriptDependent(String namespace, String scriptValue) { + this.namespace = namespace; + FunctionScript functionScript = new FunctionScript(scriptValue); functionScript.init(); init(functionScript.getScriptExpressions()); } public ScriptDependent(FunctionScript functionScript) { - this.namespace=functionScript.getNameSpace(); - init(functionScript.getScriptExpressions()); + this.namespace = functionScript.getNameSpace(); + init(functionScript.getScriptExpressions()); } public ScriptDependent(List scriptExpressions) { init(scriptExpressions); } - private void init(List scriptExpressions){ + private void init(List scriptExpressions) { for (IScriptExpression expression : scriptExpressions) { if (expression.getNewFieldNames() != null && expression.getNewFieldNames().size() == 1) { String newFieldName = expression.getNewFieldNames().iterator().next(); - if(CaseFunction.isCaseFunction(expression.getFunctionName())) { + if (CaseFunction.isCaseFunction(expression.getFunctionName())) { String expressionStr = FunctionUtils.getConstant(((IScriptParamter) expression.getScriptParamters().get(0)).getScriptParameterStr()); varName2Scripts.put(newFieldName, expression); Rule rule = ExpressionBuilder.createRule("tmp", "tmp", expressionStr); @@ -73,108 +81,101 @@ private void init(List scriptExpressions){ varName2DependentFields.put(newFieldName, expression.getDependentFields()); } - if(EvalFunction.isFunction(expression.getFunctionName())){ - hasExecFunction=true; + if (EvalFunction.isFunction(expression.getFunctionName())) { + hasExecFunction = true; } } } - - private static Set udtfNames=new HashSet<>(); - static { - for(int i=0;i<100;i++){ - udtfNames.add(FunctionType.UDTF.getName()+""+i); - } - } - /** * trace root field dependent in this script + * * @param varName * @return */ public Set traceaField(String varName, AtomicBoolean isBreak, List commonExpressions) { - if(hasExecFunction){ + if (hasExecFunction) { isBreak.set(true); return null; } - Set hasTraceFieldNames=new HashSet<>(); + Set hasTraceFieldNames = new HashSet<>(); hasTraceFieldNames.add(varName); - Set vars=traceaField(varName,hasTraceFieldNames,true,isBreak,commonExpressions); + Set vars = traceaField(varName, hasTraceFieldNames, true, isBreak, commonExpressions); return vars; } /** * trace root field dependent in this script + * * @param varName * @return */ - public Set traceaField(String varName,Set hasTraceFieldNames,boolean isFirst, AtomicBoolean isBreak, List commonExpressions) { + public Set traceaField(String varName, Set hasTraceFieldNames, boolean isFirst, AtomicBoolean isBreak, List commonExpressions) { - if("null".equals(varName)){ + if ("null".equals(varName)) { return new HashSet<>(); } - Set fields=new HashSet<>(); + Set fields = new HashSet<>(); - if(hasTraceFieldNames.contains(varName)&&!isFirst){ + if (hasTraceFieldNames.contains(varName) && !isFirst) { fields.add(varName); return fields; } - List depenentFields=varName2DependentFields.get(varName); - if(depenentFields==null){ + List depenentFields = varName2DependentFields.get(varName); + if (depenentFields == null) { fields.add(varName); return fields; } - IScriptExpression scriptExpression=this.varName2Scripts.get(varName); - if(scriptExpression!=null){ + IScriptExpression scriptExpression = this.varName2Scripts.get(varName); + if (scriptExpression != null) { commonExpressions.add(scriptExpression); } - Set depenentFieldSet=new HashSet<>(depenentFields); - for(String newFieldName:depenentFieldSet){ - if(newFieldName.equals(varName)){ + Set depenentFieldSet = new HashSet<>(depenentFields); + for (String newFieldName : depenentFieldSet) { + if (newFieldName.equals(varName)) { fields.add(varName); continue; } - if(udtfNames.contains(newFieldName)){ + if (udtfNames.contains(newFieldName)) { isBreak.set(true); fields.add(varName); continue; } - Set dependentFields=traceaField(newFieldName,hasTraceFieldNames,false,isBreak,commonExpressions); + Set dependentFields = traceaField(newFieldName, hasTraceFieldNames, false, isBreak, commonExpressions); fields.addAll(dependentFields); } return fields; } - public List getDependencyExpression(String varName) { - IScriptExpression scriptExpression= this.varName2Scripts.get(varName); - if(scriptExpression==null){ + IScriptExpression scriptExpression = this.varName2Scripts.get(varName); + if (scriptExpression == null) { return new ArrayList<>(); } - List list=new ArrayList<>(); + List list = new ArrayList<>(); list.add(scriptExpression); - List fields=scriptExpression.getDependentFields(); - if(fields!=null){ - for(String fieldName:fields){ - List dependents=getDependencyExpression(fieldName); - if(dependents==null){ + List fields = scriptExpression.getDependentFields(); + if (fields != null) { + for (String fieldName : fields) { + List dependents = getDependencyExpression(fieldName); + if (dependents == null) { return null; - }else { + } else { list.addAll(dependents); } } } Collections.sort(list, new Comparator() { @Override public int compare(IScriptExpression o1, IScriptExpression o2) { - List varNames1= o1.getDependentFields(); - List varNames2=o2.getDependentFields(); - for(String varName:varNames1){ - if(o2.getNewFieldNames()!=null&&o2.getNewFieldNames().contains(varName)){ + List varNames1 = o1.getDependentFields(); + List varNames2 = o2.getDependentFields(); + for (String varName : varNames1) { + if (o2.getNewFieldNames() != null && o2.getNewFieldNames().contains(varName)) { return 1; } } - for(String varName:varNames2){ - if(o1.getNewFieldNames()!=null&&o1.getNewFieldNames().contains(varName)){ + for (String varName : varNames2) { + if (o1.getNewFieldNames() != null && o1.getNewFieldNames().contains(varName)) { return -1; } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/ScriptTreeNode.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/ScriptTreeNode.java index 2132e995..434aa75d 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/ScriptTreeNode.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/ScriptTreeNode.java @@ -20,7 +20,7 @@ import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; import org.apache.rocketmq.streams.common.topology.stages.ScriptChainStage; import org.apache.rocketmq.streams.filter.builder.ExpressionBuilder; import org.apache.rocketmq.streams.filter.function.script.CaseFunction; diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/SimplePipelineTree.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/SimplePipelineTree.java index 6e26bc09..b0b7aef7 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/SimplePipelineTree.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/SimplePipelineTree.java @@ -17,22 +17,22 @@ package org.apache.rocketmq.streams.filter.optimization.dependency; import java.util.List; -import java.util.Map; import java.util.Set; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; -import org.apache.rocketmq.streams.common.topology.model.Pipeline; +import org.apache.rocketmq.streams.common.topology.model.AbstractPipeline; import org.apache.rocketmq.streams.script.service.IScriptExpression; public class SimplePipelineTree { - protected Pipeline pipeline; + protected AbstractPipeline pipeline; protected TreeNode rootNode; protected TreeNode leafNode; - public SimplePipelineTree(Pipeline pipeline){ - this.pipeline=pipeline; + + public SimplePipelineTree(AbstractPipeline pipeline) { + this.pipeline = pipeline; } /** * trace root field dependent in this script + * * @param varName * @return */ diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/StateLessDependencyTree.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/StateLessDependencyTree.java index aedf8f97..f3f9a3ca 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/StateLessDependencyTree.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/StateLessDependencyTree.java @@ -16,36 +16,26 @@ */ package org.apache.rocketmq.streams.filter.optimization.dependency; -import java.util.ArrayList; -import java.util.Collection; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; -import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintCache; import org.apache.rocketmq.streams.common.optimization.fingerprint.PreFingerprint; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; -import org.apache.rocketmq.streams.common.topology.model.AbstractStage; -import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; -import org.apache.rocketmq.streams.common.topology.stages.ScriptChainStage; -import org.apache.rocketmq.streams.common.utils.CollectionUtil; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; /** * raverse the pipeline to create a prefix filter fingerprint */ -public class StateLessDependencyTree extends DependencyTree{ - public static Map> cache=new HashMap<>(); - +public class StateLessDependencyTree extends DependencyTree { + public static Map> cache = new HashMap<>(); protected ChainPipeline chainPipeline; - protected Map> preFingerprintExecutor=new HashMap<>(); + protected Map> preFingerprintExecutor = new HashMap<>(); + public StateLessDependencyTree(ChainPipeline pipeline) { - super(pipeline,null); + super(pipeline); this.chainPipeline = pipeline; } - /** * If the two pre filters are one branch, merge and replace the previous one with the latter one * The consolidation condition is that the following branches have no new fingerprint fields or more filtering conditions @@ -80,5 +70,4 @@ protected boolean mergePreFingerprint(PreFingerprint fingerprint, ChainPipeline return true; } - } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/TreeNode.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/TreeNode.java index 4f467f12..5adad81b 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/TreeNode.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/TreeNode.java @@ -22,8 +22,8 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.rocketmq.streams.common.metadata.MetaData; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; import org.apache.rocketmq.streams.common.topology.model.AbstractStage; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; import org.apache.rocketmq.streams.common.utils.CollectionUtil; import org.apache.rocketmq.streams.filter.operator.Rule; import org.apache.rocketmq.streams.filter.operator.expression.Expression; @@ -32,122 +32,125 @@ import org.apache.rocketmq.streams.script.service.IScriptExpression; public abstract class TreeNode { - protected List parents=new ArrayList<>(); - protected List children=new ArrayList<>(); + protected List parents = new ArrayList<>(); + protected List children = new ArrayList<>(); protected String name; protected ChainPipeline pipeline; protected T stage; protected MetaData soureMetaData; - public TreeNode(ChainPipeline pipeline,T stage,TreeNode parent){ - this.pipeline=pipeline; - this.stage=stage; - if(parent!=null){ + + public TreeNode(ChainPipeline pipeline, T stage, TreeNode parent) { + this.pipeline = pipeline; + this.stage = stage; + if (parent != null) { parents.add(parent); parent.getChildren().add(this); } - soureMetaData=pipeline.getChannelMetaData(); + soureMetaData = pipeline.getChannelMetaData(); } - public abstract Set traceaField(String varName, AtomicBoolean isBreak, List depenentScript) ; + public abstract Set traceaField(String varName, AtomicBoolean isBreak, List depenentScript); /** - *If the ETL procedure of the expression dependent variable can be traced to the source, the dependent expression is returned; otherwise, null is returned + * If the ETL procedure of the expression dependent variable can be traced to the source, the dependent expression is returned; otherwise, null is returned + * * @return */ public abstract List traceDepenentToSource(); - public boolean isSourceField(String varNames){ - return this.soureMetaData.getMetaDataField(varNames)!=null; + + public boolean isSourceField(String varNames) { + return this.soureMetaData.getMetaDataField(varNames) != null; } - public boolean traceDepenentToSource(TreeNode parent,CommonExpression commonExpression,String varName){ - AtomicBoolean isBreak=new AtomicBoolean(false); - List scriptExpressions=new ArrayList<>(); - Set varNames= parent.traceaField(varName,isBreak,scriptExpressions); - if(containsGroupScript(scriptExpressions)){ + public boolean traceDepenentToSource(TreeNode parent, CommonExpression commonExpression, String varName) { + AtomicBoolean isBreak = new AtomicBoolean(false); + List scriptExpressions = new ArrayList<>(); + Set varNames = parent.traceaField(varName, isBreak, scriptExpressions); + if (containsGroupScript(scriptExpressions)) { return false; } - if(isBreak.get()){ + if (isBreak.get()) { return false; } - if(parent.getStage().getPrevStageLabels().size()>1){ + if (parent.getStage().getPrevStageLabels().size() > 1) { return false; } - List treeNodes=parent.getParents(); - List parents=parent.getStage().getPrevStageLabels(); - if(CollectionUtil.isEmpty(parents)||CollectionUtil.isEmpty(treeNodes)){ + List treeNodes = parent.getParents(); + List parents = parent.getStage().getPrevStageLabels(); + if (CollectionUtil.isEmpty(parents) || CollectionUtil.isEmpty(treeNodes)) { commonExpression.addPreviewScriptDependent(scriptExpressions); return true; } - if(parents.size()>1){ + if (parents.size() > 1) { return false; } commonExpression.addPreviewScriptDependent(scriptExpressions); - for(String newDependentVarName:varNames){ - boolean success= traceDepenentToSource(treeNodes.get(0),commonExpression,newDependentVarName); - if(!success){ + for (String newDependentVarName : varNames) { + boolean success = traceDepenentToSource(treeNodes.get(0), commonExpression, newDependentVarName); + if (!success) { return false; } } return true; } - protected List traceDepenentToSource(Rule rule){ - Collection expressions= rule.getExpressionMap().values(); - List parents=this.getStage().getPrevStageLabels(); - List commonExpressions=new ArrayList<>(); - for(Expression expression:expressions){ - List commonExpressionList=traceDepenentToSource(expression); - if(commonExpressionList!=null){ + + protected List traceDepenentToSource(Rule rule) { + Collection expressions = rule.getExpressionMap().values(); + List parents = this.getStage().getPrevStageLabels(); + List commonExpressions = new ArrayList<>(); + for (Expression expression : expressions) { + List commonExpressionList = traceDepenentToSource(expression); + if (commonExpressionList != null) { commonExpressions.addAll(commonExpressionList); } } return commonExpressions; } - - protected List traceDepenentToSource(Expression expression){ - List parents=this.getStage().getPrevStageLabels(); - List commonExpressions=new ArrayList<>(); - if(RelationExpression.class.isInstance(expression)){ + protected List traceDepenentToSource(Expression expression) { + List parents = this.getStage().getPrevStageLabels(); + List commonExpressions = new ArrayList<>(); + if (RelationExpression.class.isInstance(expression)) { return null; } - String functionName=expression.getFunctionName(); - if(functionName==null){ + String functionName = expression.getFunctionName(); + if (functionName == null) { return null; } - if(CommonExpression.support(expression)){ - if(parents==null||this.getParents()==null||this.getParents().size()==0){ - CommonExpression commonExpression=new CommonExpression(expression); + if (CommonExpression.support(expression)) { + if (parents == null || this.getParents() == null || this.getParents().size() == 0) { + CommonExpression commonExpression = new CommonExpression(expression); commonExpressions.add(commonExpression); return commonExpressions; } - if(parents.size()>1){ + if (parents.size() > 1) { throw new RuntimeException("can not support mutil parent for filter stage"); } - TreeNode parent=this.getParents().get(0); - CommonExpression commonExpression=traceDepenentToSource(parent,expression); - if(commonExpression!=null){ + TreeNode parent = this.getParents().get(0); + CommonExpression commonExpression = traceDepenentToSource(parent, expression); + if (commonExpression != null) { commonExpressions.add(commonExpression); } } return commonExpressions; } - protected CommonExpression traceDepenentToSource(TreeNode parent,Expression expression){ - if(!CommonExpression.support(expression)){ + + protected CommonExpression traceDepenentToSource(TreeNode parent, Expression expression) { + if (!CommonExpression.support(expression)) { return null; } - CommonExpression commonExpression=new CommonExpression(expression); - if(parent==null){ + CommonExpression commonExpression = new CommonExpression(expression); + if (parent == null) { return commonExpression; } - traceDepenentToSource(parent,commonExpression,expression.getVarName()); + traceDepenentToSource(parent, commonExpression, expression.getVarName()); return commonExpression; } - protected boolean containsGroupScript(List expressions) { - for(IScriptExpression scriptExpression:expressions){ - if(GroupScriptExpression.class.isInstance(scriptExpression)){ + for (IScriptExpression scriptExpression : expressions) { + if (GroupScriptExpression.class.isInstance(scriptExpression)) { return true; } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/executor/AbstractExecutor.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/executor/AbstractExecutor.java index d6627cb1..7362e9a7 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/executor/AbstractExecutor.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/executor/AbstractExecutor.java @@ -20,7 +20,6 @@ import org.apache.rocketmq.streams.script.optimization.performance.IScriptOptimization; public abstract class AbstractExecutor implements IScriptOptimization.IOptimizationExecutor { - protected static RegexFunction regexFunction=new RegexFunction();// execute directly, not use reflect - + protected static RegexFunction regexFunction = new RegexFunction();// execute directly, not use reflect } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/executor/GroupByVarExecutor.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/executor/GroupByVarExecutor.java index f5e22a10..01036c21 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/executor/GroupByVarExecutor.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/executor/GroupByVarExecutor.java @@ -47,78 +47,79 @@ public class GroupByVarExecutor extends AbstractExecutor implements IScriptOptim protected String name; protected List scriptExpressions; //execute before regex execute - protected List beforeScriptExpressions=new ArrayList<>(); + protected List beforeScriptExpressions = new ArrayList<>(); + protected transient Map varName2Scripts = new HashMap<>(); + protected transient Map> varName2DependentFields = new HashMap<>(); + protected Map varName2HyperscanExecutors = new HashMap<>(); - protected transient Map varName2Scripts=new HashMap<>(); - protected transient Map> varName2DependentFields=new HashMap<>(); - public GroupByVarExecutor(String namespace,String name,List expressions){ - this.name=name; - this.namespace=namespace; - this.scriptExpressions=expressions; + public GroupByVarExecutor(String namespace, String name, List expressions) { + this.name = name; + this.namespace = namespace; + this.scriptExpressions = expressions; } - protected Map varName2HyperscanExecutors=new HashMap<>(); + @Override public FilterResultCache execute(IMessage message, AbstractContext context) { - if(CollectionUtil.isNotEmpty(beforeScriptExpressions)){ - for(IScriptExpression scriptExpression:this.beforeScriptExpressions){ - scriptExpression.executeExpression(message,(FunctionContext) context); + if (CollectionUtil.isNotEmpty(beforeScriptExpressions)) { + for (IScriptExpression scriptExpression : this.beforeScriptExpressions) { + scriptExpression.executeExpression(message, (FunctionContext) context); } } - Map expression2QuickFilterResultIndexMap=new HashMap<>(); - List results=new ArrayList<>(); - int index=0; - for(HyperscanExecutor executor:varName2HyperscanExecutors.values()){ - FilterResultCache result= executor.execute(message,context); - if(result==null){ + Map expression2QuickFilterResultIndexMap = new HashMap<>(); + List results = new ArrayList<>(); + int index = 0; + for (HyperscanExecutor executor : varName2HyperscanExecutors.values()) { + FilterResultCache result = executor.execute(message, context); + if (result == null) { continue; } results.add(result); - Map expressionIndexMap= result.getExpression2IndexMap(); - for(String key:expressionIndexMap.keySet()){ - expression2QuickFilterResultIndexMap.put(key,index); + Map expressionIndexMap = result.getExpression2IndexMap(); + for (String key : expressionIndexMap.keySet()) { + expression2QuickFilterResultIndexMap.put(key, index); } index++; } - return new GroupQuickFilterResult(expression2QuickFilterResultIndexMap,results); + return new GroupQuickFilterResult(expression2QuickFilterResultIndexMap, results); } - public void addExpression(IScriptExpression expression){ - if(expression.getNewFieldNames()!=null&&expression.getNewFieldNames().size()==1){ - String newFieldName=expression.getNewFieldNames().iterator().next(); - varName2Scripts.put(newFieldName,expression); - varName2DependentFields.put(newFieldName,expression.getDependentFields()); + public void addExpression(IScriptExpression expression) { + if (expression.getNewFieldNames() != null && expression.getNewFieldNames().size() == 1) { + String newFieldName = expression.getNewFieldNames().iterator().next(); + varName2Scripts.put(newFieldName, expression); + varName2DependentFields.put(newFieldName, expression.getDependentFields()); } - if(RegexFunction.isRegexFunction(expression.getFunctionName())){ - this.regist(IScriptOptimization.getParameterValue((IScriptParamter) expression.getScriptParamters().get(0)),expression); - }else if(expression instanceof GroupScriptExpression){ - GroupScriptExpression groupScriptExpression=(GroupScriptExpression)expression; - List caseExpressions=new ArrayList<>(); + if (RegexFunction.isRegexFunction(expression.getFunctionName())) { + this.regist(IScriptOptimization.getParameterValue((IScriptParamter) expression.getScriptParamters().get(0)), expression); + } else if (expression instanceof GroupScriptExpression) { + GroupScriptExpression groupScriptExpression = (GroupScriptExpression) expression; + List caseExpressions = new ArrayList<>(); caseExpressions.add(groupScriptExpression.getIfExpresssion()); - if(groupScriptExpression.getElseIfExpressions()!=null){ - for(GroupScriptExpression elseIf:groupScriptExpression.getElseIfExpressions()){ + if (groupScriptExpression.getElseIfExpressions() != null) { + for (GroupScriptExpression elseIf : groupScriptExpression.getElseIfExpressions()) { caseExpressions.add(elseIf.getIfExpresssion()); } } - for(IScriptExpression scriptExpression:caseExpressions){ - if(CaseFunction.isCaseFunction(scriptExpression.getFunctionName())){ - String expressionStr=IScriptOptimization.getParameterValue((IScriptParamter) scriptExpression.getScriptParamters().get(0)); - if(StringUtil.isEmpty(expressionStr)){ - continue; + for (IScriptExpression scriptExpression : caseExpressions) { + if (CaseFunction.isCaseFunction(scriptExpression.getFunctionName())) { + String expressionStr = IScriptOptimization.getParameterValue((IScriptParamter) scriptExpression.getScriptParamters().get(0)); + if (StringUtil.isEmpty(expressionStr)) { + continue; } - List expressions=new ArrayList<>(); - ExpressionBuilder.createExpression("tmp","tmp",expressionStr,expressions,new ArrayList<>()); - if(expressions.size()>0){ - for(Expression simpleExpression:expressions){ - this.regist(simpleExpression.getVarName(),simpleExpression); + List expressions = new ArrayList<>(); + ExpressionBuilder.createExpression("tmp", "tmp", expressionStr, expressions, new ArrayList<>()); + if (expressions.size() > 0) { + for (Expression simpleExpression : expressions) { + this.regist(simpleExpression.getVarName(), simpleExpression); } } - }else if(RuleExpression.class.isInstance(scriptExpression)){ - RuleExpression ruleExpression=(RuleExpression)scriptExpression; - Rule rule=ruleExpression.getRule(); - for(Expression expression1:rule.getExpressionMap().values()){ - if(!RelationExpression.class.isInstance(expression1)){ - this.regist(expression1.getVarName(),expression1); + } else if (RuleExpression.class.isInstance(scriptExpression)) { + RuleExpression ruleExpression = (RuleExpression) scriptExpression; + Rule rule = ruleExpression.getRule(); + for (Expression expression1 : rule.getExpressionMap().values()) { + if (!RelationExpression.class.isInstance(expression1)) { + this.regist(expression1.getVarName(), expression1); } } } @@ -127,24 +128,24 @@ public void addExpression(IScriptExpression expression){ } public List compile() { - for(IScriptExpression scriptExpression:this.scriptExpressions){ + for (IScriptExpression scriptExpression : this.scriptExpressions) { this.addExpression(scriptExpression); } - List scriptExpressionList=new ArrayList<>(); - for(IScriptExpression scriptExpression:this.scriptExpressions){ - if(!this.beforeScriptExpressions.contains(scriptExpression)){ + List scriptExpressionList = new ArrayList<>(); + for (IScriptExpression scriptExpression : this.scriptExpressions) { + if (!this.beforeScriptExpressions.contains(scriptExpression)) { scriptExpressionList.add(scriptExpression); } } - Map map=new HashMap<>(); - for(String key:varName2HyperscanExecutors.keySet()){ - HyperscanExecutor hyperscanExecutor=varName2HyperscanExecutors.get(key); - if(hyperscanExecutor.allExpressions.size()>5){ - map.put(key,hyperscanExecutor); + Map map = new HashMap<>(); + for (String key : varName2HyperscanExecutors.keySet()) { + HyperscanExecutor hyperscanExecutor = varName2HyperscanExecutors.get(key); + if (hyperscanExecutor.allExpressions.size() > 5) { + map.put(key, hyperscanExecutor); hyperscanExecutor.compile(); } } - this.varName2HyperscanExecutors=map; + this.varName2HyperscanExecutors = map; return scriptExpressionList; } @@ -157,67 +158,66 @@ public void setScriptExpressions( this.scriptExpressions = scriptExpressions; } + protected void regist(String varName, Object expression) { - protected void regist(String varName,Object expression){ - - List dependents= addBeforeExpression(varName); - if(dependents==null){ + List dependents = addBeforeExpression(varName); + if (dependents == null) { return; } Collections.sort(dependents, new Comparator() { @Override public int compare(IScriptExpression o1, IScriptExpression o2) { - List varNames1= o1.getDependentFields(); - List varNames2=o2.getDependentFields(); - for(String varName:varNames1){ - if(o2.getNewFieldNames()!=null&&o2.getNewFieldNames().contains(varName)){ + List varNames1 = o1.getDependentFields(); + List varNames2 = o2.getDependentFields(); + for (String varName : varNames1) { + if (o2.getNewFieldNames() != null && o2.getNewFieldNames().contains(varName)) { return 1; } } - for(String varName:varNames2){ - if(o1.getNewFieldNames()!=null&&o1.getNewFieldNames().contains(varName)){ + for (String varName : varNames2) { + if (o1.getNewFieldNames() != null && o1.getNewFieldNames().contains(varName)) { return -1; } } return 0; } }); - for(IScriptExpression scriptExpression:dependents){ - if(!this.beforeScriptExpressions.contains(scriptExpression)){ + for (IScriptExpression scriptExpression : dependents) { + if (!this.beforeScriptExpressions.contains(scriptExpression)) { this.beforeScriptExpressions.add(scriptExpression); } } - HyperscanExecutor hyperscanExecutor=varName2HyperscanExecutors.get(varName); - if(hyperscanExecutor==null){ - hyperscanExecutor=new HyperscanExecutor(namespace,name); - hyperscanExecutor.varName=varName; - varName2HyperscanExecutors.put(varName,hyperscanExecutor); + HyperscanExecutor hyperscanExecutor = varName2HyperscanExecutors.get(varName); + if (hyperscanExecutor == null) { + hyperscanExecutor = new HyperscanExecutor(namespace, name); + hyperscanExecutor.varName = varName; + varName2HyperscanExecutors.put(varName, hyperscanExecutor); } hyperscanExecutor.addExpression(expression); } protected List addBeforeExpression(String varName) { - IScriptExpression scriptExpression= this.varName2Scripts.get(varName); - if(scriptExpression==null){ - return new ArrayList<>(); - } - if(!ToLowerFunction.isLowFunction(scriptExpression.getFunctionName())){ - return null; - } - List list=new ArrayList<>(); - list.add(scriptExpression); - //this.beforeScriptExpressions.add(scriptExpression); - List fields=scriptExpression.getDependentFields(); - if(fields!=null){ - for(String fieldName:fields){ - List dependents=addBeforeExpression(fieldName); - if(dependents==null){ + IScriptExpression scriptExpression = this.varName2Scripts.get(varName); + if (scriptExpression == null) { + return new ArrayList<>(); + } + if (!ToLowerFunction.isLowFunction(scriptExpression.getFunctionName())) { + return null; + } + List list = new ArrayList<>(); + list.add(scriptExpression); + //this.beforeScriptExpressions.add(scriptExpression); + List fields = scriptExpression.getDependentFields(); + if (fields != null) { + for (String fieldName : fields) { + List dependents = addBeforeExpression(fieldName); + if (dependents == null) { return null; - }else { + } else { list.addAll(dependents); } - } - } - return list; + } + } + return list; } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/homologous/HomologousCompute.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/homologous/HomologousCompute.java index f9b2f16d..9051bed3 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/homologous/HomologousCompute.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/homologous/HomologousCompute.java @@ -25,6 +25,7 @@ import org.apache.rocketmq.streams.common.cache.compress.BitSetCache; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.optimization.IHomologousCalculate; import org.apache.rocketmq.streams.common.optimization.LikeRegex; import org.apache.rocketmq.streams.common.optimization.RegexEngine; import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintCache; @@ -40,7 +41,7 @@ /** * Advance the regular and like expression of homologous rules and make global optimization */ -public class HomologousCompute { +public class HomologousCompute implements IHomologousCalculate { protected transient static String MUTIL_BLINK = "\\s+"; protected transient List commonExpressions;//all expression /** @@ -62,27 +63,25 @@ public class HomologousCompute { * cache like regex key:sourceVarName */ protected transient Map likeRegexMap = new HashMap<>(); - - private transient List notsupportExpressions = new ArrayList<>(); - /** * cache of finger cache */ protected transient FingerprintCache fingerprintCache; + protected transient Long firstReceiveTime = null; + protected transient AtomicLong COUNT = new AtomicLong(0); + protected transient RegexFunction replaceFunction = new RegexFunction(); + private transient List notsupportExpressions = new ArrayList<>(); - public HomologousCompute(List commonExpressions, int cacheSize) { + public HomologousCompute(List commonExpressions) { this.commonExpressions = commonExpressions; Map groupBySourceVarName = groupBySourceVarName(commonExpressions); this.groupBySourceVarName = groupBySourceVarName; -// registHyperscan(groupBySourceVarName); + registHyperscan(groupBySourceVarName); createExpressionIndexAndHomologousVar(groupBySourceVarName); - fingerprintCache = new FingerprintCache(cacheSize); + fingerprintCache = FingerprintCache.getInstance(); } - protected transient Long firstReceiveTime = null; - protected transient AtomicLong COUNT = new AtomicLong(0); - protected transient RegexFunction replaceFunction = new RegexFunction(); - + @Override public void calculate(IMessage message, AbstractContext context) { printQPS(); FunctionContext functionContext = new FunctionContext(message); @@ -282,6 +281,10 @@ private void printQPS() { } + public Map getGroupBySourceVarName() { + return groupBySourceVarName; + } + protected class SameVarExpressionGroup { protected List regexs = new ArrayList<>(); protected List others = new ArrayList<>(); @@ -312,8 +315,4 @@ public int size() { return regexs.size() + others.size(); } } - - public Map getGroupBySourceVarName() { - return groupBySourceVarName; - } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/homologous/HomologousOptimization.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/homologous/HomologousOptimization.java index f5542a14..c988d4d2 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/homologous/HomologousOptimization.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/homologous/HomologousOptimization.java @@ -22,53 +22,113 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.rocketmq.streams.common.context.AbstractContext; -import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.optimization.IHomologousCalculate; import org.apache.rocketmq.streams.common.optimization.IHomologousOptimization; -import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintCache; import org.apache.rocketmq.streams.common.optimization.fingerprint.PreFingerprint; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; +import org.apache.rocketmq.streams.common.topology.metric.NotFireReason; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; +import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; +import org.apache.rocketmq.streams.common.utils.IdUtil; import org.apache.rocketmq.streams.common.utils.JsonableUtil; +import org.apache.rocketmq.streams.filter.operator.Rule; +import org.apache.rocketmq.streams.filter.operator.expression.Expression; +import org.apache.rocketmq.streams.filter.operator.expression.RelationExpression; import org.apache.rocketmq.streams.filter.optimization.dependency.CommonExpression; import org.apache.rocketmq.streams.filter.optimization.dependency.DependencyTree; +import org.apache.rocketmq.streams.script.service.IScriptExpression; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @AutoService(IHomologousOptimization.class) public class HomologousOptimization implements IHomologousOptimization { - protected transient HomologousCompute homologousCompute; - protected static Map homologousComputeCache=new HashMap<>(); + + private static final Logger LOGGER = LoggerFactory.getLogger(HomologousOptimization.class); + protected List commonExpressions; + private Map dependencyTreeMap = new HashMap<>(); @Override - public void optimizate(List> pipelines, int cacheSize, int preFingerprintCacheSize) { + public void optimizate(List> pipelines) { List commonExpressions = new ArrayList<>(); - FingerprintCache fingerprintCache = new FingerprintCache(preFingerprintCacheSize); for (ChainPipeline pipeline : pipelines) { - DependencyTree dependencyTree = new DependencyTree(pipeline, fingerprintCache); + DependencyTree dependencyTree = new DependencyTree(pipeline); List commonExpressionList = dependencyTree.parse(); if (commonExpressionList != null) { commonExpressions.addAll(commonExpressionList); } - printOptimizatePipeline(pipeline); - } - homologousCompute = new HomologousCompute(commonExpressions, cacheSize); - for(ChainPipeline chainPipeline:pipelines){ - homologousComputeCache.put(chainPipeline,homologousCompute); + dependencyTreeMap.put(pipeline, dependencyTree); + printOptimizePipeline(pipeline); } - } + this.commonExpressions = commonExpressions; - public HomologousCompute getHomologousCompute(ChainPipeline chainPipeline){ - return homologousComputeCache.get(chainPipeline); } + @Override public IHomologousCalculate createHomologousCalculate() { + HomologousCompute homologousCompute = null; + homologousCompute = new HomologousCompute(commonExpressions); + return homologousCompute; + } - @Override - public void calculate(IMessage message, AbstractContext context) { - homologousCompute.calculate(message, context); + @Override public void setPeFingerprintForPipeline(ChainPipeline pipeline) { + /** + * Create prefix fingerprint objects by branch and Merge branch + * + */ + DependencyTree dependencyTree = dependencyTreeMap.get(pipeline); + pipeline.setPreFingerprintExecutor(dependencyTree.getPreFingerprintExecutor()); + } + + @Override public NotFireReason analysisNotFireReason(FilterChainStage stage, String fieldValue, List notFireExpressionMonitor) { + NotFireReason notFireReason = new NotFireReason(stage, fieldValue); + if (notFireReason != null) { + ChainPipeline chainPipeline = (ChainPipeline) stage.getPipeline(); + DependencyTree dependencyTree = dependencyTreeMap.get(chainPipeline); + List commonExpressions = dependencyTree.getCommonExpressions(); + Map> filterFieldName2ETLScriptList = new HashMap<>(); + Map filterFieldName2OriFieldName = new HashMap<>(); + for (CommonExpression commonExpression : commonExpressions) { + String filterFieldName = commonExpression.getVarName(); + String origFieldName = commonExpression.getSourceVarName(); + filterFieldName2OriFieldName.put(filterFieldName, origFieldName); + List etlScript = filterFieldName2ETLScriptList.get(filterFieldName); + if (etlScript == null) { + etlScript = new ArrayList<>(); + filterFieldName2ETLScriptList.put(filterFieldName, etlScript); + } + for (IScriptExpression scriptExpression : commonExpression.getScriptExpressions()) { + if (!etlScript.contains(scriptExpression.toString())) { + etlScript.add(scriptExpression.toString()); + } + } + } + List filterFieldNames = new ArrayList<>(); + List expressions = new ArrayList<>(); + List expressionNames = notFireExpressionMonitor; + Rule rule = (Rule) stage.getRule(); + for (String expressionName : expressionNames) { + Expression expression = rule.getExpressionMap().get(expressionName); + if (expression == null) { + expressions.add(expressionName); + } else if (RelationExpression.class.isInstance(expression)) { + expressions.add(expression.toExpressionString(rule.getExpressionMap())); + filterFieldNames.addAll(expression.getDependentFields(rule.getExpressionMap())); + } else { + filterFieldNames.add(expression.getVarName()); + expressions.add(expression.toExpressionString(rule.getExpressionMap())); + } + } + notFireReason.setFilterFieldName2ETLScriptList(filterFieldName2ETLScriptList); + notFireReason.setFilterFieldName2OriFieldName(filterFieldName2OriFieldName); + notFireReason.setExpressions(expressions); + notFireReason.setFilterFieldNames(filterFieldNames); + return notFireReason; + } + return null; } - protected void printOptimizatePipeline(ChainPipeline pipeline) { - System.out.println(pipeline.getConfigureName() + " finish optimizate, the detail is :"); - Map> prefingers = pipeline.getPreFingerprintExecutor(); + protected void printOptimizePipeline(ChainPipeline pipeline) { + DependencyTree dependencyTree = dependencyTreeMap.get(pipeline); + Map> prefingers = dependencyTree.getPreFingerprintExecutor(); JSONObject detail = new JSONObject(); for (String prefinger : prefingers.keySet()) { Map branchs = prefingers.get(prefinger); @@ -77,7 +137,7 @@ protected void printOptimizatePipeline(ChainPipeline pipeline) { detail.put("prefiger." + (prefinger.equals(pipeline.getChannelName()) ? "source" : prefinger), preFingerprint.getLogFingerFieldNames()); } } - System.out.println(JsonableUtil.formatJson(detail)); + LOGGER.info("[{}][{}] Finish_Optimize_Detail_Is_[{}]", IdUtil.instanceId(), pipeline.getName(), JsonableUtil.formatJson(detail)); } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/result/GroupQuickFilterResult.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/result/GroupQuickFilterResult.java index 96970b4e..84ba394f 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/result/GroupQuickFilterResult.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/result/GroupQuickFilterResult.java @@ -30,41 +30,41 @@ import org.apache.rocketmq.streams.script.service.IScriptExpression; import org.apache.rocketmq.streams.script.service.IScriptParamter; -public class GroupQuickFilterResult extends FilterResultCache { - Map expression2QuickFilterResultIndexMap=new HashMap<>(); - List results=new ArrayList<>(); - public GroupQuickFilterResult(Map expression2QuickFilterResultIndexMap, List results){ - super(null,null); - this.expression2QuickFilterResultIndexMap=expression2QuickFilterResultIndexMap; - this.results=results; +public class GroupQuickFilterResult extends FilterResultCache { + Map expression2QuickFilterResultIndexMap = new HashMap<>(); + List results = new ArrayList<>(); + + public GroupQuickFilterResult(Map expression2QuickFilterResultIndexMap, List results) { + super(null, null); + this.expression2QuickFilterResultIndexMap = expression2QuickFilterResultIndexMap; + this.results = results; } @Override public Boolean isMatch(IMessage msg, Object expression) { - String key=null; - if(expression instanceof IScriptExpression){ - if(RegexFunction.isRegexFunction(((IScriptExpression) expression).getFunctionName())){ - IScriptExpression scriptExpression=(IScriptExpression)expression; - String varName= IScriptOptimization.getParameterValue((IScriptParamter)scriptExpression.getScriptParamters().get(0)); - String regex=IScriptOptimization.getParameterValue((IScriptParamter)scriptExpression.getScriptParamters().get(1)); - key= MapKeyUtil.createKey(varName,scriptExpression.getFunctionName(),regex); + String key = null; + if (expression instanceof IScriptExpression) { + if (RegexFunction.isRegexFunction(((IScriptExpression) expression).getFunctionName())) { + IScriptExpression scriptExpression = (IScriptExpression) expression; + String varName = IScriptOptimization.getParameterValue((IScriptParamter) scriptExpression.getScriptParamters().get(0)); + String regex = IScriptOptimization.getParameterValue((IScriptParamter) scriptExpression.getScriptParamters().get(1)); + key = MapKeyUtil.createKey(varName, scriptExpression.getFunctionName(), regex); } - }else if(expression instanceof Expression){ - Expression filterExpression=(Expression)expression; - if(LikeFunction.isLikeFunciton(filterExpression.getFunctionName())|| org.apache.rocketmq.streams.filter.function.expression.RegexFunction.isRegex(filterExpression.getFunctionName())){ - key= MapKeyUtil.createKey(filterExpression.getVarName(),filterExpression.getFunctionName(),(String)filterExpression.getValue()); + } else if (expression instanceof Expression) { + Expression filterExpression = (Expression) expression; + if (LikeFunction.isLikeFunciton(filterExpression.getFunctionName()) || org.apache.rocketmq.streams.filter.function.expression.RegexFunction.isRegex(filterExpression.getFunctionName())) { + key = MapKeyUtil.createKey(filterExpression.getVarName(), filterExpression.getFunctionName(), (String) filterExpression.getValue()); } } - if(key!=null){ - Integer index= expression2QuickFilterResultIndexMap.get(key); - if(index==null){ + if (key != null) { + Integer index = expression2QuickFilterResultIndexMap.get(key); + if (index == null) { return null; } - FilterResultCache quickFilterResult=results.get(index); - return quickFilterResult.isMatch(msg,expression); + FilterResultCache quickFilterResult = results.get(index); + return quickFilterResult.isMatch(msg, expression); } return null; } - } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/script/ScriptOptimization.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/script/ScriptOptimization.java index 7b2f3103..547a490c 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/script/ScriptOptimization.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/script/ScriptOptimization.java @@ -27,8 +27,8 @@ import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintCache; import org.apache.rocketmq.streams.filter.operator.expression.Expression; +import org.apache.rocketmq.streams.filter.optimization.casewhen.IFExpressionOptimization; import org.apache.rocketmq.streams.filter.optimization.dependency.BlinkRuleV2Expression; -import org.apache.rocketmq.streams.filter.optimization.executor.GroupByVarExecutor; import org.apache.rocketmq.streams.script.context.FunctionContext; import org.apache.rocketmq.streams.script.operator.impl.FunctionScript; import org.apache.rocketmq.streams.script.optimization.performance.IScriptOptimization; @@ -38,17 +38,14 @@ @AutoService(IScriptOptimization.class) public class ScriptOptimization implements IScriptOptimization { - public static interface IExpressionExecutor { - boolean execute(IMessage message, AbstractContext context); - } - @Override public IOptimizationCompiler compile(List expressions, IConfigurableIdentification configurableIdentification) { if (expressions != null) { - GroupByVarExecutor groupByVarExecutor = new GroupByVarExecutor(configurableIdentification.getNameSpace(), configurableIdentification.getConfigureName(), expressions); - expressions = blinkRuleV2ExpressionOptimizate(expressions, configurableIdentification); - groupByVarExecutor.setScriptExpressions(expressions); - return groupByVarExecutor; +// GroupByVarExecutor groupByVarExecutor = new GroupByVarExecutor(configurableIdentification.getNameSpace(), configurableIdentification.getConfigureName(), expressions); +// expressions = blinkRuleV2ExpressionOptimizate(expressions, configurableIdentification); +// groupByVarExecutor.setScriptExpressions(expressions); +// return groupByVarExecutor; + return new IFExpressionOptimization(configurableIdentification.getNameSpace(), configurableIdentification.getName(), expressions); } return null; } @@ -62,7 +59,7 @@ private List blinkRuleV2ExpressionOptimizate(List blinkRuleV2ExpressionOptimizate(List { + boolean execute(IMessage message, AbstractContext context); + } + public static class BlinkRuleV2Exprssion implements IScriptExpression { protected IScriptExpression ori; protected BlinkRuleV2Expression.RuleSetGroup group; @@ -93,7 +94,7 @@ public BlinkRuleV2Exprssion(IScriptExpression ori, BlinkRuleV2Expression.RuleSet this.ori = ori; this.group = group; ruleSetByCoreVarNames = group.optimizate(); - fingerprintCache = new FingerprintCache(1000000); + fingerprintCache = FingerprintCache.getInstance(); } @Override public Integer executeExpression(IMessage message, FunctionContext context) { diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/service/IRuleEngineService.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/service/IRuleEngineService.java index cb3fd6c3..89d34d76 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/service/IRuleEngineService.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/service/IRuleEngineService.java @@ -26,7 +26,6 @@ */ public interface IRuleEngineService extends IFilterService { - /** * 执行规则,返回触发的规则 * diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/service/impl/RuleEngineServiceImpl.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/service/impl/RuleEngineServiceImpl.java index 585be14a..dc15cf6e 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/service/impl/RuleEngineServiceImpl.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/service/impl/RuleEngineServiceImpl.java @@ -20,15 +20,12 @@ import java.io.Serializable; import java.util.ArrayList; import java.util.List; -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.Context; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.Message; import org.apache.rocketmq.streams.filter.builder.RuleBuilder; import org.apache.rocketmq.streams.filter.context.ContextConfigure; -import org.apache.rocketmq.streams.filter.context.RuleContext; import org.apache.rocketmq.streams.filter.engine.IRuleEngine; import org.apache.rocketmq.streams.filter.engine.impl.DefaultRuleEngine; import org.apache.rocketmq.streams.filter.operator.Rule; @@ -40,7 +37,6 @@ public class RuleEngineServiceImpl implements IRuleEngineService, Serializable { private static final long serialVersionUID = 5932020315482204865L; - private static final Log LOG = LogFactory.getLog(RuleEngineServiceImpl.class); protected IRuleEngine ruleEngine = new DefaultRuleEngine(); private ContextConfigure contextConfigure = new ContextConfigure(null); @@ -48,7 +44,6 @@ public class RuleEngineServiceImpl implements IRuleEngineService, Serializable { // @PostConstruct public void initRuleContext(ContextConfigure contextConfigure) { this.contextConfigure = contextConfigure; - RuleContext.initSuperRuleContext(contextConfigure); } @Override @@ -57,6 +52,11 @@ public List excuteRule(JSONObject message, Rule... rules) { return this.executeRule(msg, new Context(msg), rules); } + @Override + public List executeRule(IMessage message, AbstractContext context, Rule rule) { + return executeRule(message, context, new Rule[] {rule}); + } + @Override public List executeRule(IMessage message, AbstractContext context, Rule... rules) { if (rules == null || rules.length == 0) { @@ -89,7 +89,7 @@ public List excuteRule(JSONObject message, List rules) { @Override public Rule createRule(String namespace, String ruleName, String expressionStr, String... msgMetaInfo) { RuleBuilder ruleCreator = new RuleBuilder(namespace, ruleName, expressionStr, msgMetaInfo); - Rule rule = ruleCreator.generateRule(null); + Rule rule = ruleCreator.generateRule(); return rule; } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/utils/IPUtil.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/utils/IPUtil.java index 854401de..f68e256d 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/utils/IPUtil.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/utils/IPUtil.java @@ -21,22 +21,20 @@ import java.util.Enumeration; import java.util.regex.Matcher; import java.util.regex.Pattern; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class IPUtil { - private static final Log LOG = LogFactory.getLog(IPUtil.class); - - private static volatile InetAddress LOCAL_ADDRESS = null; - private static final Pattern IP_PATTERN = Pattern.compile("\\d{1,3}(\\.\\d{1,3}){3,5}$"); - public static final String LOCALHOST = "127.0.0.1"; public static final String ANYHOST = "0.0.0.0"; + private static final Logger LOGGER = LoggerFactory.getLogger(IPUtil.class); + private static final Pattern IP_PATTERN = Pattern.compile("\\d{1,3}(\\.\\d{1,3}){3,5}$"); private static final Pattern pattern = Pattern.compile( "^(1\\d{2}|2[0-4]\\d|25[0-5]|[1-9]\\d|[1-9])" + "\\.(1\\d{2}|2[0-4]\\d|25[0-5]|[1-9]\\d|\\d)" + "\\.(1\\d{2}|2[0-4]\\d|25[0-5]|[1-9]\\d|\\d)" + "\\.(1\\d{2}|2[0-4]\\d|25[0-5]|[1-9]\\d|\\d)$"); - private static final long ALLONES = (long)(Math.pow(2, 32) - 1); + private static final long ALLONES = (long) (Math.pow(2, 32) - 1); + private static volatile InetAddress LOCAL_ADDRESS = null; public static String getLocalIP() { InetAddress address = getLocalIPAddress(); @@ -60,7 +58,7 @@ private static InetAddress getLocalAddress0() { return localAddress; } } catch (Throwable e) { - LOG.warn("Failed to retrieving ip address, " + e.getMessage(), e); + LOGGER.warn("Failed to retrieving ip address, " + e.getMessage(), e); } try { Enumeration interfaces = NetworkInterface.getNetworkInterfaces(); @@ -77,19 +75,19 @@ private static InetAddress getLocalAddress0() { return address; } } catch (Throwable e) { - LOG.warn("Failed to retrieving ip address, " + e.getMessage(), e); + LOGGER.warn("Failed to retrieving ip address, " + e.getMessage(), e); } } } } catch (Throwable e) { - LOG.warn("Failed to retrieving ip address, " + e.getMessage(), e); + LOGGER.warn("Failed to retrieving ip address, " + e.getMessage(), e); } } } } catch (Throwable e) { - LOG.warn("Failed to retrieving ip address, " + e.getMessage(), e); + LOGGER.warn("Failed to retrieving ip address, " + e.getMessage(), e); } - LOG.error("Could not get local host ip address, will use 127.0.0.1 instead."); + LOGGER.error("Could not get local host ip address, will use 127.0.0.1 instead."); return localAddress; } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/utils/RegexUtil.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/utils/RegexUtil.java index d7657301..b7c9e7df 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/utils/RegexUtil.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/utils/RegexUtil.java @@ -29,47 +29,19 @@ public class RegexUtil { // private static final long MAX_TIMEOUT = 1000;// 如果没有设置超时时间,最大的超时时间默认为1s - private static class InterruptibleCharSequence implements CharSequence { - - private CharSequence inner; - private long timeout; - private boolean isStart = false; - private long startTime = -1; - - public InterruptibleCharSequence(CharSequence inner, long timeout) { - super(); - this.inner = inner; - this.timeout = timeout; - } - - @Override - public int length() { - return inner.length(); - } - - @Override - public char charAt(int index) { - if (!isStart) { - isStart = true; - startTime = System.currentTimeMillis(); - } - if (System.currentTimeMillis() - startTime > timeout && timeout > 0) { - throw new RegexTimeoutException("正则执行超时:" + inner.toString()); - } - return inner.charAt(index); - - } - - @Override - public CharSequence subSequence(int start, int end) { - return new InterruptibleCharSequence(inner.subSequence(start, end), timeout); - } - - @Override - public String toString() { - return inner.toString(); - } - } + private static final String SPLIT_STR = + "\\d|\\w|\\s|\\|\\$|\\(|\\)|\\*|\\+|\\.|\\[|\\]|\\?|\\^|\\{|\\}|\\|"; + /** + * 根据要匹配的字符串,匹配模式字符串,是否忽略大小写和超时时间创建matcher + * + * @param content 待匹配的字符串 + * @param patternStr 匹配模式 + * @param caseInsensitive 是否忽略大小写 + * @param timeout 超时时间 + * @return + */ + private static Map pattern2MatcherForCaseInsensitive = new HashMap<>(); + private static Map pattern2Matcher = new HashMap<>(); /** * content为字符串,configure为pattern @@ -125,9 +97,6 @@ public static String groupRegex(String content, String patternStr, boolean caseI } } - private static final String SPLIT_STR = - "\\d|\\w|\\s|\\|\\$|\\(|\\)|\\*|\\+|\\.|\\[|\\]|\\?|\\^|\\{|\\}|\\|"; - public static List compilePattern(String regex) { List strs = new ArrayList<>(); // List keywords = new ArrayList(); @@ -143,18 +112,6 @@ public static List compilePattern(String regex) { return strs; } - /** - * 根据要匹配的字符串,匹配模式字符串,是否忽略大小写和超时时间创建matcher - * - * @param content 待匹配的字符串 - * @param patternStr 匹配模式 - * @param caseInsensitive 是否忽略大小写 - * @param timeout 超时时间 - * @return - */ - private static Map pattern2MatcherForCaseInsensitive = new HashMap<>(); - private static Map pattern2Matcher = new HashMap<>(); - private static Matcher createMatcher(String content, String patternStr, boolean caseInsensitive, long timeout) { Map map = null; if (caseInsensitive) { @@ -192,4 +149,46 @@ private static Matcher createMatcher(String content, String patternStr, boolean } + private static class InterruptibleCharSequence implements CharSequence { + + private CharSequence inner; + private long timeout; + private boolean isStart = false; + private long startTime = -1; + + public InterruptibleCharSequence(CharSequence inner, long timeout) { + super(); + this.inner = inner; + this.timeout = timeout; + } + + @Override + public int length() { + return inner.length(); + } + + @Override + public char charAt(int index) { + if (!isStart) { + isStart = true; + startTime = System.currentTimeMillis(); + } + if (System.currentTimeMillis() - startTime > timeout && timeout > 0) { + throw new RegexTimeoutException("正则执行超时:" + inner.toString()); + } + return inner.charAt(index); + + } + + @Override + public CharSequence subSequence(int start, int end) { + return new InterruptibleCharSequence(inner.subSequence(start, end), timeout); + } + + @Override + public String toString() { + return inner.toString(); + } + } + } diff --git a/rocketmq-streams-filter/src/test/java/org/apache/rocketmq/streams/filter/FilterComponentTest.java b/rocketmq-streams-filter/src/test/java/org/apache/rocketmq/streams/filter/FilterComponentTest.java index 90b5cfdd..0fa8303b 100644 --- a/rocketmq-streams-filter/src/test/java/org/apache/rocketmq/streams/filter/FilterComponentTest.java +++ b/rocketmq-streams-filter/src/test/java/org/apache/rocketmq/streams/filter/FilterComponentTest.java @@ -30,7 +30,7 @@ public void testFilter() { jsonObject.put("ip", "1.2.2.3"); jsonObject.put("uid", 1224); jsonObject.put("vmip", "1.1.1.1"); - List ruleList=filterComponent.excuteRule(jsonObject, rule); + List ruleList = filterComponent.excuteRule(jsonObject, rule); System.out.println(ruleList.size()); } } diff --git a/rocketmq-streams-filter/src/test/java/org/apache/rocketmq/streams/filter/operator/RuleTest.java b/rocketmq-streams-filter/src/test/java/org/apache/rocketmq/streams/filter/operator/RuleTest.java deleted file mode 100644 index ca7900a0..00000000 --- a/rocketmq-streams-filter/src/test/java/org/apache/rocketmq/streams/filter/operator/RuleTest.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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.filter.operator; - -import com.alibaba.fastjson.JSONObject; -import org.apache.rocketmq.streams.configurable.ConfigurableComponent; -import org.apache.rocketmq.streams.filter.builder.RuleBuilder; -import org.junit.Test; - -import static junit.framework.TestCase.assertTrue; - -public class RuleTest { - @Test - public void testRule() { - ConfigurableComponent configurableComponent = ConfigurableComponent.getInstance("namespace"); - JSONObject jsonObject = new JSONObject(); - jsonObject.put("ip", "1.2.2.3"); - jsonObject.put("uid", 1224); - jsonObject.put("vmip", "1.1.1.1"); - - RuleBuilder ruleBuilder = new RuleBuilder("namespace", "ruleName", "(ip,==,1.2.2.3)&((uid,=,1224)|(vmip,=,1.1.11.1))", "uid;int"); - boolean isMatch = ruleBuilder.generateRule(configurableComponent.getService()).execute(jsonObject); - assertTrue(isMatch); - } -} diff --git a/rocketmq-streams-filter/src/test/resources/log4j.xml b/rocketmq-streams-filter/src/test/resources/log4j.xml old mode 100755 new mode 100644 diff --git a/rocketmq-streams-lease/pom.xml b/rocketmq-streams-lease/pom.xml index 51eb7183..d512df3d 100755 --- a/rocketmq-streams-lease/pom.xml +++ b/rocketmq-streams-lease/pom.xml @@ -15,14 +15,14 @@ See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 org.apache.rocketmq rocketmq-streams - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT rocketmq-streams-lease ROCKETMQ STREAMS :: lease @@ -30,11 +30,11 @@ org.apache.rocketmq - rocketmq-streams-channel-db + rocketmq-streams-db-operator org.apache.rocketmq - rocketmq-streams-db-operator + rocketmq-streams-serviceloader diff --git a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/LeaseComponent.java b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/LeaseComponent.java index 614b5aaf..2e16026d 100644 --- a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/LeaseComponent.java +++ b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/LeaseComponent.java @@ -17,14 +17,11 @@ package org.apache.rocketmq.streams.lease; import java.util.Properties; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.component.AbstractComponent; import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.component.ConfigureDescriptor; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.configuration.SystemContext; import org.apache.rocketmq.streams.common.utils.StringUtil; -import org.apache.rocketmq.streams.configurable.service.ConfigurableServcieType; import org.apache.rocketmq.streams.lease.service.ILeaseService; import org.apache.rocketmq.streams.lease.service.ILeaseStorage; import org.apache.rocketmq.streams.lease.service.impl.LeaseServiceImpl; @@ -40,13 +37,11 @@ public class LeaseComponent extends AbstractComponent { private static LeaseComponent leaseComponent = null; - private static final Log LOG = LogFactory.getLog(LeaseComponent.class); private ILeaseService leaseService; public LeaseComponent() { initConfigurableServiceDescriptor(); - addConfigureDescriptor( - new ConfigureDescriptor(CONNECT_TYPE, false, ConfigurableServcieType.DEFAULT_SERVICE_NAME)); + } public static LeaseComponent getInstance() { @@ -77,25 +72,25 @@ protected boolean startComponent(String namespace) { @Override protected boolean initProperties(Properties properties) { - String connectType = properties.getProperty(JDBC_URL); + String connectType = properties.getProperty(ConfigurationKey.JDBC_URL); if (StringUtil.isEmpty(connectType)) { this.leaseService = new MockLeaseImpl(); return true; } LeaseServiceImpl leaseService = new LeaseServiceImpl(); - String storageName = ComponentCreator.getProperties().getProperty(ConfigureFileKey.LEASE_STORAGE_NAME); - ILeaseStorage storasge = null; + String storageName = SystemContext.getProperty(ConfigurationKey.LEASE_STORAGE_NAME); + ILeaseStorage storage = null; if (StringUtil.isEmpty(storageName)) { - String jdbc = properties.getProperty(AbstractComponent.JDBC_DRIVER); - String url = properties.getProperty(AbstractComponent.JDBC_URL); - String userName = properties.getProperty(AbstractComponent.JDBC_USERNAME); - String password = properties.getProperty(AbstractComponent.JDBC_PASSWORD); - storasge = new DBLeaseStorage(jdbc, url, userName, password); + String jdbc = properties.getProperty(ConfigurationKey.JDBC_DRIVER); + String url = properties.getProperty(ConfigurationKey.JDBC_URL); + String userName = properties.getProperty(ConfigurationKey.JDBC_USERNAME); + String password = properties.getProperty(ConfigurationKey.JDBC_PASSWORD); + storage = new DBLeaseStorage(jdbc, url, userName, password); } else { - storasge = (ILeaseStorage)ServiceLoaderComponent.getInstance(ILeaseStorage.class).loadService(storageName); + storage = (ILeaseStorage) ServiceLoaderComponent.getInstance(ILeaseStorage.class).loadService(storageName); } - leaseService.setLeaseStorage(storasge); + leaseService.setLeaseStorage(storage); this.leaseService = leaseService; return true; } diff --git a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseService.java b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseService.java index 56c09816..7e8fa1be 100644 --- a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseService.java +++ b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseService.java @@ -27,7 +27,7 @@ public interface ILeaseService { /** * 默认锁定时间 */ - static final int DEFALUT_LOCK_TIME = 60 * 5; + static final int DEFAULT_LOCK_TIME = 60 * 5; /** * 检查某用户当前时间是否具有租约。这个方法是纯内存操作,无性能开销 @@ -60,12 +60,19 @@ public interface ILeaseService { */ void startLeaseTask(final String name, int leaseTermSecond, ILeaseGetCallback callback); + /** + * 释放租约,并且停止底层的调度线程 + * + * @param name 租约名称 + */ + void stopLeaseTask(final String name); + /** * 申请锁,无论成功与否,立刻返回。如果不释放,最大锁定时间是5分钟 * * @param name 业务名称 * @param lockerName 锁名称 - * @return 是否枷锁成功 + * @return 是否加锁成功 */ boolean lock(String name, String lockerName); @@ -75,8 +82,7 @@ public interface ILeaseService { * @param name 业务名称 * @param lockerName 锁名称 * @param lockTimeSecond 如果不释放,锁定的最大时间,单位是秒 - * @return 是否枷锁成功 - * @return + * @return 是否加锁成功 */ boolean lock(String name, String lockerName, int lockTimeSecond); @@ -86,7 +92,7 @@ public interface ILeaseService { * @param name 业务名称 * @param lockerName 锁名称 * @param waitTime 没获取锁时,最大等待多长时间,如果是-1 则无限等待 - * @return 是否枷锁成功 + * @return 是否加锁成功 */ boolean tryLocker(String name, String lockerName, long waitTime); @@ -97,16 +103,16 @@ public interface ILeaseService { * @param lockerName 锁名称 * @param waitTime 没获取锁时,最大等待多长时间,如果是-1 则无限等待 * @param lockTimeSecond 如果不释放,锁定的最大时间,单位是秒 - * @return 是否枷锁成功 + * @return 是否加锁成功 */ boolean tryLocker(String name, String lockerName, long waitTime, int lockTimeSecond); /** * 释放锁 * - * @param name - * @param lockerName - * @return + * @param name 业务名称 + * @param lockerName 锁名称 + * @return 是否释放成功 */ boolean unlock(String name, String lockerName); @@ -125,10 +131,17 @@ public interface ILeaseService { * * @param name 业务名称 * @param lockerName 锁名称 - * @return + * @return 是否持有锁 */ boolean hasHoldLock(String name, String lockerName); + /** + * 获取该业务下的所有锁 + * + * @param name 业务名称 + * @param lockerNamePrefix 锁名前缀 + * @return 所有的锁 + */ List queryLockedInstanceByNamePrefix(String name, String lockerNamePrefix); } diff --git a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseStorage.java b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseStorage.java index a85d3a6d..63a3c214 100644 --- a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseStorage.java +++ b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseStorage.java @@ -68,4 +68,12 @@ public interface ILeaseStorage { */ void addLeaseInfo(LeaseInfo leaseInfo); + /** + * 释放租约 + * + * @param leaseName + * @return + */ + void deleteLeaseInfo(String leaseName); + } diff --git a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseStorasge.java b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseStorasge.java index cbfe26e7..4acacd7d 100644 --- a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseStorasge.java +++ b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseStorasge.java @@ -21,10 +21,9 @@ public interface ILeaseStorasge { - - /** * 更新lease info,需要是原子操作,存储保障多线程操作的原子性 + * * @param leaseInfo 租约表数据 * @return */ @@ -32,6 +31,7 @@ public interface ILeaseStorasge { /** * 统计这个租约名称下,LeaseInfo对象个数 + * * @param leaseName 租约名称,无特殊要求,相同名称会竞争租约 * @return */ @@ -39,14 +39,15 @@ public interface ILeaseStorasge { /** * 查询无效的的租约 + * * @param leaseName 租约名称,无特殊要求,相同名称会竞争租约 * @return */ LeaseInfo queryInValidateLease(String leaseName); - /** * 查询无效的的租约 + * * @param leaseName 租约名称,无特殊要求,相同名称会竞争租约 * @return */ @@ -54,10 +55,9 @@ public interface ILeaseStorasge { /** * 增加租约 + * * @param leaseInfo 租约名称,无特殊要求,相同名称会竞争租约 */ void addLeaseInfo(LeaseInfo leaseInfo); - - } diff --git a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/BasedLesaseImpl.java b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/BasedLeaseImpl.java similarity index 80% rename from rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/BasedLesaseImpl.java rename to rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/BasedLeaseImpl.java index 9584a46e..45e269a2 100644 --- a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/BasedLesaseImpl.java +++ b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/BasedLeaseImpl.java @@ -23,12 +23,9 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import org.apache.rocketmq.streams.common.threadpool.ScheduleFactory; import org.apache.rocketmq.streams.common.utils.DateUtil; import org.apache.rocketmq.streams.common.utils.IPUtil; import org.apache.rocketmq.streams.common.utils.RuntimeUtil; @@ -36,23 +33,31 @@ import org.apache.rocketmq.streams.lease.service.ILeaseGetCallback; import org.apache.rocketmq.streams.lease.service.ILeaseService; import org.apache.rocketmq.streams.lease.service.ILeaseStorage; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public abstract class BasedLesaseImpl implements ILeaseService { - private static final Log LOG = LogFactory.getLog(BasedLesaseImpl.class); +public abstract class BasedLeaseImpl implements ILeaseService { + private static final Logger LOGGER = LoggerFactory.getLogger(BasedLeaseImpl.class); private static final String CONSISTENT_HASH_PREFIX = "consistent_hash_"; private static final AtomicBoolean syncStart = new AtomicBoolean(false); private static final int synTime = 120; // 5分钟的一致性hash同步时间太久了,改为2分钟 - protected ScheduledExecutorService taskExecutor = null; + private final Map startLeaseMap = new HashMap<>(); protected int leaseTerm = 300 * 2; // 租约时间 - // protected transient JDBCDriver jdbcDataSource = null; protected ILeaseStorage leaseStorage; protected volatile Map leaseName2Date = new ConcurrentHashMap<>(); // 每个lease name对应的租约到期时间 - public BasedLesaseImpl() { + public BasedLeaseImpl() { + } - taskExecutor = new ScheduledThreadPoolExecutor(10); + /** + * 本地ip地址作为自己的唯一标识 + * + * @return + */ + public static String getLocalName() { + return IPUtil.getLocalIdentification() + ":" + Optional.ofNullable(RuntimeUtil.getPid()).orElse("UNKNOWN"); } /** @@ -61,8 +66,7 @@ public BasedLesaseImpl() { * @param name * @return */ - @Override - public boolean hasLease(String name) { + @Override public boolean hasLease(String name) { // 内存中没有租约信息则表示 没有租约 Date leaseEndTime = leaseName2Date.get(name); if (leaseEndTime == null) { @@ -79,20 +83,15 @@ public boolean hasLease(String name) { return false; } - private final Map startLeaseMap = new HashMap<>(); - - @Override - public void startLeaseTask(final String name) { + @Override public void startLeaseTask(final String name) { startLeaseTask(name, this.leaseTerm, null); } - @Override - public void startLeaseTask(final String name, ILeaseGetCallback callback) { + @Override public void startLeaseTask(final String name, ILeaseGetCallback callback) { startLeaseTask(name, this.leaseTerm, callback); } - @Override - public void startLeaseTask(final String name, int leaseTerm, ILeaseGetCallback callback) { + @Override public void startLeaseTask(final String name, int leaseTerm, ILeaseGetCallback callback) { ApplyTask applyTask = new ApplyTask(leaseTerm, name, callback); startLeaseTask(name, applyTask, leaseTerm / 2, true); } @@ -120,51 +119,22 @@ protected void startLeaseTask(final String name, Runnable runnable, int schedule if (startNow) { runnable.run(); } - taskExecutor.scheduleWithFixedDelay(runnable, 0, scheduleTime, TimeUnit.SECONDS); + ScheduleFactory.getInstance().execute(BasedLeaseImpl.class.getName() + "-base_lease_schedule", runnable, 0, scheduleTime, TimeUnit.SECONDS); } } - /** - * 续约任务 - */ - protected class ApplyTask implements Runnable { - - protected String name; - protected int leaseTerm; - protected ILeaseGetCallback callback; - - public ApplyTask(int leaseTerm, String name) { - this(leaseTerm, name, null); - } - - public ApplyTask(int leaseTerm, String name, ILeaseGetCallback callback) { - this.name = name; - this.leaseTerm = leaseTerm; - this.callback = callback; - } - - @Override - public void run() { - try { - // LOG.info("LeaseServiceImpl name: " + name + "开始获取租约..."); - AtomicBoolean newApplyLease = new AtomicBoolean(false); - Date leaseDate = applyLeaseTask(leaseTerm, name, newApplyLease); - if (leaseDate != null) { - leaseName2Date.put(name, leaseDate); - LOG.info("LeaseServiceImpl, name: " + name + " " + getSelfUser() + " 获取租约成功, 租约到期时间为 " - + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").format(leaseDate)); - } else { - // fix.2020.08.13 这时name对应的租约可能还在有效期内,或者本机还持有租约,需要remove - // leaseName2Date.remove(name); - LOG.info("LeaseServiceImpl name: " + name + " " + getSelfUser() + " 获取租约失败 "); - } - if (newApplyLease.get() && callback != null) { - callback.callback(leaseDate); - } - } catch (Exception e) { - LOG.error(" LeaseServiceImpl name: " + name + " " + getSelfUser() + " 获取租约出现异常 ", e); + @Override public void stopLeaseTask(String name) { + AtomicBoolean isStartLease = startLeaseMap.get(name);//多次调用,只启动一次定时任务 + if (isStartLease == null) { + synchronized (this) { + isStartLease = startLeaseMap.get(name); } - + } + if (isStartLease == null || isStartLease.compareAndSet(true, false)) { + ScheduleFactory.getInstance().cancel(BasedLeaseImpl.class.getName() + "-base_lease_schedule"); + //释放 + releaseLeaseInfo(name); + leaseName2Date.remove(name); } } @@ -182,7 +152,7 @@ protected Date applyLeaseTask(int leaseTerm, String name, AtomicBoolean newApply // 更新数据库 LeaseInfo leaseInfo = queryValidateLease(name); if (leaseInfo == null) { - LOG.error("LeaseServiceImpl applyLeaseTask leaseInfo is null"); + LOGGER.error("LeaseServiceImpl applyLeaseTask leaseInfo is null"); return null; } // fix.2020.08.13,与本机ip相等且满足一致性hash分配策略,才续约,其他情况为null @@ -255,17 +225,17 @@ protected boolean tryGetLease(String name, Date time) { validateLeaseInfo.setStatus(1); validateLeaseInfo.setVersion(1); if (insert(validateLeaseInfo)) { - LOG.info("数据库中暂时没有租约信息,原子插入成功,获取租约成功:" + name); + LOGGER.info("数据库中暂时没有租约信息,原子插入成功,获取租约成功:" + name); return true; } else { - LOG.info("数据库中暂时没有租约信息,原子插入失败,已经被其他机器获取租约:" + name); + LOGGER.info("数据库中暂时没有租约信息,原子插入失败,已经被其他机器获取租约:" + name); return false; } } else { // 表示数据库里面有一条但是无效,这里需要两台机器按照version进行原子更新,更新成功的获取租约 // LOG.info("数据库中有一条无效的租约信息,尝试根据版本号去原子更新租约信息:" + name); LeaseInfo inValidateLeaseInfo = queryInValidateLease(name); if (inValidateLeaseInfo == null) {// 说明这个时候另外一台机器获取成功了 - LOG.info("另外一台机器获取成功了租约:" + name); + LOGGER.info("另外一台机器获取成功了租约:" + name); return false; } // fix.2020.08.13,机器重启之后,该名字的任务已经不分配在此机器上执行,直接返回,无需更新数据库 @@ -278,9 +248,9 @@ protected boolean tryGetLease(String name, Date time) { inValidateLeaseInfo.setStatus(1); boolean success = updateDBLeaseInfo(inValidateLeaseInfo); if (success) { - LOG.info("LeaseServiceImpl 原子更新租约成功,当前机器获取到了租约信息:" + name); + LOGGER.info("LeaseServiceImpl 原子更新租约成功,当前机器获取到了租约信息:" + name); } else { - LOG.info("LeaseServiceImpl 原子更新租约失败,租约被其他机器获取:" + name); + LOGGER.info("LeaseServiceImpl 原子更新租约失败,租约被其他机器获取:" + name); } return success; } @@ -294,13 +264,10 @@ protected boolean tryGetLease(String name, Date time) { validateLeaseInfo.setLeaseEndDate(time); boolean hasUpdate = updateLeaseInfo(validateLeaseInfo); if (hasUpdate) { - LOG.info( - "LeaseServiceImpl机器重启情况,当前用户有租约信息,并且更新数据库成功,租约信息为 name :" + validateLeaseInfo.getLeaseName() - + " ip : " + validateLeaseInfo.getLeaseUserIp() + " 到期时间 : " + new SimpleDateFormat( - "yyyy-MM-dd HH:mm:ss").format(validateLeaseInfo.getLeaseEndDate())); + LOGGER.info("LeaseServiceImpl机器重启情况,当前用户有租约信息,并且更新数据库成功,租约信息为 name :" + validateLeaseInfo.getLeaseName() + " ip : " + validateLeaseInfo.getLeaseUserIp() + " 到期时间 : " + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").format(validateLeaseInfo.getLeaseEndDate())); return true; } else { - LOG.info("LeaseServiceImpl 机器重启情况,当前用户有租约信息,并且更新数据库失败,表示失去租约:" + name); + LOGGER.info("LeaseServiceImpl 机器重启情况,当前用户有租约信息,并且更新数据库失败,表示失去租约:" + name); return false; } } @@ -334,7 +301,7 @@ private boolean insert(LeaseInfo leaseInfo) { addLeaseInfo(leaseInfo); return true; } catch (Exception e) { - LOG.error("LeaseServiceImpl insert error", e); + LOGGER.error("LeaseServiceImpl insert error", e); return false; } } @@ -350,33 +317,32 @@ protected boolean updateDBLeaseInfo(LeaseInfo leaseInfo) { } protected boolean updateLeaseInfo(LeaseInfo leaseInfo) { - return leaseStorage.updateLeaseInfo(leaseInfo); } protected Integer countLeaseInfo(String name) { - return leaseStorage.countLeaseInfo(name); } protected LeaseInfo queryInValidateLease(String name) { - return leaseStorage.queryInValidateLease(name); } protected void addLeaseInfo(LeaseInfo leaseInfo) { - leaseStorage.addLeaseInfo(leaseInfo); - } - /** - * 本地ip地址作为自己的唯一标识 - * - * @return - */ - public static String getLocalName() { - return IPUtil.getLocalIdentification() + ":" + Optional.ofNullable(RuntimeUtil.getPid()).orElse("UNKNOWN"); + protected void releaseLeaseInfo(String name) { + // LOG.info("LeaseServiceImpl unlock,name:" + name); + LeaseInfo validateLeaseInfo = queryValidateLease(name); + if (validateLeaseInfo == null) { + LOGGER.warn("LeaseServiceImpl unlock,validateLeaseInfo is null,lockerName:" + name); + } + if (validateLeaseInfo != null && validateLeaseInfo.getLeaseUserIp().equals(getSelfUser())) { + validateLeaseInfo.setStatus(0); + updateDBLeaseInfo(validateLeaseInfo); + } + leaseName2Date.remove(name); } /** @@ -399,4 +365,46 @@ private String getConsistentHashHost(String name) { public void setLeaseStorage(ILeaseStorage leaseStorage) { this.leaseStorage = leaseStorage; } + + /** + * 续约任务 + */ + protected class ApplyTask implements Runnable { + + protected String name; + protected int leaseTerm; + protected ILeaseGetCallback callback; + + public ApplyTask(int leaseTerm, String name) { + this(leaseTerm, name, null); + } + + public ApplyTask(int leaseTerm, String name, ILeaseGetCallback callback) { + this.name = name; + this.leaseTerm = leaseTerm; + this.callback = callback; + } + + @Override public void run() { + try { + // LOG.info("LeaseServiceImpl name: " + name + "开始获取租约..."); + AtomicBoolean newApplyLease = new AtomicBoolean(false); + Date leaseDate = applyLeaseTask(leaseTerm, name, newApplyLease); + if (leaseDate != null) { + leaseName2Date.put(name, leaseDate); + LOGGER.info("LeaseServiceImpl, name: " + name + " " + getSelfUser() + " 获取租约成功, 租约到期时间为 " + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").format(leaseDate)); + } else { + // fix.2020.08.13 这时name对应的租约可能还在有效期内,或者本机还持有租约,需要remove + // leaseName2Date.remove(name); + LOGGER.info("LeaseServiceImpl name: " + name + " " + getSelfUser() + " 获取租约失败 "); + } + if (newApplyLease.get() && callback != null) { + callback.callback(leaseDate); + } + } catch (Exception e) { + LOGGER.error(" LeaseServiceImpl name: " + name + " " + getSelfUser() + " 获取租约出现异常 ", e); + } + + } + } } diff --git a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/LeaseServiceImpl.java b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/LeaseServiceImpl.java index f10ad0a9..9bd6ebf9 100644 --- a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/LeaseServiceImpl.java +++ b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/LeaseServiceImpl.java @@ -22,25 +22,22 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import org.apache.rocketmq.streams.common.threadpool.ScheduleFactory; import org.apache.rocketmq.streams.common.utils.DateUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.lease.model.LeaseInfo; import org.apache.rocketmq.streams.lease.service.ILeaseService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class LeaseServiceImpl extends BasedLesaseImpl { - - private static final Log LOG = LogFactory.getLog(LeaseServiceImpl.class); +public class LeaseServiceImpl extends BasedLeaseImpl { + private static final Logger LOGGER = LoggerFactory.getLogger(LeaseServiceImpl.class); + protected ConcurrentHashMap seizeLockingFutures = new ConcurrentHashMap<>(); private transient ConcurrentHashMap holdLockTasks = new ConcurrentHashMap(); - - protected ConcurrentHashMap seizeLockingFuntures = new ConcurrentHashMap<>(); //如果是抢占锁状态中,则不允许申请锁 public LeaseServiceImpl() { @@ -57,7 +54,7 @@ public LeaseServiceImpl() { */ @Override public boolean tryLocker(String name, String lockerName, long waitTime) { - return tryLocker(name, lockerName, waitTime, ILeaseService.DEFALUT_LOCK_TIME); + return tryLocker(name, lockerName, waitTime, ILeaseService.DEFAULT_LOCK_TIME); } @Override @@ -75,7 +72,7 @@ public boolean tryLocker(String name, String lockerName, long waitTime, int lock try { Thread.sleep(100); } catch (InterruptedException e) { - LOG.error("LeaseServiceImpl try locker error", e); + LOGGER.error("LeaseServiceImpl try locker error", e); } } return success; @@ -84,18 +81,17 @@ public boolean tryLocker(String name, String lockerName, long waitTime, int lock @Override public boolean lock(String name, String lockerName) { - return lock(name, lockerName, ILeaseService.DEFALUT_LOCK_TIME); + return lock(name, lockerName, ILeaseService.DEFAULT_LOCK_TIME); } @Override public boolean lock(String name, String lockerName, int leaseSecond) { lockerName = createLockName(name, lockerName); - Future future = seizeLockingFuntures.get(lockerName); - if (future != null && ((HoldLockFunture)future).isDone == false) { + Future future = seizeLockingFutures.get(lockerName); + if (future != null && ((HoldLockFunture) future).isDone == false) { return false; } - Date nextLeaseDate = - DateUtil.addSecond(new Date(), leaseSecond);// 默认锁定5分钟,用完需要立刻释放.如果时间不同步,可能导致锁失败 + Date nextLeaseDate = DateUtil.addSecond(new Date(), leaseSecond);// 默认锁定5分钟,用完需要立刻释放.如果时间不同步,可能导致锁失败 return tryGetLease(lockerName, nextLeaseDate); } @@ -105,7 +101,7 @@ public boolean unlock(String name, String lockerName) { lockerName = createLockName(name, lockerName); LeaseInfo validateLeaseInfo = queryValidateLease(lockerName); if (validateLeaseInfo == null) { - LOG.warn("LeaseServiceImpl unlock,validateLeaseInfo is null,lockerName:" + lockerName); + LOGGER.info("LeaseServiceImpl unlock,validateLeaseInfo is null,lockerName:" + lockerName); } if (validateLeaseInfo != null && validateLeaseInfo.getLeaseUserIp().equals(getSelfUser())) { validateLeaseInfo.setStatus(0); @@ -123,22 +119,21 @@ public boolean unlock(String name, String lockerName) { * 如果有锁,则一直持有,如果不能获取,则结束。和租约不同,租约是没有也会尝试重试,一备对方挂机,自己可以接手工作 * * @param name - * @param secondeName + * @param secondName * @param lockTimeSecond 获取锁的时间 * @return */ @Override - public boolean holdLock(String name, String secondeName, int lockTimeSecond) { - if (hasHoldLock(name, secondeName)) { + public boolean holdLock(String name, String secondName, int lockTimeSecond) { + if (hasHoldLock(name, secondName)) { return true; } synchronized (this) { - if (hasHoldLock(name, secondeName)) { + if (hasHoldLock(name, secondName)) { return true; } - String lockerName = createLockName(name, secondeName); - Date nextLeaseDate = - DateUtil.addSecond(new Date(), lockTimeSecond); + String lockerName = createLockName(name, secondName); + Date nextLeaseDate = DateUtil.addSecond(new Date(), lockTimeSecond); boolean success = tryGetLease(lockerName, nextLeaseDate);// 申请锁,锁的时间是leaseTerm if (!success) { return false; @@ -151,7 +146,6 @@ public boolean holdLock(String name, String secondeName, int lockTimeSecond) { holdLockTasks.putIfAbsent(lockerName, holdLockTask); } } - return true; } @@ -159,12 +153,12 @@ public boolean holdLock(String name, String secondeName, int lockTimeSecond) { * 是否持有锁,不访问数据库,直接看本地 * * @param name - * @param secondeName + * @param secondName * @return */ @Override - public boolean hasHoldLock(String name, String secondeName) { - String lockerName = createLockName(name, secondeName); + public boolean hasHoldLock(String name, String secondName) { + String lockerName = createLockName(name, secondName); return hasLease(lockerName); } @@ -181,24 +175,20 @@ private String createLockName(String name, String lockerName) { private class HoldLockTask extends ApplyTask { protected volatile boolean isContinue = true; protected LeaseServiceImpl leaseService; - protected ScheduledExecutorService scheduledExecutor; public HoldLockTask(int leaseTerm, String name, LeaseServiceImpl leaseService) { super(leaseTerm, name); this.leaseService = leaseService; - scheduledExecutor = new ScheduledThreadPoolExecutor(1); } public void start() { - scheduledExecutor.scheduleWithFixedDelay(this, leaseTerm / 2, leaseTerm / 2, TimeUnit.SECONDS); + ScheduleFactory.getInstance().execute(name + "-" + getLocalName() + "-hold_lock_schedule", this, leaseTerm / 2, leaseTerm / 2, TimeUnit.SECONDS); } public void close() { isContinue = false; - if (scheduledExecutor != null) { - scheduledExecutor.shutdown(); - } + ScheduleFactory.getInstance().cancel(name + "-" + getLocalName() + "-hold_lock_schedule"); } public boolean isContinue() { @@ -214,18 +204,17 @@ public void run() { Date leaseDate = applyLeaseTask(leaseTerm, name, new AtomicBoolean(false)); if (leaseDate != null) { leaseName2Date.put(name, leaseDate); - LOG.debug("LeaseServiceImpl, name: " + name + " " + getSelfUser() + " 续约锁成功, 租约到期时间为 " - + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").format(leaseDate)); + LOGGER.info("LeaseServiceImpl, name: " + name + " " + getSelfUser() + " 续约锁成功, 租约到期时间为 " + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").format(leaseDate)); } else { isContinue = false; synchronized (leaseService) { holdLockTasks.remove(name); } - LOG.info("LeaseServiceImpl name: " + name + " " + getSelfUser() + " 续约锁失败,续锁程序会停止"); + LOGGER.info("LeaseServiceImpl name: " + name + " " + getSelfUser() + " 续约锁失败,续锁程序会停止"); } } catch (Exception e) { isContinue = false; - LOG.error(" LeaseServiceImpl name: " + name + " " + getSelfUser() + " 续约锁出现异常,续锁程序会停止", e); + LOGGER.error(" LeaseServiceImpl name: " + name + " " + getSelfUser() + " 续约锁出现异常,续锁程序会停止", e); } } @@ -238,6 +227,7 @@ public void run() { protected class HoldLockFunture implements Future { private volatile boolean isDone = false; private volatile Date date = null; + private long startTime = System.currentTimeMillis(); @Override public boolean cancel(boolean mayInterruptIfRunning) { @@ -266,8 +256,6 @@ public Boolean get() throws InterruptedException, ExecutionException { return true; } - private long startTime = System.currentTimeMillis(); - @Override public Boolean get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { diff --git a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/MockLeaseImpl.java b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/MockLeaseImpl.java index 73549562..2d239a24 100644 --- a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/MockLeaseImpl.java +++ b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/MockLeaseImpl.java @@ -47,6 +47,11 @@ public void startLeaseTask(String name, int leaseTerm, ILeaseGetCallback callbac } + @Override + public void stopLeaseTask(String name) { + + } + @Override public boolean lock(String name, String lockerName) { return true; diff --git a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/storages/DBLeaseStorage.java b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/storages/DBLeaseStorage.java index d3d16b94..0ff75375 100644 --- a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/storages/DBLeaseStorage.java +++ b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/storages/DBLeaseStorage.java @@ -21,8 +21,6 @@ import java.util.Date; import java.util.List; import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.utils.DateUtil; import org.apache.rocketmq.streams.common.utils.SQLUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; @@ -30,14 +28,16 @@ import org.apache.rocketmq.streams.db.driver.JDBCDriver; import org.apache.rocketmq.streams.lease.model.LeaseInfo; import org.apache.rocketmq.streams.lease.service.ILeaseStorage; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class DBLeaseStorage implements ILeaseStorage { - private static final Log LOG = LogFactory.getLog(DBLeaseStorage.class); + private static final Logger LOGGER = LoggerFactory.getLogger(DBLeaseStorage.class); protected JDBCDriver jdbcDataSource; - private String url; protected String userName; protected String password; protected String jdbc; + private String url; public DBLeaseStorage(String jdbc, String url, String userName, String password) { this.jdbc = jdbc; @@ -75,7 +75,7 @@ public boolean updateLeaseInfo(LeaseInfo leaseInfo) { } return success; } catch (Exception e) { - LOG.error("LeaseServiceImpl updateLeaseInfo excuteUpdate error", e); + LOGGER.error("LeaseServiceImpl updateLeaseInfo excuteUpdate error", e); throw new RuntimeException("execute sql error " + sql, e); } } @@ -99,7 +99,7 @@ public Integer countLeaseInfo(String leaseName) { @Override public LeaseInfo queryInValidateLease(String leaseName) { String sql = "SELECT * FROM lease_info WHERE lease_name ='" + leaseName + "' and status=1 and lease_end_time<'" + DateUtil.getCurrentTimeString() + "'"; - LOG.info("LeaseServiceImpl queryInValidateLease builder:" + sql); + LOGGER.info("LeaseServiceImpl queryInValidateLease builder:" + sql); return queryLease(leaseName, sql); } @@ -139,7 +139,17 @@ public void addLeaseInfo(LeaseInfo leaseInfo) { getOrCreateJDBCDataSource().execute(sql); } catch (Exception e) { - LOG.error("LeaseServiceImpl execute sql error,sql:" + sql, e); + LOGGER.error("LeaseServiceImpl execute sql error,sql:" + sql, e); + throw new RuntimeException("execute sql error " + sql, e); + } + } + + @Override public void deleteLeaseInfo(String leaseName) { + String sql = " DELETE FROM lease_info WHERE lease_name = '" + leaseName + "'"; + try { + getOrCreateJDBCDataSource().execute(sql); + } catch (Exception e) { + LOGGER.error("LeaseServiceImpl execute sql error,sql:" + sql, e); throw new RuntimeException("execute sql error " + sql, e); } } diff --git a/rocketmq-streams-lease/src/test/java/org/apache/rocketmq/streams/lease/LeaseComponentTest.java b/rocketmq-streams-lease/src/test/java/org/apache/rocketmq/streams/lease/LeaseComponentTest.java index bff40c10..32fc14ae 100644 --- a/rocketmq-streams-lease/src/test/java/org/apache/rocketmq/streams/lease/LeaseComponentTest.java +++ b/rocketmq-streams-lease/src/test/java/org/apache/rocketmq/streams/lease/LeaseComponentTest.java @@ -17,8 +17,7 @@ package org.apache.rocketmq.streams.lease; import java.util.Date; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import java.util.List; import org.apache.rocketmq.streams.db.driver.DriverBuilder; import org.apache.rocketmq.streams.db.driver.JDBCDriver; import org.apache.rocketmq.streams.lease.model.LeaseInfo; @@ -31,19 +30,13 @@ public class LeaseComponentTest { - private String URL = ""; protected String USER_NAME = ""; protected String PASSWORD = ""; + private String URL = ""; public LeaseComponentTest() { - //正式使用时,在配置文件配置 - ComponentCreator.getProperties().put(ConfigureFileKey.CONNECT_TYPE, "DB"); - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_URL, URL);//数据库连接url - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_USERNAME, USER_NAME);//用户名 - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_PASSWORD, PASSWORD);//password - - JDBCDriver driver = DriverBuilder.createDriver(); + JDBCDriver driver = DriverBuilder.createDriver(null, URL, USER_NAME, PASSWORD); driver.execute(LeaseInfo.createTableSQL()); } @@ -83,7 +76,7 @@ public void testLock() throws InterruptedException { @Test public void testHoldLock() throws InterruptedException { String name = "dipper"; - String lockName = "lease.test"; + String lockName = "lease.test1"; int leaseTime = 6; boolean success = LeaseComponent.getInstance().getService().holdLock(name, lockName, leaseTime);//锁定5秒钟 assertTrue(success);//获取锁 @@ -105,6 +98,15 @@ public void testHoldLockContinue() throws InterruptedException { } } + @Test + public void testQueryLockedInstanceByNamePrefix() throws InterruptedException { + String name = "dipper"; + List leaseInfoList = LeaseComponent.getInstance().getService().queryLockedInstanceByNamePrefix(name, ""); + for (LeaseInfo leaseInfo : leaseInfoList) { + System.out.println(leaseInfo.getLeaseName()); + } + } + protected boolean holdLock(String name, String lockName, int leaseTime) { ILeaseService leaseService = LeaseComponent.getInstance().getService(); if (leaseService.hasHoldLock(name, lockName)) { diff --git a/rocketmq-streams-lease/src/test/resources/log4j.xml b/rocketmq-streams-lease/src/test/resources/log4j.xml old mode 100755 new mode 100644 diff --git a/rocketmq-streams-configurable/pom.xml b/rocketmq-streams-local-cache/pom.xml similarity index 69% rename from rocketmq-streams-configurable/pom.xml rename to rocketmq-streams-local-cache/pom.xml index 1ea0eb2f..0f56815c 100755 --- a/rocketmq-streams-configurable/pom.xml +++ b/rocketmq-streams-local-cache/pom.xml @@ -16,27 +16,22 @@ See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 org.apache.rocketmq rocketmq-streams - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT - rocketmq-streams-configurable - ROCKETMQ STREAMS :: configurable + rocketmq-streams-local-cache + ROCKETMQ STREAMS :: local-cache jar - org.apache.rocketmq - rocketmq-streams-serviceloader - - - - org.apache.httpcomponents - httpclient + junit + junit diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/AdditionStore.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/AdditionStore.java similarity index 99% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/AdditionStore.java rename to rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/AdditionStore.java index 34fec039..af97144e 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/AdditionStore.java +++ b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/AdditionStore.java @@ -26,37 +26,31 @@ public class AdditionStore { * 每个冲突域列表,每个列表的最大值 */ public static final int CONFLICT_UNIT_SIZE = 16777216; - + /** + * 每个存储单元的大小 + */ + protected int blockSize = CONFLICT_UNIT_SIZE; /** * 如果value是非int值,可以通过这个值存储。原来value部分存储地址 */ private List 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(int elementSize, int blockSize) { this.elementSize = elementSize; if (elementSize > 0) { @@ -71,24 +65,6 @@ public AdditionStore(int elementSize) { this(elementSize, CONFLICT_UNIT_SIZE); } - public class DataElement { - protected byte[] bytes; - protected KVAddress mapAddress; - - public DataElement(byte[] bytes, KVAddress mapAddress) { - this.bytes = bytes; - this.mapAddress = mapAddress; - } - - public byte[] getBytes() { - return bytes; - } - - public KVAddress getMapAddress() { - return mapAddress; - } - } - public Iterator iterator() { return new Iterator() { int index = 0; @@ -207,19 +183,37 @@ public int getConflictIndex() { return conflictIndex; } + public void setConflictIndex(int conflictIndex) { + this.conflictIndex = conflictIndex; + } + public int getConflictOffset() { return conflictOffset; } + public void setConflictOffset(int conflictOffset) { + this.conflictOffset = conflictOffset; + } + public int getBlockSize() { return blockSize; } - public void setConflictIndex(int conflictIndex) { - this.conflictIndex = conflictIndex; - } + public class DataElement { + protected byte[] bytes; + protected KVAddress mapAddress; - public void setConflictOffset(int conflictOffset) { - this.conflictOffset = conflictOffset; + public DataElement(byte[] bytes, KVAddress mapAddress) { + this.bytes = bytes; + this.mapAddress = mapAddress; + } + + public byte[] getBytes() { + return bytes; + } + + public KVAddress getMapAddress() { + return mapAddress; + } } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BigMapAddress.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BigMapAddress.java similarity index 100% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BigMapAddress.java rename to rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BigMapAddress.java diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BitSetCache.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BitSetCache.java similarity index 100% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BitSetCache.java rename to rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BitSetCache.java index 2df54588..e7bbba40 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BitSetCache.java +++ b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BitSetCache.java @@ -26,10 +26,62 @@ public class BitSetCache { protected int capacity; protected int bitSetSize; + public BitSetCache(int capacity) { + cache = new ByteArrayValueKV(capacity); + this.capacity = capacity; + } + + public BitSetCache(int bitSetSize, int capacity) { + this.byteSetSize = bitSetSize / 8 + (bitSetSize % 8 == 0 ? 0 : 1); + this.bitSetSize = bitSetSize; + cache = new ByteArrayValueKV(capacity); + this.capacity = capacity; + cache = new ByteArrayValueKV(capacity, bitSetSize); + } + + public static void main(String[] args) { + BitSet bitSet = new BitSet(200); + bitSet.set(1); + System.out.println(bitSet.get(0)); + for (int i = 0; i < 200; i++) { + System.out.println(bitSet.get(i)); + } + + System.out.println("finish"); + } + + public BitSet createBitSet(int bitSetSize) { + return new BitSet(bitSetSize); + } + + public BitSet createBitSet() { + if (bitSetSize == 0) { + throw new RuntimeException("can not support this method"); + } + return new BitSet(bitSetSize); + } + + public void put(String key, BitSet bitSet) { + cache.put(key, bitSet.getBytes()); + } + + public BitSet get(String key) { + byte[] bytes = cache.get(key); + if (bytes == null) { + return null; + } + return new BitSet(bytes); + + } + + public long size() { + return this.cache.getSize(); + } + public static class BitSet { - private byte[] bytes; protected int byteSetSize; protected int bitSetSize; + private byte[] bytes; public BitSet(int bitSetSize) { this.byteSetSize = bitSetSize / 8 + (bitSetSize % 8 == 0 ? 0 : 1); @@ -107,56 +159,4 @@ public byte[] getBytes() { } } - public BitSet createBitSet(int bitSetSize) { - return new BitSet(bitSetSize); - } - - public BitSet createBitSet() { - if (bitSetSize == 0) { - throw new RuntimeException("can not support this method"); - } - return new BitSet(bitSetSize); - } - - public BitSetCache(int capacity) { - cache = new ByteArrayValueKV(capacity); - this.capacity = capacity; - } - - public BitSetCache(int bitSetSize, int capacity) { - this.byteSetSize = bitSetSize / 8 + (bitSetSize % 8 == 0 ? 0 : 1); - this.bitSetSize = bitSetSize; - cache = new ByteArrayValueKV(capacity); - this.capacity = capacity; - cache = new ByteArrayValueKV(capacity, bitSetSize); - } - - public void put(String key, BitSet bitSet) { - cache.put(key, bitSet.getBytes()); - } - - public static void main(String[] args) { - BitSet bitSet = new BitSet(200); - bitSet.set(1); - System.out.println(bitSet.get(0)); - for (int i = 0; i < 200; i++) { - System.out.println(bitSet.get(i)); - } - - System.out.println("finish"); - } - - public BitSet get(String key) { - byte[] bytes = cache.get(key); - if (bytes == null) { - return null; - } - return new BitSet(bytes); - - } - - public long size() { - return this.cache.getSize(); - } - } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArray.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArray.java similarity index 100% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArray.java rename to rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArray.java diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArrayValueKV.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArrayValueKV.java similarity index 100% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArrayValueKV.java rename to rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArrayValueKV.java diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteStore.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteStore.java similarity index 100% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteStore.java rename to rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteStore.java index 00b93e80..cb5304b5 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteStore.java +++ b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteStore.java @@ -71,24 +71,6 @@ public ByteStore(int elementSize) { this(elementSize, CONFLICT_UNIT_SIZE); } - public class DataElement { - protected byte[] bytes; - protected KVAddress mapAddress; - - public DataElement(byte[] bytes, KVAddress mapAddress) { - this.bytes = bytes; - this.mapAddress = mapAddress; - } - - public byte[] getBytes() { - return bytes; - } - - public KVAddress getMapAddress() { - return mapAddress; - } - } - public Iterator iterator() { return new Iterator() { int index = 0; @@ -208,19 +190,37 @@ public int getConflictIndex() { return conflictIndex; } + public void setConflictIndex(int conflictIndex) { + this.conflictIndex = conflictIndex; + } + public int getConflictOffset() { return conflictOffset; } + public void setConflictOffset(int conflictOffset) { + this.conflictOffset = conflictOffset; + } + public int getBlockSize() { return blockSize; } - public void setConflictIndex(int conflictIndex) { - this.conflictIndex = conflictIndex; - } + public class DataElement { + protected byte[] bytes; + protected KVAddress mapAddress; - public void setConflictOffset(int conflictOffset) { - this.conflictOffset = conflictOffset; + public DataElement(byte[] bytes, KVAddress mapAddress) { + this.bytes = bytes; + this.mapAddress = mapAddress; + } + + public byte[] getBytes() { + return bytes; + } + + public KVAddress getMapAddress() { + return mapAddress; + } } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/CacheKV.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/CacheKV.java similarity index 94% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/CacheKV.java rename to rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/CacheKV.java index 34a34fca..fc349e43 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/CacheKV.java +++ b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/CacheKV.java @@ -16,7 +16,8 @@ */ package org.apache.rocketmq.streams.common.cache.compress; -import org.apache.rocketmq.streams.common.utils.AESUtil; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; import org.apache.rocketmq.streams.common.utils.NumberUtils; /** @@ -69,6 +70,18 @@ public CacheKV(int capacity, boolean hasValue) { conflicts = new ByteStore(elementSize); } + public static byte[] stringToMD5(String plainText) { + byte[] secretBytes = null; + try { + secretBytes = MessageDigest.getInstance("md5").digest( + plainText.getBytes()); + return secretBytes; + } catch (NoSuchAlgorithmException e) { + throw new RuntimeException("没有这个md5算法!"); + } + + } + @Override public abstract T get(String key); @@ -140,7 +153,7 @@ public boolean putInner(String key, ByteArray value, boolean supportUpdate) { */ public MapElementContext queryMapElementByHashCode(String key) { int offset = getElementIndex(key); - byte[] hashCodes = AESUtil.stringToMD5(key); + byte[] hashCodes = stringToMD5(key); KVAddress address = new KVAddress(offset);//先从map中查找 @@ -277,5 +290,4 @@ public boolean isMatchKey() { return isMatchKey; } } - } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ICacheKV.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ICacheKV.java similarity index 100% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ICacheKV.java rename to rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ICacheKV.java diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVAddress.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVAddress.java similarity index 79% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVAddress.java rename to rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVAddress.java index 912a7dd2..fec3fb8c 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVAddress.java +++ b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVAddress.java @@ -50,10 +50,6 @@ public KVAddress() { } - public boolean isEmpty() { - return !isConflict && conflictIndex == 0 && offset == 0; - } - /** * 高位是0则全部四位代表在map中的偏移量,高位是1则最高位代表冲突域的索引,第三位代表在冲突域链表中的地址 * @@ -74,19 +70,49 @@ public KVAddress(ByteArray byteArray) { return; } - byte[] bytes=new byte[2]; - bytes[0]=byteArray.getByte(3); - int x=conflictValue&127; - bytes[1]= NumberUtils.toByte(x)[0]; + byte[] bytes = new byte[2]; + bytes[0] = byteArray.getByte(3); + int x = conflictValue & 127; + bytes[1] = NumberUtils.toByte(x)[0]; this.conflictIndex = NumberUtils.toInt(bytes); this.offset = byteArray.castInt(0, 3); } + public static KVAddress createMapAddressFromLongValue(Long value) { + byte[] bytes = NumberUtils.toByte(value); + return createAddressFromBytes(bytes); + } + + public static KVAddress createAddressFromBytes(byte[] bytes) { + int offset = NumberUtils.toInt(bytes, 0, 3); + byte firstByte = bytes[4]; + if (firstByte < 0) { + bytes[4] = 0; + } + int index = NumberUtils.toInt(bytes, 3, 2); + KVAddress mapAddress = new KVAddress(index, offset); + return mapAddress; + + } + + public static void main(String[] args) { +// System.out.println(1L << 63); +// System.out.println(0x8000000000000000L); + + KVAddress kvAddress = new KVAddress(new ByteArray(NumberUtils.toByte((long) Integer.MAX_VALUE + 1L))); + System.out.println(kvAddress); + + } + + public boolean isEmpty() { + return !isConflict && conflictIndex == 0 && offset == 0; + } + public byte[] createBytes() { - byte[] values = NumberUtils.toByte(offset,5); - byte[] indexBytes= NumberUtils.toByte(conflictIndex); - values[3]=indexBytes[0]; - byte fisrtByte =indexBytes[1]; + byte[] values = NumberUtils.toByte(offset, 5); + byte[] indexBytes = NumberUtils.toByte(conflictIndex); + values[3] = indexBytes[0]; + byte fisrtByte = indexBytes[1]; int value = 0; if (isConflict) { @@ -96,40 +122,23 @@ public byte[] createBytes() { } values[4] = (byte) (value & 0xff); - return values; + return values; } public byte[] createBytesIngoreFirstBit() { - byte[] values = NumberUtils.toByte(offset,5); - byte[] indexBytes= NumberUtils.toByte(conflictIndex); - values[3]=indexBytes[0]; - values[4]=indexBytes[1]; + byte[] values = NumberUtils.toByte(offset, 5); + byte[] indexBytes = NumberUtils.toByte(conflictIndex); + values[3] = indexBytes[0]; + values[4] = indexBytes[1]; return values; } - public Long convertLongValue(){ - byte[] bytes=createBytesIngoreFirstBit(); + public Long convertLongValue() { + byte[] bytes = createBytesIngoreFirstBit(); return NumberUtils.toLong(bytes); } - public static KVAddress createMapAddressFromLongValue(Long value){ - byte[] bytes=NumberUtils.toByte(value); - return createAddressFromBytes(bytes); - } - - public static KVAddress createAddressFromBytes(byte[] bytes){ - int offset=NumberUtils.toInt(bytes,0,3); - byte firstByte =bytes[4]; - if(firstByte<0){ - bytes[4]=0; - } - int index=NumberUtils.toInt(bytes,3,2); - KVAddress mapAddress=new KVAddress(index,offset); - return mapAddress; - - } - public int getConflictIndex() { return conflictIndex; } @@ -137,13 +146,4 @@ public int getConflictIndex() { public int getOffset() { return offset; } - - public static void main(String[] args) { -// System.out.println(1L << 63); -// System.out.println(0x8000000000000000L); - - KVAddress kvAddress = new KVAddress(new ByteArray(NumberUtils.toByte((long) Integer.MAX_VALUE + 1L))); - System.out.println(kvAddress); - - } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVElement.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVElement.java similarity index 100% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVElement.java rename to rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVElement.java index c5139ab0..6feaabc3 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVElement.java +++ b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVElement.java @@ -57,6 +57,16 @@ public KVElement(ByteArray byteArray, boolean hasValue) { } + private KVElement(KVAddress nextAddress, byte[] keyHashCode, ByteArray value) { + this.nextAddress = nextAddress; + this.keyHashCode = new ByteArray(keyHashCode, 0, keyHashCode.length); + this.value = value; + if (value != null) { + this.hasValue = true; + this.elementSize = 26; + } + } + public static byte[] createByteArray(KVAddress nextAddress, byte[] keyHashCode, ByteArray value, boolean hasValue) { KVElement element = new KVElement(nextAddress, keyHashCode, value); if (!hasValue) { @@ -68,16 +78,6 @@ public static byte[] createByteArray(KVAddress nextAddress, byte[] keyHashCode, return element.getBytes(); } - private KVElement(KVAddress nextAddress, byte[] keyHashCode, ByteArray value) { - this.nextAddress = nextAddress; - this.keyHashCode = new ByteArray(keyHashCode, 0, keyHashCode.length); - this.value = value; - if (value != null) { - this.hasValue = true; - this.elementSize = 26; - } - } - public boolean isEmpty() { boolean empytHashCode = true; for (int i = 0; i < keyHashCode.getSize(); i++) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/MappedByteBufferStore.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/MappedByteBufferStore.java similarity index 100% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/MappedByteBufferStore.java rename to rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/MappedByteBufferStore.java diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/AbstractListKV.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/AbstractListKV.java similarity index 100% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/AbstractListKV.java rename to rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/AbstractListKV.java index 873c44fc..9ac9bfe4 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/AbstractListKV.java +++ b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/AbstractListKV.java @@ -26,6 +26,8 @@ public abstract class AbstractListKV extends CacheKV> { + private static byte[] ZERO_BYTE = NumberUtils.toByte(0, 5); + public AbstractListKV(int capacity) { super(capacity, true); } @@ -103,8 +105,6 @@ protected void add(String key, T value, ByteArray last) { } } - private static byte[] ZERO_BYTE = NumberUtils.toByte(0, 5); - /** * 创建一个元素,包含两部分:int值,int * diff --git a/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/IntValueKV.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/IntValueKV.java new file mode 100644 index 00000000..7d01618a --- /dev/null +++ b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/IntValueKV.java @@ -0,0 +1,72 @@ +/* + * 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.ByteArray; +import org.apache.rocketmq.streams.common.cache.compress.ByteStore; +import org.apache.rocketmq.streams.common.cache.compress.CacheKV; +import org.apache.rocketmq.streams.common.utils.NumberUtils; + +/** + * 支持key是string,value是int的场景,支持size不大于10000000.只支持int,long,boolean,string类型 + */ +public class IntValueKV extends CacheKV { + + protected ByteStore conflicts = new ByteStore(4); + + /** + * TODO remove the key from the sinkcache and return the removed value + * + * @return + */ + public IntValueKV(int capacity) { + super(capacity, true); + } + + @Override + public Integer get(String key) { + ByteArray byteArray = super.getInner(key); + if (byteArray == null) { + return null; + } + int value = byteArray.castInt(0, 4); + return value; + } + + @Override + public void put(String key, Integer value) { + byte[] bytes = NumberUtils.toByte(value, 5); + bytes[4] = (byte) 0; + super.putInner(key, new ByteArray(bytes), true); + } + + @Override + public boolean contains(String key) { + Integer value = get(key); + if (value == null) { + return false; + } + return true; + + } + + @Override + public int calMemory() { + return super.calMemory() + (this.conflicts.getConflictIndex() + 1) * this.conflicts.getBlockSize(); + } + +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/KeySet.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/KeySet.java similarity index 100% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/KeySet.java rename to rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/KeySet.java index 5c09412a..fa9e23d7 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/KeySet.java +++ b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/KeySet.java @@ -28,6 +28,30 @@ public KeySet(int capacity) { } + public static void main(String[] args) throws InterruptedException { + int count = 10000000; + KeySet map = new KeySet(count); + long start = System.currentTimeMillis(); + // Set values=new HashSet<>(); + //Map 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)); + } + @Override public Object get(String key) { boolean exist = contains(key); @@ -54,28 +78,4 @@ public boolean contains(String key) { public void add(String key) { super.putInner(key, null, true); } - - public static void main(String[] args) throws InterruptedException { - int count = 10000000; - KeySet map = new KeySet(count); - long start = System.currentTimeMillis(); - // Set values=new HashSet<>(); - //Map 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-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LightCache.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LightCache.java new file mode 100644 index 00000000..d12d4add --- /dev/null +++ b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LightCache.java @@ -0,0 +1,51 @@ +/* + * 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; + +public class LightCache { + protected IntValueKV intValueKV; + protected int rowCount; + + public LightCache(int rowCount) { + intValueKV = new IntValueKV(rowCount); + this.rowCount = rowCount; + } + + public Integer get(String key) { + return intValueKV.get(key); + } + + public void put(String key, Integer value) { + if (intValueKV.getSize() > rowCount) { + synchronized (this) { + if (intValueKV.getSize() >= rowCount) { + intValueKV = new IntValueKV(rowCount); + } + } + } + intValueKV.put(key, value); + } + + public boolean contains(String key) { + return intValueKV.contains(key); + } + + public int calMemory() { + return intValueKV.calMemory(); + } + +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LongListKV.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LongListKV.java similarity index 78% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LongListKV.java rename to rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LongListKV.java index f37e1748..1de559f9 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LongListKV.java +++ b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LongListKV.java @@ -20,7 +20,6 @@ import org.apache.rocketmq.streams.common.cache.compress.ByteArray; import org.apache.rocketmq.streams.common.cache.compress.ByteStore; import org.apache.rocketmq.streams.common.cache.compress.KVAddress; -import org.apache.rocketmq.streams.common.utils.AESUtil; import org.apache.rocketmq.streams.common.utils.NumberUtils; public class LongListKV extends AbstractListKV { @@ -31,25 +30,6 @@ public LongListKV(int capacity) { super(capacity); } - public static void main(String[] args) { - LongListKV intListKV = new LongListKV(10); - List values = null; -// byte[] nameBytes = "name".getBytes(); - byte[] nameHashCodeBytes = NumberUtils.toByte(calHashCode("name")); - byte[] ageHashCodeBytes = NumberUtils.toByte(calHashCode("age")); - byte[] md5 = AESUtil.stringToMD5("name"); - intListKV.add("name", 1L); - intListKV.add("name", 2L); - intListKV.add("name", 3L); - intListKV.add("name", 4L); - intListKV.add("age", 1L); - intListKV.add("age", 2L); - values = intListKV.get("name"); - for (long value : values) { - System.out.println(value); - } - } - private static int calHashCode(String key) { int hashCode = key.hashCode(); int value = hashCode ^ (hashCode >>> 16); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LongValueKV.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LongValueKV.java similarity index 100% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LongValueKV.java rename to rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LongValueKV.java diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MapAddressListKV.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MapAddressListKV.java similarity index 100% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MapAddressListKV.java rename to rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MapAddressListKV.java index dec4cd43..3fdc75da 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MapAddressListKV.java +++ b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MapAddressListKV.java @@ -31,6 +31,21 @@ public MapAddressListKV(int capacity) { super(capacity); } + public static void main(String[] args) { + int rowSize = 50000000; + MapAddressListKV intListKV = new MapAddressListKV(rowSize); + List values = null; + + for (int i = 0; i < rowSize; i++) { + intListKV.addLongValue("chris" + i, Long.valueOf(i)); + List rowIds = intListKV.getLongValue("chris" + i); + if (i != rowIds.get(0).intValue()) { + throw new RuntimeException("expect " + i + " " + rowIds.get(0)); + } + } + System.out.println("success"); + } + @Override protected byte[] convertByte(KVAddress value) { return value.createBytes(); } @@ -104,21 +119,6 @@ private byte[] createElement(ByteArray value) { return createElement(value.getByteArray()); } - public static void main(String[] args) { - int rowSize = 50000000; - MapAddressListKV intListKV = new MapAddressListKV(rowSize); - List values = null; - - for (int i = 0; i < rowSize; i++) { - intListKV.addLongValue("chris" + i, Long.valueOf(i)); - List rowIds = intListKV.getLongValue("chris" + i); - if (i != rowIds.get(0).intValue()) { - throw new RuntimeException("expect " + i + " " + rowIds.get(0)); - } - } - System.out.println("success"); - } - /** * 获取最后一个元素 * diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MultiValueKV.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MultiValueKV.java similarity index 100% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MultiValueKV.java rename to rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MultiValueKV.java diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MutilValueKV.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MutilValueKV.java similarity index 100% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MutilValueKV.java rename to rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MutilValueKV.java diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/SplitCache.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/SplitCache.java similarity index 100% rename from rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/SplitCache.java rename to rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/SplitCache.java diff --git a/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/StringValueKV.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/StringValueKV.java new file mode 100644 index 00000000..b46ec20d --- /dev/null +++ b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/StringValueKV.java @@ -0,0 +1,75 @@ +/* + * 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.io.UnsupportedEncodingException; +import org.apache.rocketmq.streams.common.cache.compress.ByteArrayValueKV; +import org.apache.rocketmq.streams.common.cache.compress.ICacheKV; + +/** + * 支持key是string,value是int的场景,支持size不大于10000000.只支持int,long,boolean,string类型 只能一次行load,不能进行更新 + */ +public class StringValueKV implements ICacheKV { + + protected final static String CODE = "UTF-8"; + protected ByteArrayValueKV values; + + public StringValueKV(int capacity) { + values = new ByteArrayValueKV(capacity); + } + + @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(); + } + +} diff --git a/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/utils/NumberUtils.java b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/utils/NumberUtils.java new file mode 100644 index 00000000..46b78eb2 --- /dev/null +++ b/rocketmq-streams-local-cache/src/main/java/org/apache/rocketmq/streams/common/utils/NumberUtils.java @@ -0,0 +1,393 @@ +/* + * 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.utils; + +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.zip.DataFormatException; +import java.util.zip.Deflater; +import java.util.zip.Inflater; +import org.apache.rocketmq.streams.common.cache.compress.ByteArray; + +public class NumberUtils { + + //todo 精度损失, 比如 3.2/9.6 + public static byte[] float2Bytes(float number) { + int i = Float.floatToIntBits(number); + byte[] b = new byte[4]; + for (int j = 0; j < 4; j++) { + b[j] = (byte) (i >> (24 - j * 8)); + } + System.out.println(i); + return b; + } + + public static float bytes2Float(byte[] bytes) { + int i = (((int) bytes[0]) << 24) + (((int) bytes[1]) << 16) + + (((int) bytes[2]) << 8) + bytes[3]; + System.out.println(i); + return Float.intBitsToFloat(i); + } + + /** + * BIG_ENDIAN + * + * @param n + * @param mask + * @return + */ + public static byte[] toByteArray(int n, int mask) { + int arrayLength = 4; + if (mask == 0xff) { + arrayLength = 1; + } else if (mask == 0xffff) { + arrayLength = 2; + } else if (mask == 0xffffff) { + arrayLength = 3; + } else if (mask == 0xffffffff) { + arrayLength = 4; + } else { + throw new RuntimeException("mask must be '0xff, 0xffff, 0xffffff, 0xffffffff.'"); + } + n = n & mask; + byte[] b = new byte[arrayLength]; + for (int i = 0; i < arrayLength; i++) { + int shift = i; + if (shift > 0) { + shift = 8 * shift; + n = (n >> shift); + } + b[i] = (byte) (n & 0xff); + } + return b; + } + + public static byte[] toByte(int n, int byteSize) { + byte[] b = new byte[byteSize]; + b[0] = (byte) (n & 0xff); + b[1] = (byte) (n >> 8 & 0xff); + b[2] = (byte) (n >> 16 & 0xff); + b[3] = (byte) (n >> 24 & 0xff); + return b; + } + + public static byte[] toByte(int n) { + return toByte(n, 4); + } + + public static byte[] toByte(long n) { + byte[] b = new byte[8]; + b[0] = (byte) (n & 0xff); + b[1] = (byte) (n >> 8 & 0xff); + b[2] = (byte) (n >> 16 & 0xff); + b[3] = (byte) (n >> 24 & 0xff); + b[4] = (byte) (n >> 32 & 0xff); + b[5] = (byte) (n >> 40 & 0xff); + b[6] = (byte) (n >> 48 & 0xff); + b[7] = (byte) (n >> 56 & 0xff); + return b; + } + + /** + * byte[] to int, index 0 means low + * + * @param b + * @return + */ + public static long toLong(byte[] b) { + long res = 0; + int i = 0; + for (; i < 8; i++) { + byte byteValue = 0; + if (i < b.length) { + byteValue = b[i]; + res += (byteValue & 0xff) << (i * 8); + } else { + res += (byteValue & 0) << (i * 8); + } + + } + + return res; + } + + public static int toInt(byte b) { + byte[] bytes = new byte[1]; + bytes[0] = b; + return toInt(bytes); + } + + public static boolean isFirstBitZero(ByteArray byteArray) { + byte firstByte = byteArray.getByte(byteArray.getSize() - 1); + return isFirstBitZero(firstByte); + } + + public static boolean isFirstBitZero(Integer integer) { + ByteArray byteArray = new ByteArray(NumberUtils.toByte(integer)); + return isFirstBitZero(byteArray); + } + + public static boolean isFirstBitZero(byte firstByte) { + int conflictValue = NumberUtils.toInt(firstByte); + int conflictFlag = conflictValue >> 7; + if (conflictFlag == 1) { + return false; + } else { + return true; + } + } + + public static int toInt(byte[] b, int startIndex, int size) { + int res = 0; + for (int i = 0; i < size; i++) { + res += (b[i + startIndex] & 0xff) << (i * 8); + } + return res; + } + + /** + * byte[] to int, index 0 means low + * + * @param b + * @return + */ + public static int toInt(byte[] b) { + int res = 0; + for (int i = 0; i < b.length; i++) { + res += (b[i] & 0xff) << (i * 8); + } + return res; + } + + public static int toInt(byte[] globalBytes, int index) { + byte[] bytes = getSubByteFromIndex(globalBytes, index, 4); + return toInt(bytes); + } + + /** + * 把一个子字节数组,插入到大大字节数组中 + * + * @param globalBytes 全局字节数组 + * @param index 从哪里插入 + * @param subBytes 待插入的子字节 + */ + public static void putSubByte2ByteArray(byte[] globalBytes, int index, byte[] subBytes) { + if (index < 0 || subBytes == null) { + return; + } + for (int i = 0; i < subBytes.length; i++) { + globalBytes[i + index] = subBytes[i]; + } + } + + /** + * 从总字节中获取部分字节 + * + * @param globalBytes + * @param index + * @param size + * @return + */ + public static byte[] getSubByteFromIndex(byte[] globalBytes, int index, int size) { + if (index < 0 || size < 0) { + return null; + } + byte[] subBytes = Arrays.copyOfRange(globalBytes, index, size + index); + return subBytes; + } + + /** + * 把存储0/1字符串的值,转化成bit + * + * @param booleanValues + * @return + */ + public static int createBitMapInt(Boolean... booleanValues) { + List values = new ArrayList<>(); + for (Boolean value : booleanValues) { + if (value != null && value == true) { + values.add("1"); + } else { + values.add(null); + } + + } + return createBitMapInt(values); + } + + /** + * 把存储0/1字符串的值,转化成bit + * + * @param values + * @return + */ + public static int createBitMapInt(List values) { + if (values == null) { + return 0; + } + int result = 0; + int i = 0; + for (String value : values) { + if (value != null && "1".equals(value)) { + result = (result | (1 << i)); + } + i++; + } + return result; + } + + /** + * 获取某位的值,如果是1,返回字符串1,否则返回null + * + * @param num + * @param index + * @return + */ + public static boolean getNumFromBitMapInt(int num, int index) { + boolean isTrue = ((num & (1 << index)) != 0);//true 表示第i位为1,否则为0 + return isTrue; + } + + /** + * 获取某位的值,如果是1,返回字符串1,否则返回null + * + * @param num + * @param index + * @return + */ + public static int setNumFromBitMapInt(int num, int index) { + num = (num | (1 << index)); + return num; + } + + //添加input的length + public static byte[] zlibCompress(byte[] input) { +// try { +// ByteArrayOutputStream out = new ByteArrayOutputStream(); +// GZIPOutputStream gzip = new GZIPOutputStream(out); +// gzip.write(input); +// gzip.close(); +// byte[] dataArray = out.toString("ISO-8859-1").getBytes("ISO-8859-1"); +//// return dataArray; +// byte[] lengthArray = toByteArray(dataArray.length, 0xffff); +// byte[] ret = new byte[dataArray.length + lengthArray.length]; +// ret[0] = lengthArray[0]; +// ret[1] = lengthArray[1]; +// System.arraycopy(dataArray, 0, ret, 2, dataArray.length); +// return ret; +// } catch (IOException e) { +// e.printStackTrace(); +// return null; +// } + +// byte[] output = new byte[input.length + 10 + new Double(Math.ceil(input.length*0.25f)).intValue()]; + if (input == null || input.length == 0) { + return new byte[0]; + } + int length = input.length; + if (length < 256) { + length = length * 3 + 10; + } else { + length = input.length + 10 + new Double(Math.ceil(input.length * 0.25f)).intValue(); + } + byte[] output = new byte[length]; + Deflater compresser = new Deflater(); + compresser.setInput(input); + compresser.finish(); + int compressedDataLength = compresser.deflate(output); + byte[] lengthArray = toByteArray(input.length, 0xffff); + byte[] ret = new byte[compressedDataLength + lengthArray.length]; + ret[0] = lengthArray[0]; + ret[1] = lengthArray[1]; + compresser.end(); + System.arraycopy(output, 0, ret, 2, compressedDataLength); + return ret; + } + + /** + * 解压缩 + * + * @param barr 须要解压缩的字节数组 + * @return + * @throws Exception + */ + public static byte[] zlibInfCompress(byte[] barr) throws DataFormatException { + if (barr == null || barr.length == 0) { + return new byte[0]; + } + int length = (int) (barr[0] & 0xff) + ((barr[1] & 0xff) << 8); + byte[] data = new byte[barr.length - 2]; + System.arraycopy(barr, 2, data, 0, data.length); +// ByteArrayOutputStream out = new ByteArrayOutputStream(); +// ByteArrayInputStream in = new ByteArrayInputStream(data); +// GZIPInputStream gzip1 = new GZIPInputStream(in); +// byte[] output = new byte[length]; +// int n; +// while( (n = gzip1.read(output)) >= 0){ +// out.write(output, 0, n); +// } +// return output; + byte[] result = new byte[length]; + Inflater inf = new Inflater(); + inf.setInput(data); + int infLen = inf.inflate(result); + inf.end(); + return Arrays.copyOf(result, infLen); + } + + /** + * 去掉小数点,防止序列化反序列化过程中出现类型不一致,因为计算过程中统一使用了double类型 + * + * @param value + * @return + */ + public static Number stripTrailingZeros(double value) { + BigDecimal decimal = new BigDecimal(value); + BigDecimal result = decimal.stripTrailingZeros(); + int scale = result.scale(); + if (scale <= 0) { + return decimal.intValue(); + } else { + return result.doubleValue(); + } + } + + public static void main(String[] args) throws Exception { + String[] arrs = new String[8]; + arrs[0] = "ab947a25-d5ae-41c7-9675-1a34b66c55aa_30312_rg cnt/20211101/202111010000.xz cnt/20211101/202111010001.xz cnt/20211101/202111010002.xz cnt/20211101/202111010003.xz cnt/20211101/202111010004.xz cnt/20211101/202111010005.xz cnt/20211101/202111010006.xz cnt/20211101/202111010007.xz cnt/20211101/202111010008.xz cnt/20211101/202111010009.xz cnt/20211101/202111010010.xz cnt/20211101/202111010011.xz cnt/20211101/202111010012.xz cnt/20211101/202111010013.xz cnt/20211101/202111010014.xz cnt/20211101/202111010015.xz cnt/20211101/202111010016.xz cnt/20211101/202111010017.xz cnt/20211101/202111010018.xz cnt/20211101/202111010019.xz cnt/20211101/202111010020.xz cnt/20211101/202111010021.xz cnt/20211101/202111010022.xz cnt/20211101/202111010023.xz cnt/20211101/202111010024.xz cnt/20211101/202111010025.xz cnt/20211101/202111010026.xz cnt/20211101/202111010027.xz cnt/20211101/202111010028.xz cnt/20211101/202111010029.xz cnt/20211101/202111010030.xz cnt/20211101/202111010031.xz cnt/20211101/202111010032.xz cnt/20211101/202111010033.xz cnt/20211101/202111010034.xz cnt/20211101/202111010035.xz cnt/20211101/202111010036.xz cnt/20211101/202111010037.xz cnt/20211101/202111010038.xz cnt/20211101/202111010039.xz cnt/20211101/202111010040.xz cnt/20211101/202111010041.xz cnt/20211101/202111010042.xz cnt/20211101/202111010043.xz cnt/20211101/202111010044.xz cnt/20211101/202111010045.xz cnt/20211101/202111010046.xz cnt/20211101/202111010047.xz cnt/20211101/202111010048.xz cnt/20211101/202111010049.xz cnt/20211101/202111010050.xz cnt/20211101/202111010051.xz cnt/20211101/202111010052.xz cnt/20211101/202111010053.xz cnt/20211101/202111010054.xz cnt/20211101/202111010055.xz cnt/20211101/202111010056.xz cnt/20211101/202111010057.xz cnt/20211101/202111010058.xz cnt/20211101/202111010059.xz cnt/20211101/202111010100.xz cnt/20211101/202111010101.xz cnt/20211101/202111010102.xz cnt/20211101/202111010103.xz cnt/20211101/202111010104.xz cnt/20211101/202111010105.xz cnt/20211101/202111010106.xz cnt/20211101/202111010107.xz cnt/20211101/202111010108.xz cnt/20211101/202111010109.xz cnt/20211101/202111010110.xz cnt/20211101/202111010111.xz cnt/20211101/202111010112.xz cnt/20211101/202111010113.xz cnt/20211101/202111010114.xz cnt/20211101/202111010115.xz cnt/20211101/202111010116.xz cnt/20211101/202111010117.xz cnt/20211101/202111010118.xz cnt/20211101/202111010119.xz cnt/20211101/202111010120.xz cnt/20211101/202111010121.xz cnt/20211101/202111010122.xz cnt/20211101/202111010123.xz cnt/20211101/202111010124.xz cnt/20211101/202111010125.xz cnt/20211101/202111010126.xz cnt/20211101/202111010127.xz cnt/20211101/202111010128.xz cnt/20211101/202111010129.xz cnt/20211101/202111010130.xz cnt/20211101/202111010131.xz cnt/20211101/202111010132.xz cnt/20211101/202111010133.xz cnt/20211101/202111010134.xz cnt/20211101/202111010135.xz cnt/20211101/202111010136.xz cnt/20211101/202111010137.xz cnt/20211101/202111010138.xz cnt/20211101/202111010139.xz cnt/20211101/202111010140.xz cnt/20211101/202111010141.xz cnt/20211101/202111010142.xz cnt/20211101/202111010143.xz cnt/20211101/202111010144.xz cnt/20211101/202111010145.xz cnt/20211101/202111010146.xz cnt/20211101/202111010147.xz cnt/20211101/202111010148.xz cnt/20211101/202111010149.xz cnt/20211101/202111010150.xz cnt/20211101/202111010151.xz cnt/20211101/202111010152.xz cnt/20211101/202111010153.xz cnt/20211101/202111010154.xz cnt/20211101/202111010155.xz cnt/20211101/202111010156.xz cnt/20211101/202111010157.xz cnt/20211101/202111010158.xz cnt/20211101/202111010159.xz cnt/20211101/202111010200.xz cnt/20211101/202111010201.xz cnt/20211101/202111010202.xz cnt/20211101/202111010203.xz cnt/20211101/202111010204.xz cnt/20211101/202111010205.xz cnt/20211101/202111010206.xz cnt/20211101/202111010207.xz cnt/20211101/202111010208.xz cnt/20211101/202111010209.xz cnt/20211101/202111010210.xz cnt/20211101/202111010211.xz cnt/20211101/202111010212.xz cnt/20211101/202111010213.xz cnt/20211101/202111010214.xz cnt/20211101/202111010215.xz cnt/20211101/202111010216.xz cnt/20211101/202111010217.xz cnt/20211101/202111010218.xz cnt/20211101/202111010219.xz cnt/20211101/202111010220.xz cnt/20211101/202111010221.xz cnt/20211101/202111010222.xz cnt/20211101/202111010223.xz cnt/20211101/202111010224.xz cnt/20211101/202111010225.xz cnt/20211101/202111010226.xz cnt/20211101/202111010227.xz cnt/20211101/202111010228.xz cnt/20211101/202111010229.xz cnt/20211101/202111010230.xz cnt/20211101/202111010231.xz cnt/20211101/202111010232.xz cnt/20211101/202111010233.xz cnt/20211101/202111010234.xz cnt/20211101/202111010235.xz cnt/20211101/202111010236.xz cnt/20211101/202111010237.xz cnt/20211101/202111010238.xz cnt/20211101/202111010239.xz cnt/20211101/202111010240.xz cnt/20211101/202111010241.xz cnt/20211101/202111010242.xz cnt/20211101/202111010243.xz cnt/20211101/202111010244.xz cnt/20211101/202111010245.xz cnt/20211101/202111010246.xz cnt/20211101/202111010247.xz cnt/20211101/202111010248.xz cnt/20211101/202111010249.xz cnt/20211101/202111010250.xz cnt/20211101/202111010251.xz cnt/20211101/202111010252.xz cnt/20211101/202111010253.xz cnt/20211101/202111010254.xz cnt/20211101/202111010255.xz cnt/20211101/202111010256.xz cnt/20211101/202111010257.xz cnt/20211101/202111010258.xz cnt/20211101/202111010259.xz cnt/20211101/202111010300.xz cnt/20211101/202111010301.xz cnt/20211101/202111010302.xz cnt/20211101/202111010303.xz cnt/20211101/202111010304.xz cnt/20211101/202111010305.xz cnt/20211101/202111010306.xz cnt/20211101/202111010307.xz cnt/20211101/202111010308.xz cnt/20211101/202111010309.xz cnt/20211101/202111010310.xz cnt/20211101/202111010311.xz cnt/20211101/202111010312.xz cnt/20211101/202111010313.xz cnt/20211101/202111010314.xz cnt/20211101/202111010315.xz cnt/20211101/202111010316.xz cnt/20211101/202111010317.xz cnt/20211101/202111010318.xz cnt/20211101/202111010319.xz cnt/20211101/202111010320.xz cnt/20211101/202111010321.xz cnt/20211101/202111010322.xz cnt/20211101/202111010323.xz cnt/20211101/202111010324.xz cnt/20211101/202111010325.xz cnt/20211101/202111010326.xz cnt/20211101/202111010327.xz cnt/20211101/202111010328.xz cnt/20211101/202111010329.xz cnt/20211101/202111010330.xz cnt/20211101/202111010331.xz cnt/20211101/202111010332.xz cnt/20211101/202111010333.xz cnt/20211101/202111010334.xz cnt/20211101/202111010335.xz cnt/20211101/202111010336.xz cnt/20211101/202111010337.xz cnt/20211101/202111010338.xz cnt/20211101/202111010339.xz cnt/20211101/202111010340.xz cnt/20211101/202111010341.xz cnt/20211101/202111010342.xz cnt/20211101/202111010343.xz cnt/20211101/202111010344.xz cnt/20211101/202111010345.xz cnt/20211101/202111010346.xz cnt/20211101/202111010347.xz cnt/20211101/202111010348.xz cnt/20211101/202111010349.xz cnt/20211101/202111010350.xz cnt/20211101/202111010351.xz cnt/20211101/202111010352.xz cnt/20211101/202111010353.xz cnt/20211101/202111010354.xz cnt/20211101/202111010355.xz cnt/20211101/202111010356.xz cnt/20211101/202111010357.xz cnt/20211101/202111010358.xz cnt/20211101/202111010359.xz cnt/20211101/202111010400.xz cnt/20211101/202111010401.xz cnt/20211101/202111010402.xz cnt/20211101/202111010403.xz cnt/20211101/202111010404.xz cnt/20211101/202111010405.xz cnt/20211101/202111010406.xz cnt/20211101/202111010407.xz cnt/20211101/202111010408.xz cnt/20211101/202111010409.xz cnt/20211101/202111010410.xz cnt/20211101/202111010411.xz cnt/20211101/202111010412.xz cnt/20211101/202111010413.xz cnt/20211101/202111010414.xz cnt/20211101/202111010415.xz cnt/20211101/202111010416.xz cnt/20211101/202111010417.xz cnt/20211101/202111010418.xz cnt/20211101/202111010419.xz cnt/20211101/202111010420.xz cnt/20211101/202111010421.xz cnt/20211101/202111010422.xz cnt/20211101/202111010423.xz cnt/20211101/202111010424.xz cnt/20211101/202111010425.xz cnt/20211101/202111010426.xz cnt/20211101/202111010427.xz cnt/20211101/202111010428.xz cnt/20211101/202111010429.xz cnt/20211101/202111010430.xz cnt/20211101/202111010431.xz cnt/20211101/202111010432.xz cnt/20211101/202111010433.xz cnt/20211101/202111010434.xz cnt/20211101/202111010435.xz cnt/20211101/202111010436.xz cnt/20211101/202111010437.xz cnt/20211101/202111010438.xz cnt/20211101/202111010439.xz cnt/20211101/202111010440.xz cnt/20211101/202111010441.xz cnt/2021110_/usr/local/ripgrep-13.0.0-x86_64-unknown-linux-musl/rg_30282"; + arrs[1] = "10.125.126.01"; + arrs[2] = "ababababab"; + arrs[3] = "1234567890"; + arrs[4] = "1234567890abcdefghigklmnopqrstuvwxyz"; + arrs[5] = "1"; + arrs[6] = "(squid-1) -f /etc/squid/squid.conf"; + arrs[7] = "1af15f44-8b7f-4814-9a65-02b6c77d5309"; + + for (String str : arrs) { + System.out.print("raw string : " + str.getBytes().length); + byte[] bytes1 = zlibCompress(str.getBytes()); + System.out.print(", compress string length : " + bytes1.length); + byte[] bytes2 = zlibInfCompress(bytes1); + String str2 = new String(bytes2); + System.out.println(", equals : " + str2.equals(str)); + } + + } + +} diff --git a/rocketmq-streams-local-cache/src/test/java/org/apache/rocketmq/streams/common/cache/compress/FingerprintCacheTest.java b/rocketmq-streams-local-cache/src/test/java/org/apache/rocketmq/streams/common/cache/compress/FingerprintCacheTest.java new file mode 100644 index 00000000..c03f9686 --- /dev/null +++ b/rocketmq-streams-local-cache/src/test/java/org/apache/rocketmq/streams/common/cache/compress/FingerprintCacheTest.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.cache.compress; + +import org.apache.rocketmq.streams.common.cache.compress.impl.LightCache; +import org.junit.Test; + +public class FingerprintCacheTest { + + @Test + public void testFingerprintCache() { + int cacheSize = 10000000; + LightCache localCache = new LightCache(cacheSize); + for (int i = 0; i < cacheSize; i++) { + localCache.put("1.1.1." + i, i); + } + System.out.println(cacheSize + " cost memory is " + localCache.calMemory() + "M"); + } +} diff --git a/rocketmq-streams-local-cache/src/test/java/org/apache/rocketmq/streams/common/cache/compress/IntValueKVTest.java b/rocketmq-streams-local-cache/src/test/java/org/apache/rocketmq/streams/common/cache/compress/IntValueKVTest.java new file mode 100644 index 00000000..9529a521 --- /dev/null +++ b/rocketmq-streams-local-cache/src/test/java/org/apache/rocketmq/streams/common/cache/compress/IntValueKVTest.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; + +import org.apache.rocketmq.streams.common.cache.compress.impl.IntValueKV; +import org.apache.rocketmq.streams.common.cache.compress.impl.KeySet; +import org.apache.rocketmq.streams.common.utils.NumberUtils; +import org.junit.Test; + +public class IntValueKVTest { + + /** + * 高压缩缓存,只支持固定值存储,目前主要应用IntValueKV,key是任意字符串,value是int值 + */ + @Test + public void testIntValueKV() { + int capacity = 1000000; + IntValueKV cache = new IntValueKV(capacity); + System.out.println(cache.calMemory());//内存预先分配,100w预先分配22m内存 + for (int i = 0; i < capacity; i++) { + cache.put("name" + i, i); + } + System.out.println(cache.calMemory());//由于存在hash冲突,插入100w数据,内存大概31m + System.out.println(cache.get("name0") == 0); + } + + /** + * 内存会占用更少 + */ + @Test + public void testKeySet() { + int capacity = 1000000; + KeySet cache = new KeySet(capacity); + System.out.println(cache.calMemory());//内存预先分配,100w预先分配19m内存 + for (int i = 0; i < capacity; i++) { + cache.add("name" + i); + } + System.out.println(cache.calMemory());//由于存在hash冲突,插入100w数据,内存大概27m + Boolean result = cache.contains("name1") == true; + System.out.println(result); + } + + /** + * 一个int数据,可以按位存储boolean信息,提供了NumberUtils工具类处理数字的bit操作 缓存可以结合NumberUtils,使int存储多个boolean值 + */ + @Test + public void testNumUtil() { + int num = 0; + num = NumberUtils.setNumFromBitMapInt(num, 3);//设置num的第3位为1 + System.out.println(NumberUtils.getNumFromBitMapInt(num, 3) == true);//获取第3位是否为1 + } +} diff --git a/rocketmq-streams-commons/src/test/resources/log4j.xml b/rocketmq-streams-local-cache/src/test/resources/log4j.xml old mode 100755 new mode 100644 similarity index 100% rename from rocketmq-streams-commons/src/test/resources/log4j.xml rename to rocketmq-streams-local-cache/src/test/resources/log4j.xml diff --git a/rocketmq-streams-runner/assembly/distribution.xml b/rocketmq-streams-runner/assembly/distribution.xml deleted file mode 100644 index f44e34fe..00000000 --- a/rocketmq-streams-runner/assembly/distribution.xml +++ /dev/null @@ -1,69 +0,0 @@ - - - distribution - - tar.gz - - - - 0775 - target/rocketmq-streams-${project.version}-standalone/bin - bin - - - target/rocketmq-streams-${project.version}-standalone/conf - conf - - - target/rocketmq-streams-${project.version}-standalone/jobs - jobs - - - 0775 - target/rocketmq-streams-${project.version}-standalone/lib - lib - - - 0775 - target/rocketmq-streams-${project.version}-standalone/log - log - - - - - target/rocketmq-streams-${project.version}-standalone/LICENSE - - - - target/rocketmq-streams-${project.version}-standalone/NOTICE - - - - target/rocketmq-streams-${project.version}-standalone/quick_start.md - - - - target/rocketmq-streams-${project.version}-standalone/README.md - - - - target/rocketmq-streams-${project.version}-standalone/README-chinese.md - - - - diff --git a/rocketmq-streams-runner/assembly/standalone.xml b/rocketmq-streams-runner/assembly/standalone.xml deleted file mode 100644 index 69075a35..00000000 --- a/rocketmq-streams-runner/assembly/standalone.xml +++ /dev/null @@ -1,72 +0,0 @@ - - - standalone - - dir - - false - - - - bin - /bin - 0755 - - - src/main/resources - /conf - 0755 - - - conf - /conf - 0755 - - - jobs - /jobs - 0755 - - - log - /log - 0755 - - - ../ - / - - LICENSE - NOTICE - quick_start.md - README.md - README-chinese.md - - - - - - - true - lib - - runtime - - - - diff --git a/rocketmq-streams-runner/bin/start.sh b/rocketmq-streams-runner/bin/start.sh deleted file mode 100755 index 44797c45..00000000 --- a/rocketmq-streams-runner/bin/start.sh +++ /dev/null @@ -1,58 +0,0 @@ -#!/bin/sh -set -e - -PROG_NAME=$0 -MAIN_CLASS=$1 - -if [ -z "${MAIN_CLASS}" ]; then - usage -fi - -usage() { - echo "Usage: $PROG_NAME {mainClass or mainClasses splited with comma}" - exit 2 # bad usage -} - - -JVM_CONFIG=$2 -if [ -z "${JVM_CONFIG}" ]; then - JVM_CONFIG="-Xms2048m -Xmx2048m -Xss512k" -fi - -ROCKETMQ_STREAMS_HOME=$(cd $(dirname ${BASH_SOURCE[0]})/..; pwd) -ROCKETMQ_STREAMS_JOBS_DIR=$ROCKETMQ_STREAMS_HOME/jobs -ROCKETMQ_STREAMS_DEPENDENCIES=$ROCKETMQ_STREAMS_HOME/lib -ROCKETMQ_STREAMS_LOGS=$ROCKETMQ_STREAMS_HOME/log/catalina.out - -if [ -z "${JAVA_HOME:-}" ]; then - JAVA="java -server" -else - JAVA="$JAVA_HOME/bin/java -server" -fi - -JAVA_OPTIONS=${JAVA_OPTIONS:-} - -JVM_OPTS=() -if [ ! -z "${JAVA_OPTIONS}" ]; then - JVM_OPTS+=("${JAVA_OPTIONS}") -fi -if [ ! -z "${JVM_CONFIG}" ]; then - JVM_OPTS+=("${JVM_CONFIG}") -fi - -JVM_OPTS+=("-Dlogback.configurationFile=conf/logback.xml") - - - -# shellcheck disable=SC2039 -# shellcheck disable=SC2206 -array=(${MAIN_CLASS//,/ }) - -# shellcheck disable=SC2068 -# shellcheck disable=SC2039 -for var in ${array[@]} -do - # shellcheck disable=SC2068 - # shellcheck disable=SC2039 - eval exec $JAVA ${JVM_OPTS[@]} -classpath "$ROCKETMQ_STREAMS_JOBS_DIR/*:$ROCKETMQ_STREAMS_DEPENDENCIES/*" $var "&" >>"$ROCKETMQ_STREAMS_LOGS" 2>&1 -done diff --git a/rocketmq-streams-runner/bin/stop.sh b/rocketmq-streams-runner/bin/stop.sh deleted file mode 100755 index 5734c926..00000000 --- a/rocketmq-streams-runner/bin/stop.sh +++ /dev/null @@ -1,33 +0,0 @@ -#!/bin/sh -set -e -PROG_NAME=$0 -MAIN_CLASS=$1 - -if [ -z "${MAIN_CLASS}" ]; then - usage -fi - -# shellcheck disable=SC2039 -# shellcheck disable=SC2206 -array=(${MAIN_CLASS//,/ }) - -# shellcheck disable=SC2068 -# shellcheck disable=SC2039 -for var in ${array[@]} -do - STREAM_JOB_PIC="$(ps -ef | grep "$var" | grep -v grep | grep -v "$PROG_NAME" | awk '{print $2}' | sed 's/addr://g')" - if [ ! -z "$STREAM_JOB_PIC" ]; then - echo $STREAM_JOB_PIC - echo "Stop rocketmq-streams job" - echo "kill -9 $STREAM_JOB_PIC" - kill -9 $STREAM_JOB_PIC - echo "Job($MAIN_CLASS) shutdown completed." - else - echo "Job($MAIN_CLASS) not started." - fi -done - - - - - diff --git a/rocketmq-streams-runner/log/catalina.out b/rocketmq-streams-runner/log/catalina.out deleted file mode 100644 index e69de29b..00000000 diff --git a/rocketmq-streams-runner/pom.xml b/rocketmq-streams-runner/pom.xml deleted file mode 100644 index bcee09c8..00000000 --- a/rocketmq-streams-runner/pom.xml +++ /dev/null @@ -1,66 +0,0 @@ - - - - rocketmq-streams - org.apache.rocketmq - 1.0.0-SNAPSHOT - - 4.0.0 - - rocketmq-streams-runner - ROCKETMQ STREAMS :: runner - - - 8 - 8 - - - - org.apache.rocketmq - rocketmq-streams-clients - - - org.apache.rocketmq - rocketmq-streams-examples - - - - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - org.apache.maven.plugins - maven-failsafe-plugin - - - org.apache.maven.plugins - maven-assembly-plugin - - - package - - single - - - - - true - - assembly/standalone.xml - assembly/distribution.xml - - rocketmq-streams-${project.version} - target - target/assembly/work - warn - - - - - \ No newline at end of file diff --git a/rocketmq-streams-runner/src/main/resources/log4j.xml b/rocketmq-streams-runner/src/main/resources/log4j.xml deleted file mode 100644 index de547783..00000000 --- a/rocketmq-streams-runner/src/main/resources/log4j.xml +++ /dev/null @@ -1,51 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/rocketmq-streams-schedule/pom.xml b/rocketmq-streams-schedule/pom.xml index 51fce7c3..a8c265ce 100755 --- a/rocketmq-streams-schedule/pom.xml +++ b/rocketmq-streams-schedule/pom.xml @@ -15,14 +15,14 @@ See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 org.apache.rocketmq rocketmq-streams - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT rocketmq-streams-schedule ROCKETMQ STREAMS :: schedule @@ -34,14 +34,14 @@ org.quartz-scheduler quartz - - ch.qos.logback - logback-classic - - - ch.qos.logback - logback-core - + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + diff --git a/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/ScheduleComponent.java b/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/ScheduleComponent.java index c06a7d40..6bc21d70 100644 --- a/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/ScheduleComponent.java +++ b/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/ScheduleComponent.java @@ -26,12 +26,6 @@ public class ScheduleComponent extends AbstractComponent { private static ScheduleComponent scheduleComponent; protected ScheduleServiceImpl scheduleService = new ScheduleServiceImpl(); - @Override - public boolean stop() { - scheduleService.stop(); - return true; - } - public static ScheduleComponent getInstance() { if (scheduleComponent != null) { return scheduleComponent; @@ -47,12 +41,16 @@ public static ScheduleComponent getInstance() { } @Override - public IScheduleService getService() { + public boolean stop() { + scheduleService.stop(); + return true; + } + + @Override public IScheduleService getService() { return scheduleService; } - @Override - protected boolean startComponent(String name) { + @Override protected boolean startComponent(String namespace) { scheduleService.start(); return true; } diff --git a/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/job/ConfigurableExecutorJob.java b/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/job/ConfigurableExecutorJob.java index 2b2c3ed5..6008b1b3 100644 --- a/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/job/ConfigurableExecutorJob.java +++ b/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/job/ConfigurableExecutorJob.java @@ -16,22 +16,13 @@ */ package org.apache.rocketmq.streams.schedule.job; -import com.alibaba.fastjson.JSON; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.interfaces.IScheduleExecutor; -import org.apache.rocketmq.streams.common.monitor.IMonitor; -import org.apache.rocketmq.streams.common.monitor.MonitorFactory; -import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.quartz.DisallowConcurrentExecution; -import org.quartz.JobDetail; import org.quartz.JobExecutionContext; import org.quartz.JobExecutionException; import org.quartz.StatefulJob; @DisallowConcurrentExecution public class ConfigurableExecutorJob implements StatefulJob { - private Log LOG = LogFactory.getLog(ConfigurableExecutorJob.class); @Override public void execute(JobExecutionContext context) throws JobExecutionException { diff --git a/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/service/impl/ScheduleServiceImpl.java b/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/service/impl/ScheduleServiceImpl.java index a481ce21..36720539 100644 --- a/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/service/impl/ScheduleServiceImpl.java +++ b/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/service/impl/ScheduleServiceImpl.java @@ -18,8 +18,6 @@ import java.util.Date; import java.util.concurrent.TimeUnit; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.interfaces.IScheduleExecutor; import org.apache.rocketmq.streams.common.utils.DateUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; @@ -37,7 +35,6 @@ import static org.quartz.TriggerBuilder.newTrigger; public class ScheduleServiceImpl implements IScheduleService { - private static final Log LOG = LogFactory.getLog(ScheduleServiceImpl.class); protected Scheduler scheduler; public ScheduleServiceImpl() { @@ -48,12 +45,24 @@ public ScheduleServiceImpl() { } } + public static void main(String[] args) throws SchedulerException { + Date date = new Date(); + int second = DateUtil.getSecond(date); + int mintue = DateUtil.getMinute(date); + int hour = DateUtil.getHour(date); + int day = DateUtil.getDay(date); + int month = DateUtil.getMonth(date); + String year = DateUtil.getYear(date); + String cron = second + " " + mintue + " " + hour + " " + day + " " + month + " ? " + year; + System.out.println(cron); + } + @Override - public void startSchedule(IScheduleExecutor channelExecutor, String crotabStr, boolean startNow) { - String name = MapKeyUtil.createKey(channelExecutor.getType(), channelExecutor.getConfigureName()); + public void startSchedule(IScheduleExecutor channelExecutor, String cronTable, boolean startNow) { + String name = MapKeyUtil.createKey(channelExecutor.getType(), channelExecutor.getName()); Trigger trigger = newTrigger() .withIdentity(name, channelExecutor.getNameSpace()) - .withSchedule(cronSchedule(crotabStr)) + .withSchedule(cronSchedule(cronTable)) .forJob(name, channelExecutor.getNameSpace()) .build(); try { @@ -97,11 +106,11 @@ public String convertCron(int secondPollingTime) { String cron = null; if (secondPollingTime < 60) { cron = "0/" + secondPollingTime + " * * * * ?"; - } else if (secondPollingTime >= 60 && secondPollingTime < 3600) { + } else if (secondPollingTime < 3600) { int minute = secondPollingTime / 60; int second = secondPollingTime % 60; cron = second + " 0/" + minute + " * * * ?"; - } else if (secondPollingTime >= 3600 && secondPollingTime < 86400) { + } else if (secondPollingTime < 86400) { int hour = secondPollingTime / 3600; int other = secondPollingTime % 3600; String minuteStr = "0"; @@ -115,7 +124,7 @@ public String convertCron(int secondPollingTime) { secondStr = other + ""; } cron = secondStr + " " + minuteStr + " " + "0/" + hour + " * * ?"; - } else if (secondPollingTime >= 86400 && secondPollingTime < 86400 * 2) { + } else if (secondPollingTime < 86400 * 2) { secondPollingTime = secondPollingTime % 86400; int hour = secondPollingTime / 3600; int other = secondPollingTime % 3600; @@ -138,20 +147,20 @@ public String convertCron(int secondPollingTime) { @Override public void startScheduleUsingPollingTime(IScheduleExecutor channelExecutor, int pollingTime, TimeUnit timeUnit, boolean startNow) { - String crotabStr = createCrotablStr(pollingTime, timeUnit); - startSchedule(channelExecutor, crotabStr, startNow); + String cronTable = createCronTableStr(pollingTime, timeUnit); + startSchedule(channelExecutor, cronTable, startNow); } @Override public void startScheduleDailyTime(IScheduleExecutor channelExecutor, String dateTime, boolean startNow) { - String crotabStr = createCrotablStr(dateTime, TimeUnit.DAYS); - startSchedule(channelExecutor, crotabStr, startNow); + String cronTable = createCronTableStr(dateTime, TimeUnit.DAYS); + startSchedule(channelExecutor, cronTable, startNow); } @Override public void startScheduleHourTime(IScheduleExecutor channelExecutor, String dateTime, boolean startNow) { - String crotabStr = createCrotablStr(dateTime, TimeUnit.HOURS); - startSchedule(channelExecutor, crotabStr, startNow); + String cronTable = createCronTableStr(dateTime, TimeUnit.HOURS); + startSchedule(channelExecutor, cronTable, startNow); } public void start() { @@ -179,12 +188,11 @@ public void stop() { protected JobDetail createJobDetail(IScheduleExecutor executor) { JobDataMap jobDataMap = new JobDataMap(); jobDataMap.put(IScheduleExecutor.class.getName(), executor); - String name = MapKeyUtil.createKey(executor.getType(), executor.getConfigureName()); - JobDetail jobDetail = JobBuilder.newJob(ConfigurableExecutorJob.class) + String name = MapKeyUtil.createKey(executor.getType(), executor.getName()); + return JobBuilder.newJob(ConfigurableExecutorJob.class) .withIdentity(name, executor.getNameSpace()) // name "myJob", group "group1" .usingJobData(jobDataMap) .build(); - return jobDetail; } /** @@ -194,8 +202,8 @@ protected JobDetail createJobDetail(IScheduleExecutor executor) { * @param timeUnit * @return */ - protected String createCrotablStr(int pollingTime, TimeUnit timeUnit) { - String crotab = null; + protected String createCronTableStr(int pollingTime, TimeUnit timeUnit) { + String cronTable = null; if (pollingTime > 60) { throw new RuntimeException("pollingTime can not exceed 60, must in the unit " + timeUnit + ". the value is " + pollingTime); } @@ -203,17 +211,17 @@ protected String createCrotablStr(int pollingTime, TimeUnit timeUnit) { throw new RuntimeException("pollingTime can not exceed 31, must in the day unit . the value is " + pollingTime); } if (TimeUnit.SECONDS == timeUnit) { - crotab = "0/" + pollingTime + " * * * * ?"; + cronTable = "0/" + pollingTime + " * * * * ?"; } else if (TimeUnit.MINUTES == timeUnit) { - crotab = "0 0/" + pollingTime + " * * * ?"; + cronTable = "0 0/" + pollingTime + " * * * ?"; } else if (TimeUnit.HOURS == timeUnit) { - crotab = "0 0 0/" + pollingTime + " * * ?"; + cronTable = "0 0 0/" + pollingTime + " * * ?"; } else if (TimeUnit.DAYS == timeUnit) { - crotab = "0 0 0 0/" + pollingTime + " * ?"; + cronTable = "0 0 0 0/" + pollingTime + " * ?"; } else { throw new RuntimeException("can not support the timeunit"); } - return crotab; + return cronTable; } /** @@ -223,23 +231,23 @@ protected String createCrotablStr(int pollingTime, TimeUnit timeUnit) { * @param timeUnit * @return */ - protected String createCrotablStr(String dateTime, TimeUnit timeUnit) { - String crotab = null; + protected String createCronTableStr(String dateTime, TimeUnit timeUnit) { + String cronTable = null; if (TimeUnit.DAYS == timeUnit) { String[] values = dateTime.split(":"); String hour = getTimeValue(values[0]); String minute = getTimeValue(values[1]); String second = getTimeValue(values[2]); - crotab = second + " " + minute + " " + hour + " 0/1 * ?"; + cronTable = second + " " + minute + " " + hour + " 0/1 * ?"; } else if (TimeUnit.HOURS == timeUnit) { String[] values = dateTime.split(":"); String minute = getTimeValue(values[0]); String second = getTimeValue(values[1]); - crotab = second + " " + minute + " 0/1 * * ?"; + cronTable = second + " " + minute + " 0/1 * * ?"; } else { throw new RuntimeException("can not support the timeunit"); } - return crotab; + return cronTable; } private String getTimeValue(String value) { @@ -248,16 +256,4 @@ private String getTimeValue(String value) { } return value; } - - public static void main(String[] args) throws SchedulerException { - Date date = new Date(); - int second = DateUtil.getSecond(date); - int mintue = DateUtil.getMinute(date); - int hour = DateUtil.getHour(date); - int day = DateUtil.getDay(date); - int month = DateUtil.getMonth(date); - String year = DateUtil.getYear(date); - String cron = second + " " + mintue + " " + hour + " " + day + " " + month + " ? " + year; - System.out.println(cron); - } } diff --git a/rocketmq-streams-schedule/src/test/java/org/apache/rocketmq/stream/schedule/ScheduleComponentTest.java b/rocketmq-streams-schedule/src/test/java/org/apache/rocketmq/stream/schedule/ScheduleComponentTest.java index 69e34e6f..8ce1bdf2 100644 --- a/rocketmq-streams-schedule/src/test/java/org/apache/rocketmq/stream/schedule/ScheduleComponentTest.java +++ b/rocketmq-streams-schedule/src/test/java/org/apache/rocketmq/stream/schedule/ScheduleComponentTest.java @@ -42,7 +42,7 @@ public void doExecute() throws InterruptedException { } @Override - public String getConfigureName() { + public String getName() { return "name"; } diff --git a/rocketmq-streams-schedule/src/test/resources/log4j.xml b/rocketmq-streams-schedule/src/test/resources/log4j.xml old mode 100755 new mode 100644 diff --git a/rocketmq-streams-script/pom.xml b/rocketmq-streams-script/pom.xml index e3838fdd..aad6d568 100755 --- a/rocketmq-streams-script/pom.xml +++ b/rocketmq-streams-script/pom.xml @@ -15,14 +15,14 @@ See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 org.apache.rocketmq rocketmq-streams - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT rocketmq-streams-script ROCKETMQ STREAMS :: script @@ -55,5 +55,9 @@ org.python jython-standalone + + com.aliyun.oss + aliyun-sdk-oss + diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/ScriptComponent.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/ScriptComponent.java index 87dbdc35..ede5ff3e 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/ScriptComponent.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/ScriptComponent.java @@ -17,8 +17,6 @@ package org.apache.rocketmq.streams.script; import java.util.Properties; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.component.AbstractComponent; import org.apache.rocketmq.streams.common.component.ComponentCreator; import org.apache.rocketmq.streams.common.component.IgnoreNameSpace; @@ -31,8 +29,6 @@ */ public class ScriptComponent extends AbstractComponent implements IgnoreNameSpace { - private static final Log LOG = LogFactory.getLog(ScriptComponent.class); - private static final ScriptComponent scriptComponent = ComponentCreator.getComponent(null, ScriptComponent.class); /** * 所有实现IScriptInit接口的对象和method @@ -47,8 +43,8 @@ public static ScriptComponent getInstance() { @Override public boolean startComponent(String namespace) { - functionService.scanePackage("org.apache.rocketmq.streams.script.function.impl"); - functionService.scanePackage("org.apache.rocketmq.streams.dim.function.script"); +// functionService.scanPackage("org.apache.rocketmq.streams.script.function.impl"); +// functionService.scanPackage("org.apache.rocketmq.streams.dim.function.script"); return true; } @@ -62,8 +58,7 @@ public IScriptService getService() { return scriptService; } - @Override - protected boolean initProperties(Properties properties) { + @Override protected boolean initProperties(Properties properties) { scriptService = new ScriptServiceImpl(); return true; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/context/FunctionContext.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/context/FunctionContext.java index 8790d1f1..90b96907 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/context/FunctionContext.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/context/FunctionContext.java @@ -28,6 +28,7 @@ */ public class FunctionContext extends AbstractContext { + protected Object returnValue; protected transient IFunctionService functionService = ScanFunctionService.getInstance(); @@ -41,7 +42,6 @@ public AbstractContext copy() { FunctionContext context = new FunctionContext(message); super.copyProperty(context); context.setFunctionService(this.functionService); - context.setConfigurableService(this.configurableService); return context; } @@ -66,6 +66,14 @@ public T executeFunction(String functionName, IMessage message, Object... pa } + public Object getReturnValue() { + return returnValue; + } + + public void setReturnValue(Object returnValue) { + this.returnValue = returnValue; + } + public IFunctionService getFunctionService() { return functionService; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/exception/ScriptExpresionRuntimeExpression.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/exception/ScriptExpresionRuntimeExpression.java new file mode 100644 index 00000000..b63d77e7 --- /dev/null +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/exception/ScriptExpresionRuntimeExpression.java @@ -0,0 +1,23 @@ +/* + * 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.script.exception; + +import org.apache.rocketmq.streams.script.operator.expression.ScriptExpression; + +public class ScriptExpresionRuntimeExpression extends RuntimeException { + protected ScriptExpression scriptExpression; +} diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/AverageAccumulator.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/AverageAccumulator.java index 924c9c19..1caec57f 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/AverageAccumulator.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/AverageAccumulator.java @@ -16,27 +16,20 @@ */ package org.apache.rocketmq.streams.script.function.aggregation; -import java.util.Iterator; import org.apache.rocketmq.streams.common.utils.CollectionUtil; import org.apache.rocketmq.streams.common.utils.NumberUtils; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.UDAFFunction; import org.apache.rocketmq.streams.script.function.aggregation.AverageAccumulator.AverageAccum; import org.apache.rocketmq.streams.script.service.IAccumulator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @Function @UDAFFunction("avg") public class AverageAccumulator implements IAccumulator { - public static class AverageAccum { - - public Number sum; - - public Number value; - - public int count; - - } + private static final Logger LOGGER = LoggerFactory.getLogger(AverageAccumulator.class); @Override public AverageAccum createAccumulator() { @@ -70,16 +63,14 @@ public void accumulate(AverageAccum accumulator, Object... parameters) { accumulator.count += 1; accumulator.value = NumberUtils.stripTrailingZeros(accumulator.sum.doubleValue() / accumulator.count); } catch (Exception e) { - System.out.println("The value is [" + parameters[0] + "]"); + LOGGER.error("The value is [{}}]", parameters[0]); throw e; } } @Override public void merge(AverageAccum accumulator, Iterable its) { - Iterator iterator = its.iterator(); - while (iterator.hasNext()) { - AverageAccum next = iterator.next(); + for (AverageAccum next : its) { if (next == null) { continue; } @@ -100,4 +91,14 @@ public void retract(AverageAccum accumulator, String... parameters) { //TODO } + public static class AverageAccum { + + public Number sum; + + public Number value; + + public int count; + + } + } \ No newline at end of file diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/ConcatAccumulator.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/ConcatAccumulator.java index 3586fbc1..2dc85d62 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/ConcatAccumulator.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/ConcatAccumulator.java @@ -32,13 +32,6 @@ public class ConcatAccumulator implements IAccumulator values = new ArrayList<>(); - } - @Override public ConcatAccum createAccumulator() { return new ConcatAccum(); @@ -63,10 +56,10 @@ public void accumulate(ConcatAccum accumulator, Object... parameters) { } try { if (USE_DEFAULT_SEPARATOR == parameters.length) { - accumulator.values.add((String)parameters[0]); + accumulator.values.add((String) parameters[0]); } else if (USE_DEFINED_SEPARATOR == parameters.length) { - accumulator.separator = (String)parameters[0]; - accumulator.values.add((String)parameters[1]); + accumulator.separator = (String) parameters[0]; + accumulator.values.add((String) parameters[1]); } } catch (Exception e) { throw e; @@ -92,4 +85,11 @@ public void retract(ConcatAccum accumulator, String... parameters) { //TODO } + public static class ConcatAccum { + + public String separator = ","; + + public List values = new ArrayList<>(); + } + } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/ConcatDistinctAccumulator.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/ConcatDistinctAccumulator.java index 183f78cf..e20cb6b6 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/ConcatDistinctAccumulator.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/ConcatDistinctAccumulator.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.aggregation; -import java.util.Collections; import java.util.HashSet; import java.util.Iterator; import java.util.Set; @@ -33,13 +32,6 @@ public class ConcatDistinctAccumulator implements IAccumulator values = Collections.synchronizedSet(new HashSet<>()); - } - @Override public ConcatDistinctAccum createAccumulator() { return new ConcatDistinctAccum(); @@ -65,10 +57,10 @@ public void accumulate(ConcatDistinctAccum accumulator, Object... parameters) { } try { if (USE_DEFAULT_SEPARATOR == parameters.length) { - accumulator.values.add((String)parameters[0]); + accumulator.values.add((String) parameters[0]); } else if (USE_DEFINED_SEPARATOR == parameters.length) { - accumulator.separator = (String)parameters[0]; - accumulator.values.add((String)parameters[1]); + accumulator.separator = (String) parameters[0]; + accumulator.values.add((String) parameters[1]); } } catch (Exception e) { throw e; @@ -94,4 +86,11 @@ public void retract(ConcatDistinctAccum accumulator, String... parameters) { //TODO } + public static class ConcatDistinctAccum { + + public String separator = ","; + + public Set values = new HashSet<>(); + } + } \ No newline at end of file diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/CountAccumulator.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/CountAccumulator.java index b4eae602..443dc2e9 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/CountAccumulator.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/CountAccumulator.java @@ -33,10 +33,6 @@ @UDAFFunction("count") public class CountAccumulator implements IAccumulator { - public static class CountAccum { - public int count = 0; - } - protected Integer count = 0; @Override @@ -93,4 +89,8 @@ public void retract(CountAccum accumulator, String... parameters) { //TODO } + public static class CountAccum { + public int count = 0; + } + } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/CountDistinctAccumulator.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/CountDistinctAccumulator.java index 1f2ee24d..f5f3dc92 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/CountDistinctAccumulator.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/CountDistinctAccumulator.java @@ -20,6 +20,7 @@ import com.google.common.hash.Hashing; import java.util.Iterator; import net.agkn.hll.HLL; +import org.apache.rocketmq.streams.common.interfaces.ISerialize; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.UDAFFunction; import org.apache.rocketmq.streams.script.service.IAccumulator; @@ -66,7 +67,7 @@ public class CountDistinctAccumulator implements IAccumulator { - public static class DistinctAccum { - public Set values = Collections.synchronizedSet(new HashSet<>()); - } - @Override public DistinctAccum createAccumulator() { return new DistinctAccum(); @@ -76,4 +71,8 @@ public void retract(DistinctAccum accumulator, String... parameters) { //TODO } + public static class DistinctAccum { + public Set values = new HashSet(); + } + } \ No newline at end of file diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/DistinctAccumulator2.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/DistinctAccumulator2.java index a4ceecd3..78bedb0f 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/DistinctAccumulator2.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/DistinctAccumulator2.java @@ -38,11 +38,6 @@ public class DistinctAccumulator2 implements IAccumulator its) { public void retract(DistinctAccum2 accumulator, String... parameters) { } + public static class DistinctAccum2 { + public String windowInstanceId; + public String groupByMd5; + } + } \ No newline at end of file diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/FirstValueAccumulator.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/FirstValueAccumulator.java index 0388ad3a..bee5d49e 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/FirstValueAccumulator.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/FirstValueAccumulator.java @@ -26,10 +26,6 @@ @UDAFFunction("FIRST_VALUE") public class FirstValueAccumulator implements IAccumulator { - public static class FirstValue { - public T value; - } - @Override public FirstValue createAccumulator() { return new FirstValue(); @@ -71,4 +67,8 @@ public void retract(FirstValue accumulator, String... parameters) { //TODO } + public static class FirstValue { + public T value; + } + } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/LastValueAccumulator.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/LastValueAccumulator.java index 987cc73d..53719708 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/LastValueAccumulator.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/LastValueAccumulator.java @@ -26,10 +26,6 @@ @UDAFFunction("LAST_VALUE") public class LastValueAccumulator implements IAccumulator { - public static class LastValue { - public T value; - } - @Override public LastValue createAccumulator() { return new LastValue(); @@ -64,4 +60,8 @@ public void retract(LastValue accumulator, String... parameters) { //TODO } + public static class LastValue { + public T value; + } + } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/MaxAccumulator.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/MaxAccumulator.java index ef418a9f..30576f84 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/MaxAccumulator.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/MaxAccumulator.java @@ -29,10 +29,6 @@ @UDAFFunction("max") public class MaxAccumulator implements IAccumulator { - public static class MaxAccum { - public String max; - } - private transient static SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); @Override @@ -52,7 +48,7 @@ public void accumulate(MaxAccum accumulator, Object... parameters) { } try { if (parameters[0] instanceof Number) { - Number input = (Number)parameters[0]; + Number input = (Number) parameters[0]; if (accumulator.max == null) { accumulator.max = input.toString(); return; @@ -61,7 +57,7 @@ public void accumulate(MaxAccum accumulator, Object... parameters) { accumulator.max = max.doubleValue() >= input.doubleValue() ? accumulator.max : input.toString(); } else if (parameters[0] instanceof Date) { try { - Date input = (Date)parameters[0]; + Date input = (Date) parameters[0]; if (accumulator.max == null) { accumulator.max = dateFormat.format(input); return; @@ -72,7 +68,7 @@ public void accumulate(MaxAccum accumulator, Object... parameters) { throw new RuntimeException(e); } } else if (parameters[0] instanceof String) { - String input = (String)parameters[0]; + String input = (String) parameters[0]; if (accumulator.max == null) { accumulator.max = input; return; @@ -107,4 +103,8 @@ public void retract(MaxAccum accumulator, String... parameters) { //TODO } + public static class MaxAccum { + public String max; + } + } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/MinAccumulator.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/MinAccumulator.java index e3279e33..ad2b9a35 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/MinAccumulator.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/MinAccumulator.java @@ -29,10 +29,6 @@ @UDAFFunction("min") public class MinAccumulator implements IAccumulator { - public static class MinAccum { - public String min; - } - private transient static SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); @Override @@ -52,7 +48,7 @@ public void accumulate(MinAccum accumulator, Object... parameters) { } try { if (parameters[0] instanceof Number) { - Number input = (Number)parameters[0]; + Number input = (Number) parameters[0]; if (accumulator.min == null) { accumulator.min = input.toString(); return; @@ -61,7 +57,7 @@ public void accumulate(MinAccum accumulator, Object... parameters) { accumulator.min = min.doubleValue() <= input.doubleValue() ? accumulator.min : input.toString(); } else if (parameters[0] instanceof Date) { try { - Date input = (Date)parameters[0]; + Date input = (Date) parameters[0]; if (accumulator.min == null) { accumulator.min = dateFormat.format(input); return; @@ -72,8 +68,8 @@ public void accumulate(MinAccum accumulator, Object... parameters) { throw new RuntimeException(e); } } else if (parameters[0] instanceof String) { - String input = (String)parameters[0]; - if("".equals(input)){ + String input = (String) parameters[0]; + if ("".equals(input)) { return; } if (accumulator.min == null) { @@ -110,4 +106,8 @@ public void retract(MinAccum accumulator, String... parameters) { //TODO } + public static class MinAccum { + public String min; + } + } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/SumAccumulator.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/SumAccumulator.java index c8d31155..4471bca0 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/SumAccumulator.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/SumAccumulator.java @@ -27,10 +27,6 @@ @UDAFFunction("sum") public class SumAccumulator implements IAccumulator { - public static class SumAccum { - public Number sum; - } - @Override public SumAccum createAccumulator() { SumAccum accum = new SumAccum(); @@ -50,10 +46,10 @@ public void accumulate(SumAccum accumulator, Object... parameters) { } try { if (parameters[0] instanceof Number) { - Number value = (Number)parameters[0]; + Number value = (Number) parameters[0]; accumulator.sum = NumberUtils.stripTrailingZeros(accumulator.sum.doubleValue() + value.doubleValue()); } else if (parameters[0] instanceof String) { - Number value = Double.valueOf((String)parameters[0]); + Number value = Double.valueOf((String) parameters[0]); accumulator.sum = NumberUtils.stripTrailingZeros(accumulator.sum.doubleValue() + value.doubleValue()); } else { throw new RuntimeException("type error!"); @@ -78,4 +74,8 @@ public void merge(SumAccum accumulator, Iterable its) { public void retract(SumAccum accumulator, String... parameters) { //TODO } + + public static class SumAccum { + public Number sum; + } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/bool/BooleanFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/bool/BooleanFunction.java index b039a412..d6bdebec 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/bool/BooleanFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/bool/BooleanFunction.java @@ -28,7 +28,7 @@ public class BooleanFunction { @FunctionMethod(value = "convertBoolen", alias = "convert_boolen", comment = "判断字段名或常量是否是Boolean类型,是返回1,否返回0") public int createInSQL(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName) { fieldName = FunctionUtils.getValueString(message, context, fieldName); if (Boolean.parseBoolean(fieldName)) { return 1; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/common/EchoFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/common/EchoFunction.java index 2fb8784b..8fffb9d8 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/common/EchoFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/common/EchoFunction.java @@ -17,23 +17,23 @@ package org.apache.rocketmq.streams.script.function.impl.common; import com.alibaba.fastjson.JSONObject; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; import org.apache.rocketmq.streams.script.annotation.FunctionParamter; import org.apache.rocketmq.streams.script.context.FunctionContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @Function public class EchoFunction { - private static final Log LOG = LogFactory.getLog(EchoFunction.class); + private static final Logger LOGGER = LoggerFactory.getLogger(EchoFunction.class); @FunctionMethod(value = "echo", comment = "输出一个JSON字符串") public String echo(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String... fieldNames) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String... fieldNames) { if (fieldNames == null) { - LOG.info("echo message:" + message.getMessageBody()); + LOGGER.info("echo message:" + message.getMessageBody()); return message.getMessageBody().toJSONString(); } JSONObject jsonObject = new JSONObject(); @@ -41,7 +41,7 @@ public String echo(IMessage message, FunctionContext context, for (String fieldName : fieldNames) { jsonObject.put(fieldName, jsonMessage.getString(fieldName)); } - LOG.info("echo message:" + jsonObject.toJSONString()); + LOGGER.info("echo message:" + jsonObject.toJSONString()); return jsonObject.toJSONString(); } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/common/EmptyFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/common/EmptyFunction.java new file mode 100644 index 00000000..c8f9b4b7 --- /dev/null +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/common/EmptyFunction.java @@ -0,0 +1,29 @@ +/* + * 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.script.function.impl.common; + +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.script.annotation.Function; +import org.apache.rocketmq.streams.script.annotation.FunctionMethod; +import org.apache.rocketmq.streams.script.context.FunctionContext; + +@Function +public class EmptyFunction { + @FunctionMethod(value = "empyt", comment = "空方法,为了解析占位使用") + public void empyt(IMessage message, FunctionContext context) { + } +} diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/EqualsFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/EqualsFunction.java index dc7e2b1e..b41e1c87 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/EqualsFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/EqualsFunction.java @@ -46,19 +46,22 @@ public static boolean isNotEquals(String functionName) { @FunctionMethod(value = "equal", alias = "==", comment = "判断两个值是否相等") public Boolean match(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名称或常量") String fieldName, - @FunctionParamter(value = "string", comment = "字段名称或常量") String value) { + @FunctionParamter(value = "string", comment = "字段名称或常量") String fieldName, + @FunctionParamter(value = "string", comment = "字段名称或常量") String value) { String leftValue = FunctionUtils.getValueString(message, context, fieldName); - if (leftValue == null && value == null || (NULL.equals(value.toLowerCase()))) { + String rightValue = FunctionUtils.getValueString(message, context, value); + + if (leftValue == null && rightValue == null) { return true; } + + if (leftValue == null || rightValue == null) { + return false; + } //FIXME 感觉这样解不合理,解析equals函数的时候应该将value置成空 - if ("".equals(leftValue) && ("".equals(value) || "''".equals(value))) { + if ("".equals(leftValue) && ("".equals(rightValue) || "''".equals(rightValue))) { return true; } - if (StringUtil.isEmpty(leftValue)) { - return false; - } if (FunctionUtils.isConstant(value)) { boolean result = leftValue.equals(FunctionUtils.getConstant(value)); return result; @@ -80,8 +83,8 @@ public Boolean match(IMessage message, FunctionContext context, @Deprecated @FunctionMethod(value = "equals", comment = "判断两个值是否相等") public boolean equals(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名称") String fieldName, - @FunctionParamter(value = "string", comment = "常量,不加引号也默认为常量") String value) { + @FunctionParamter(value = "string", comment = "字段名称") String fieldName, + @FunctionParamter(value = "string", comment = "常量,不加引号也默认为常量") String value) { return match(message, context, fieldName, value); } @@ -89,16 +92,16 @@ public boolean equals(IMessage message, FunctionContext context, @Deprecated @FunctionMethod(value = "!equals", comment = "判断两个值是否相等") public boolean notEquals(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名称") String fieldName, - @FunctionParamter(value = "string", comment = "常量,不加引号也默认为常量") String value) { + @FunctionParamter(value = "string", comment = "字段名称") String fieldName, + @FunctionParamter(value = "string", comment = "常量,不加引号也默认为常量") String value) { return !equals(message, context, fieldName, value); } @Deprecated @FunctionMethod(value = "equalsByField", comment = "判断两个值是否相等") public boolean equalsByField(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名称") String fieldName, - @FunctionParamter(value = "string", comment = "字段名称") String otherFieldName) { + @FunctionParamter(value = "string", comment = "字段名称") String fieldName, + @FunctionParamter(value = "string", comment = "字段名称") String otherFieldName) { String ori = ReflectUtil.getBeanFieldOrJsonValue(message.getMessageBody(), fieldName); String dest = ReflectUtil.getBeanFieldOrJsonValue(message.getMessageBody(), otherFieldName); if (StringUtil.isEmpty(ori) || StringUtil.isEmpty(dest)) { @@ -116,15 +119,15 @@ public boolean equalsByField(IMessage message, FunctionContext context, */ @FunctionMethod(value = "isFieldEmpty", comment = "判断一个字段是否为空") public boolean isFieldEmpty(IMessage channelMessage, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名称") String fieldName, String noUseValue) { + @FunctionParamter(value = "string", comment = "字段名称") String fieldName, String noUseValue) { Object value = FunctionUtils.getValue(channelMessage, context, fieldName); if (value == null) { return true; } - if (String.class.isInstance(value) && StringUtil.isEmpty((String)value)) { + if (String.class.isInstance(value) && StringUtil.isEmpty((String) value)) { return true; } - if (JSONArray.class.isInstance(value) && ((JSONArray)value).size() == 0) { + if (JSONArray.class.isInstance(value) && ((JSONArray) value).size() == 0) { return true; } return false; @@ -133,15 +136,15 @@ public boolean isFieldEmpty(IMessage channelMessage, FunctionContext context, @FunctionMethod(value = "isNull", alias = "null", comment = "判断值是否为NULL") public boolean isNull(IMessage channelMessage, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名称") String fieldName) { + @FunctionParamter(value = "string", comment = "字段名称") String fieldName) { Object value = FunctionUtils.getValue(channelMessage, context, fieldName); if (value == null) { return true; } - if (String.class.isInstance(value) && StringUtil.isEmpty((String)value)) { + if (String.class.isInstance(value) && StringUtil.isEmpty((String) value)) { return true; } - if (String.class.isInstance(value) && ((String)value).toLowerCase().equals("null")) { + if (String.class.isInstance(value) && ((String) value).toLowerCase().equals("null")) { return true; } return false; @@ -149,7 +152,7 @@ public boolean isNull(IMessage channelMessage, FunctionContext context, @FunctionMethod(value = "isNotNull", alias = "!null", comment = "判断值是否不为NULL") public boolean isNotNull(IMessage channelMessage, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名称") String fieldName) { + @FunctionParamter(value = "string", comment = "字段名称") String fieldName) { return !isNull(channelMessage, context, fieldName); } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/ExecuteByConditionFuction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/ExecuteByConditionFuction.java index 08f8e775..6ff12123 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/ExecuteByConditionFuction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/ExecuteByConditionFuction.java @@ -26,8 +26,8 @@ public class ExecuteByConditionFuction { @FunctionMethod("executeByCondition") public boolean executeByCondition(IMessage message, FunctionContext context, String fieldName, - String conditionFuction, String value, String executeFunction, String... parameters) { - Boolean match = (Boolean)context.executeFunction(conditionFuction, message, context, fieldName, value); + String conditionFuction, String value, String executeFunction, String... parameters) { + Boolean match = (Boolean) context.executeFunction(conditionFuction, message, context, fieldName, value); if (match) { context.executeFunction(executeFunction, message, context, parameters); } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/GreateEqualsFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/GreateEqualsFunction.java index 194f0c10..395a2676 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/GreateEqualsFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/GreateEqualsFunction.java @@ -30,8 +30,8 @@ public class GreateEqualsFunction { @FunctionMethod(value = "greateeq", alias = ">=", comment = ">=") public Boolean match(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String value) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName, + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String value) { String leftValue = FunctionUtils.getValueString(message, context, fieldName); if (FunctionUtils.isConstant(value)) { //support varchar and int transfer automatically @@ -60,8 +60,8 @@ public Boolean match(IMessage message, FunctionContext context, @Deprecated @FunctionMethod(value = "ge", comment = "建议用greateeq或>=") public boolean ge(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String value) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName, + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String value) { String fieldValue = ReflectUtil.getBeanFieldOrJsonValue(message.getMessageBody(), fieldName); return fieldValue.compareTo(value) >= 0; } @@ -69,8 +69,8 @@ public boolean ge(IMessage message, FunctionContext context, @Deprecated @FunctionMethod(value = "geByField", comment = "建议用greateeq或>=") public boolean geByField(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String otherFieldName) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName, + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String otherFieldName) { String ori = ReflectUtil.getBeanFieldOrJsonValue(message.getMessageBody(), fieldName); String dest = ReflectUtil.getBeanFieldOrJsonValue(message.getMessageBody(), otherFieldName); if (StringUtil.isEmpty(ori) || StringUtil.isEmpty(dest)) { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/GreaterFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/GreaterFunction.java index afa3e439..1f312560 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/GreaterFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/GreaterFunction.java @@ -29,9 +29,12 @@ public class GreaterFunction { @FunctionMethod(value = "great", alias = ">", comment = ">") public Boolean match(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String value) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName, + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String value) { String leftValue = FunctionUtils.getValueString(message, context, fieldName); + if (leftValue == null) { + return false; + } if (FunctionUtils.isConstant(value)) { return leftValue.equals(FunctionUtils.getConstant(value)); } else if (FunctionUtils.isLong(value)) { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/IFFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/IFFunction.java index ffb54185..c5b82703 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/IFFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/IFFunction.java @@ -13,7 +13,8 @@ * 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.script.function.impl.condition; + */ +package org.apache.rocketmq.streams.script.function.impl.condition; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/InConditionFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/InConditionFunction.java index d6f04261..cff3182d 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/InConditionFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/InConditionFunction.java @@ -40,9 +40,9 @@ public boolean notin(IMessage message, FunctionContext context, String fieldName @FunctionMethod(value = "in", alias = "contain", comment = "判断某个字符串是否在一个列表中") public boolean nin(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表待比较字符串的字段名或常量") String fieldName, - @FunctionParamter(value = "boolean", comment = "是否支持正则匹配") Boolean isRegexMatch, - @FunctionParamter(value = "array", comment = "代表字符串的字段名或常量列表") String... conditions) { + @FunctionParamter(value = "string", comment = "代表待比较字符串的字段名或常量") String fieldName, + @FunctionParamter(value = "boolean", comment = "是否支持正则匹配") Boolean isRegexMatch, + @FunctionParamter(value = "array", comment = "代表字符串的字段名或常量列表") String... conditions) { fieldName = FunctionUtils.getConstant(fieldName); String ori = FunctionUtils.getValueString(message, context, fieldName); @@ -66,8 +66,8 @@ public boolean nin(IMessage message, FunctionContext context, @FunctionMethod(value = "in", comment = "判断某个字符串是否在一个列表中") public boolean match(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名称,不加引号") String fieldName, - @FunctionParamter(value = "array", comment = "常量列表,不加引号") String... conditions) { + @FunctionParamter(value = "string", comment = "字段名称,不加引号") String fieldName, + @FunctionParamter(value = "array", comment = "常量列表,不加引号") String... conditions) { String ori = ReflectUtil.getBeanFieldOrJsonValue(message.getMessageBody(), fieldName); if (StringUtil.isEmpty(ori) || conditions == null) { return false; @@ -82,8 +82,8 @@ public boolean match(IMessage message, FunctionContext context, @FunctionMethod(value = "inByRegex", comment = "判断某个字符串是否在一个列表中") public boolean matchByRegex(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名称,不加引号") String fieldName, - @FunctionParamter(value = "array", comment = "正则列表,不加引号") String... conditions) { + @FunctionParamter(value = "string", comment = "字段名称,不加引号") String fieldName, + @FunctionParamter(value = "array", comment = "正则列表,不加引号") String... conditions) { String ori = ReflectUtil.getBeanFieldOrJsonValue(message.getMessageBody(), fieldName); if (StringUtil.isEmpty(ori) || conditions == null) { return false; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/LessEqualsFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/LessEqualsFunction.java index ba304227..ab9dbd92 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/LessEqualsFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/LessEqualsFunction.java @@ -29,8 +29,8 @@ public class LessEqualsFunction { @FunctionMethod(value = "lesseq", alias = "<=", comment = "<=") public Boolean match(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String value) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName, + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String value) { String leftValue = FunctionUtils.getValueString(message, context, fieldName); if (FunctionUtils.isConstant(value)) { return leftValue.equals(FunctionUtils.getConstant(value)); @@ -59,8 +59,8 @@ public boolean le(IMessage message, FunctionContext context, String fieldName, S @Deprecated @FunctionMethod(alias = "leByField", comment = "lesseq或<=") public boolean leByField(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String otherFieldName) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName, + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String otherFieldName) { String ori = ReflectUtil.getBeanFieldOrJsonValue(message.getMessageBody(), fieldName); String dest = ReflectUtil.getBeanFieldOrJsonValue(message.getMessageBody(), otherFieldName); if (StringUtil.isEmpty(ori) || StringUtil.isEmpty(dest)) { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/LessFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/LessFunction.java index c6530717..ec3de81b 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/LessFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/condition/LessFunction.java @@ -29,8 +29,8 @@ public class LessFunction { @FunctionMethod(value = "less", alias = "<", comment = "<") public Boolean match(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String value) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName, + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String value) { String leftValue = FunctionUtils.getValueString(message, context, fieldName); if (FunctionUtils.isConstant(value)) { return leftValue.equals(FunctionUtils.getConstant(value)); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/AddMonthFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/AddMonthFunction.java index 0fdb962c..57a6e7a8 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/AddMonthFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/AddMonthFunction.java @@ -41,9 +41,9 @@ public class AddMonthFunction { */ @FunctionMethod(value = "addmonth", comment = "增加指定日期的月数") public String addmonth(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表日期的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表日期格式的字段名或常量") String formatName, - @FunctionParamter(value = "string", comment = "待增加的月数,可以是数字,常量和代表月数的字段名") String nummonths) { + @FunctionParamter(value = "string", comment = "代表日期的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表日期格式的字段名或常量") String formatName, + @FunctionParamter(value = "string", comment = "待增加的月数,可以是数字,常量和代表月数的字段名") String nummonths) { String result = null; String format = FunctionUtils.getValueString(message, context, formatName); datetime = FunctionUtils.getValueString(message, context, datetime); @@ -65,8 +65,8 @@ public String addmonth(IMessage message, FunctionContext context, @FunctionMethod(value = "addmonth", comment = "增加指定日期的月数") public String addmonth(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表日期的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表日期格式的字段名或常量") String nummonths) { + @FunctionParamter(value = "string", comment = "代表日期的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表日期格式的字段名或常量") String nummonths) { return addmonth(message, context, datetime, "yyyy-MM-dd HH:mm:ss", nummonths); } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/CurrentTimestampFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/CurrentTimestampFunction.java index 82f6cc16..3766c25d 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/CurrentTimestampFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/CurrentTimestampFunction.java @@ -31,6 +31,12 @@ @Function public class CurrentTimestampFunction { + public static void main(String[] args) { + Timestamp timestamp = null; + timestamp = new Timestamp(System.currentTimeMillis()); + System.out.println(timestamp.getTime() / 1000); + } + /** * 获取当前时间戳 * @@ -56,7 +62,7 @@ public Long curstamp(IMessage message, FunctionContext context) { public Long curstampSecond(IMessage message, FunctionContext context) { Timestamp timestamp = null; timestamp = new Timestamp(System.currentTimeMillis()); - return timestamp.getTime()/1000; + return timestamp.getTime() / 1000; } /** @@ -67,9 +73,10 @@ public Long curstampSecond(IMessage message, FunctionContext context) { * @return */ @FunctionMethod(value = "curstamp_second", alias = "timestamp_second", comment = "生成时间戳") - public Long curstampSecond(IMessage message, FunctionContext context,String dateStr) { - return curstampSecond(message,context,dateStr,null); + public Long curstampSecond(IMessage message, FunctionContext context, String dateStr) { + return curstampSecond(message, context, dateStr, null); } + /** * 获取当前时间戳 * @@ -78,38 +85,35 @@ public Long curstampSecond(IMessage message, FunctionContext context,String date * @return */ @FunctionMethod(value = "curstamp_second", alias = "timestamp_second", comment = "生成时间戳") - public Long curstampSecond(IMessage message, FunctionContext context,String dateStr,String format) { - Long time= convert(message,context,dateStr,format); - if(time==null){ + public Long curstampSecond(IMessage message, FunctionContext context, String dateStr, String format) { + Long time = convert(message, context, dateStr, format); + if (time == null) { return null; } - return time/1000; - } - public static void main(String[] args) { - Timestamp timestamp = null; - timestamp = new Timestamp(System.currentTimeMillis()); - System.out.println( timestamp.getTime()/1000); + return time / 1000; } + @FunctionMethod(value = "curstamp", alias = "timestamp", comment = "生成指定时间的时间戳") public Long convert(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "标准时间格式的时间") String dateTime){ - return convert(message,context,dateTime,null); + @FunctionParamter(value = "string", comment = "标准时间格式的时间") String dateTime) { + return convert(message, context, dateTime, null); } + @FunctionMethod(value = "curstamp", alias = "timestamp", comment = "生成指定时间的时间戳") public Long convert(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "标准时间格式的时间") String dateTime,String format) { + @FunctionParamter(value = "string", comment = "标准时间格式的时间") String dateTime, String format) { String dateTimeStr = FunctionUtils.getValueString(message, context, dateTime); - SimpleDateFormat dateFormat =null; - if(format==null){ - dateFormat= new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); - }else { - dateFormat= new SimpleDateFormat(FunctionUtils.getValueString(message,context,format)); + SimpleDateFormat dateFormat = null; + if (format == null) { + dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + } else { + dateFormat = new SimpleDateFormat(FunctionUtils.getValueString(message, context, format)); } if (dateTime == null) { return null; } - if(StringUtil.isEmpty(dateTimeStr)){ + if (StringUtil.isEmpty(dateTimeStr)) { return null; } try { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateAddFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateAddFunction.java index f4f366ef..6d720fb7 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateAddFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateAddFunction.java @@ -43,10 +43,10 @@ public class DateAddFunction { @FunctionMethod(value = "timeadd", alias = "timeadd", comment = "对指定的时间进行年月日时分秒单位的增加,负数代表减少") public String timestampadd(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表标准格式日期的字段名或常量") String dateTime, - @FunctionParamter(value = "string", comment = "代表要增加时间单位的字段名或常量,包括YEAR,MONTH,DAY,HOUR,MINUTE,WEEK,SECOND," - + "FRAC_SECOND") String datepart, - @FunctionParamter(value = "string", comment = "增加的数值,可以是数字,常量和代表数值的字段名,支持负值") String delta) { + @FunctionParamter(value = "string", comment = "代表标准格式日期的字段名或常量") String dateTime, + @FunctionParamter(value = "string", comment = "代表要增加时间单位的字段名或常量,包括YEAR,MONTH,DAY,HOUR,MINUTE,WEEK,SECOND," + + "FRAC_SECOND") String datepart, + @FunctionParamter(value = "string", comment = "增加的数值,可以是数字,常量和代表数值的字段名,支持负值") String delta) { String dateTimeStr = FunctionUtils.getValueString(message, context, dateTime); String deltaStr = FunctionUtils.getValueString(message, context, delta); String datepartStr = FunctionUtils.getValueString(message, context, datepart); @@ -98,10 +98,10 @@ public String timestampadd(IMessage message, FunctionContext context, @FunctionMethod(value = "dateadd", alias = "adddate", comment = "对指定的时间进行年月日时分秒单位的增加,负数代表减少") public String dateadd(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表标准格式日期的字段名或常量") String dateTime, - @FunctionParamter(value = "string", comment = "代表要增加时间单位的字段名或常量,包括YEAR,MONTH,DAY,HOUR,MINUTE,WEEK,SECOND," - + "FRAC_SECOND") String datepart, - @FunctionParamter(value = "string", comment = "增加的数值,可以是数字,常量和代表数值的字段名,支持负值") String delta) { + @FunctionParamter(value = "string", comment = "代表标准格式日期的字段名或常量") String dateTime, + @FunctionParamter(value = "string", comment = "代表要增加时间单位的字段名或常量,包括YEAR,MONTH,DAY,HOUR,MINUTE,WEEK,SECOND," + + "FRAC_SECOND") String datepart, + @FunctionParamter(value = "string", comment = "增加的数值,可以是数字,常量和代表数值的字段名,支持负值") String delta) { Timestamp timestamp = null; String dateTimeStr = FunctionUtils.getValueString(message, context, dateTime); String deltaStr = FunctionUtils.getValueString(message, context, delta); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateDiffFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateDiffFunction.java index 4813ce22..d8973060 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateDiffFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateDiffFunction.java @@ -42,8 +42,8 @@ public class DateDiffFunction { */ @FunctionMethod(value = "dateDiff", comment = "计算从enddate到startdate两个时间的天数差值") public Long dateDiff(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String startdate, - @FunctionParamter(value = "String", comment = "代表时间格式化格式") String enddate) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String startdate, + @FunctionParamter(value = "String", comment = "代表时间格式化格式") String enddate) { startdate = FunctionUtils.getValueString(message, context, startdate); enddate = FunctionUtils.getValueString(message, context, enddate); Date end = DateUtil.parse(enddate, DateUtil.DEFAULT_FORMAT); @@ -65,8 +65,8 @@ public Long dateDiff(IMessage message, FunctionContext context, */ @FunctionMethod(value = "dateDiff", comment = "计算从enddate到startdate两个时间的天数差值") public Long dateDiff(IMessage message, FunctionContext context, - @FunctionParamter(value = "long", comment = "代表时间的字段名或常量") long startdate, - @FunctionParamter(value = "long", comment = "代表时间格式化格式") long enddate) { + @FunctionParamter(value = "long", comment = "代表时间的字段名或常量") long startdate, + @FunctionParamter(value = "long", comment = "代表时间格式化格式") long enddate) { Date end = new Date(enddate); Date start = new Date(startdate); @@ -87,8 +87,8 @@ public Long dateDiff(IMessage message, FunctionContext context, */ @FunctionMethod(value = "dateDiff", comment = "计算从enddate到startdate两个时间的天数差值") public Long dateDiff(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String startdate, - @FunctionParamter(value = "long", comment = "代表时间格式化格式") long enddate) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String startdate, + @FunctionParamter(value = "long", comment = "代表时间格式化格式") long enddate) { Date end = new Date(enddate); Date start = DateUtil.parse(startdate, DateUtil.DEFAULT_FORMAT); @@ -109,8 +109,8 @@ public Long dateDiff(IMessage message, FunctionContext context, */ @FunctionMethod(value = "dateDiff", comment = "计算从enddate到startdate两个时间的天数差值") public Long dateDiff(IMessage message, FunctionContext context, - @FunctionParamter(value = "long", comment = "代表时间的字段名或常量") long startdate, - @FunctionParamter(value = "string", comment = "代表时间格式化格式") String enddate) { + @FunctionParamter(value = "long", comment = "代表时间的字段名或常量") long startdate, + @FunctionParamter(value = "string", comment = "代表时间格式化格式") String enddate) { Date start = new Date(startdate); Date end = DateUtil.parse(enddate, DateUtil.DEFAULT_FORMAT); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateFormatFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateFormatFunction.java index 1d3fd9dc..091f8a34 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateFormatFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateFormatFunction.java @@ -36,14 +36,14 @@ public class DateFormatFunction { */ @FunctionMethod(value = "format", comment = "把指定的标准时间转换成某种特定的格式") public String format(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String dateFieldName, - @FunctionParamter(value = "string", comment = "代表转换时间格式的字段名或常量") String destformat) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String dateFieldName, + @FunctionParamter(value = "string", comment = "代表转换时间格式的字段名或常量") String destformatOri) { String value = FunctionUtils.getValueString(message, context, dateFieldName); - destformat = FunctionUtils.getValueString(message, context, destformat); - if(FunctionUtils.isLong(value)){ + String destformat = FunctionUtils.getValueString(message, context, destformatOri); + if (FunctionUtils.isLong(value)) { try { SimpleDateFormat destDateFormat = new SimpleDateFormat(destformat); - Date date=new Date(Long.valueOf(value)); + Date date = new Date(Long.valueOf(value)); String dateStr = destDateFormat.format(date); return dateStr; } catch (Exception e) { @@ -51,8 +51,8 @@ public String format(IMessage message, FunctionContext context, throw new RuntimeException("format函数执行错误", e); } - }else { - return format(message,context,dateFieldName,"'yyyy-MM-dd HH:mm:ss'",destformat); + } else { + return format(message, context, dateFieldName, "'yyyy-MM-dd HH:mm:ss'", destformatOri); } } @@ -63,15 +63,16 @@ public String format(IMessage message, FunctionContext context, */ @FunctionMethod(value = "format", comment = "把指定的时间转换成某种特定的格式") public String format(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String dateFieldName, - @FunctionParamter(value = "string", comment = "代表现有时间格式的字段名或常量") String oriformat, - @FunctionParamter(value = "string", comment = "代表转换时间格式的字段名或常量") String destformat) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String dateFieldName, + @FunctionParamter(value = "string", comment = "代表现有时间格式的字段名或常量") String oriformat, + @FunctionParamter(value = "string", comment = "代表转换时间格式的字段名或常量") String destformat) { String value = FunctionUtils.getValueString(message, context, dateFieldName); oriformat = FunctionUtils.getValueString(message, context, oriformat); destformat = FunctionUtils.getValueString(message, context, destformat); SimpleDateFormat oriDateFormat = new SimpleDateFormat(oriformat); SimpleDateFormat destDateFormat = new SimpleDateFormat(destformat); try { + value=value.replace("T"," "); Date date = oriDateFormat.parse(value); String dateStr = destDateFormat.format(date); return dateStr; @@ -85,8 +86,8 @@ public String format(IMessage message, FunctionContext context, @Deprecated @FunctionMethod(value = "dateFormat", alias = "modifyFormat", comment = "修改标准时间字段为某种时间格式") public String dateFormat(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名,不支持常量") String dateFieldName, - @FunctionParamter(value = "string", comment = "默认为格式常量,不加引号") String format) { + @FunctionParamter(value = "string", comment = "代表时间的字段名,不支持常量") String dateFieldName, + @FunctionParamter(value = "string", comment = "默认为格式常量,不加引号") String format) { String value = FunctionUtils.getValueString(message, context, dateFieldName); if (value == null) { return ""; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DatePartFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DatePartFunction.java index 5d6d82e9..47f27d08 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DatePartFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DatePartFunction.java @@ -30,9 +30,9 @@ public class DatePartFunction { @FunctionMethod(value = "datepart", alias = "date", comment = "获取标准格式日期的年月日时分秒") public String datePart(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表年月日时分秒的字段名或常量,值为year,month,day,hour,minute,second") - String datepart) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表年月日时分秒的字段名或常量,值为year,month,day,hour,minute,second") + String datepart) { return datePart(message, context, datetime, ("'" + DateUtil.DEFAULT_FORMAT + "'"), datepart); } @@ -47,10 +47,10 @@ public String datePart(IMessage message, FunctionContext context, */ @FunctionMethod(value = "datepart", alias = "date", comment = "获取标准格式日期的年月日时分秒") public String datePart(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format, - @FunctionParamter(value = "string", comment = "代表年月日时分秒的字段名或常量,值为year,month,day,hour,minute,second") - String datepart) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format, + @FunctionParamter(value = "string", comment = "代表年月日时分秒的字段名或常量,值为year,month,day,hour,minute,second") + String datepart) { String result = null; if (datetime == null || datepart == null) { return result; @@ -82,8 +82,8 @@ public String datePart(IMessage message, FunctionContext context, @FunctionMethod(value = "day", alias = "dd", comment = "获取日期中的日") public Integer day(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { + @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { String value = datePart(message, context, datetime, format, "'day'"); if (value == null) { return null; @@ -93,7 +93,7 @@ public Integer day(IMessage message, FunctionContext context, @FunctionMethod(value = "day", alias = "dd", comment = "获取日期中的日") public Integer day(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime) { + @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime) { String value = datePart(message, context, datetime, "'day'"); if (value == null) { return null; @@ -103,15 +103,15 @@ public Integer day(IMessage message, FunctionContext context, @FunctionMethod(value = "day", alias = "dd", comment = "获取日期中的日") public Integer day(IMessage message, FunctionContext context, - @FunctionParamter(value = "long", comment = "代表标准时间格式的字段名或常量") Long datetime) { + @FunctionParamter(value = "long", comment = "代表标准时间格式的字段名或常量") Long datetime) { Date date = new Date(datetime); return DateUtil.getDay(date); } @FunctionMethod(value = "hour", alias = "hh", comment = "获取日期中的小时") public Integer hour(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { + @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { String value = datePart(message, context, datetime, format, "'hour'"); if (value == null) { return null; @@ -121,7 +121,7 @@ public Integer hour(IMessage message, FunctionContext context, @FunctionMethod(value = "hour", alias = "hh", comment = "获取日期中的小时") public Integer hour(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime) { + @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime) { String value = datePart(message, context, datetime, "'hour'"); if (value == null) { return null; @@ -131,8 +131,8 @@ public Integer hour(IMessage message, FunctionContext context, @FunctionMethod(value = "year", alias = "yyyy", comment = "获取日期中的年") public Integer year(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { + @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { String value = datePart(message, context, datetime, format, "'year'"); if (value == null) { return null; @@ -142,7 +142,7 @@ public Integer year(IMessage message, FunctionContext context, @FunctionMethod(value = "year", alias = "yyyy", comment = "获取日期中的年") public Integer year(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime) { + @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime) { String value = datePart(message, context, datetime, "'year'"); if (value == null) { return null; @@ -152,8 +152,8 @@ public Integer year(IMessage message, FunctionContext context, @FunctionMethod(value = "month", comment = "获取日期中的月") public Integer month(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { + @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { String value = datePart(message, context, datetime, format, "'month'"); if (value == null) { return null; @@ -163,7 +163,7 @@ public Integer month(IMessage message, FunctionContext context, @FunctionMethod(value = "month", comment = "获取日期中的月") public Integer month(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime) { + @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime) { String value = datePart(message, context, datetime, "'month'"); if (value == null) { return null; @@ -173,8 +173,8 @@ public Integer month(IMessage message, FunctionContext context, @FunctionMethod(value = "minute", comment = "获取日期中的分钟") public Integer minute(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { + @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { String value = datePart(message, context, datetime, format, "'minute'"); if (value == null) { return null; @@ -184,7 +184,7 @@ public Integer minute(IMessage message, FunctionContext context, @FunctionMethod(value = "minute", comment = "获取日期中的分钟") public Integer minute(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime) { + @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime) { String value = datePart(message, context, datetime, "'minute'"); if (value == null) { return null; @@ -194,8 +194,8 @@ public Integer minute(IMessage message, FunctionContext context, @FunctionMethod(value = "second", alias = "ss", comment = "获取日期中的秒") public Integer second(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { + @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { String value = datePart(message, context, datetime, format, "'second'"); if (value == null) { return null; @@ -205,7 +205,7 @@ public Integer second(IMessage message, FunctionContext context, @FunctionMethod(value = "second", alias = "ss", comment = "获取日期中的秒") public Integer second(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime) { + @FunctionParamter(value = "string", comment = "代表标准时间格式的字段名或常量") String datetime) { String value = datePart(message, context, datetime, "'second'"); if (value == null) { return null; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateToCharFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateToCharFunction.java index d78fbb58..21f42980 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateToCharFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateToCharFunction.java @@ -32,13 +32,13 @@ public class DateToCharFunction { @FunctionMethod(value = "tochar", alias = "timestamp2string", comment = "把timestamp格式的数字转换成标准格式的字符串") public String toChar(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表timestamp格式的字段名或数字") String datetime) { + @FunctionParamter(value = "string", comment = "代表timestamp格式的字段名或数字") String datetime) { return toChar(message, context, datetime, "yyyy-MM-dd HH:mm:ss"); } @FunctionMethod(value = "linuxtochar", alias = "linuxtime2string", comment = "把linuxtime格式的数字转换成标准格式的字符串") public String linuxToChar(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表timestamp格式的字段名或数字") String datetime) { + @FunctionParamter(value = "string", comment = "代表timestamp格式的字段名或数字") String datetime) { String result = null; String timeString = FunctionUtils.getValueString(message, context, datetime); if (timeString == null || "".equals(timeString)) { @@ -70,8 +70,8 @@ public String linuxToChar(IMessage message, FunctionContext context, */ @FunctionMethod(value = "tochar", alias = "timestamp2string", comment = "把timestamp格式的数字转换成指定格式的字符串") public String toChar(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表timestamp格式的字段名或数字") String datetime, - @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { + @FunctionParamter(value = "string", comment = "代表timestamp格式的字段名或数字") String datetime, + @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { String result = null; String timeString = FunctionUtils.getValueString(message, context, datetime); Long time = FunctionUtils.getLong(timeString); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateTruncFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateTruncFunction.java index aa922e4e..a8bc6f5a 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateTruncFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateTruncFunction.java @@ -35,11 +35,11 @@ public class DateTruncFunction { @FunctionMethod(value = "for_time", alias = "timeInterval", comment = "根据时间段来进行数据拆分") public Integer splitByTimeInterval(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名称或常量") String from, - @FunctionParamter(value = "string", comment = "代表时间的字段名称或常量") String until, - @FunctionParamter(value = "string", comment = "代表要增加时间单位的字段名或常量,包括YEAR,MONTH,DAY,HOUR,MINUTE,WEEK,SECOND," - + "FRAC_SECOND") String datepart, - @FunctionParamter(value = "string", comment = "代表年月日的字段名或常量,支持year,day,month") String delta) { + @FunctionParamter(value = "string", comment = "代表时间的字段名称或常量") String from, + @FunctionParamter(value = "string", comment = "代表时间的字段名称或常量") String until, + @FunctionParamter(value = "string", comment = "代表要增加时间单位的字段名或常量,包括YEAR,MONTH,DAY,HOUR,MINUTE,WEEK,SECOND," + + "FRAC_SECOND") String datepart, + @FunctionParamter(value = "string", comment = "代表年月日的字段名或常量,支持year,day,month") String delta) { String fromValue = FunctionUtils.getValueString(message, context, from); String untilValue = FunctionUtils.getValueString(message, context, until); String datepartValue = FunctionUtils.getValueString(message, context, datepart); @@ -51,7 +51,7 @@ public Integer splitByTimeInterval(IMessage message, FunctionContext context, while (fromValue.compareTo(untilValue) < 0) { msg.put(from, fromValue); messages.add(new Message(msg)); - fromValue = (String)context.executeFunction("adddate", message, context, "'" + fromValue + "'", + fromValue = (String) context.executeFunction("adddate", message, context, "'" + fromValue + "'", "'" + datepartValue + "'", "'" + deltaValue + "'"); msg = new JSONObject(); msg.putAll(message.getMessageBody()); @@ -64,8 +64,8 @@ public Integer splitByTimeInterval(IMessage message, FunctionContext context, @FunctionMethod(value = "datetrunc", alias = "datefirst", comment = "获取标准时间格式的年月日的起始时间") public String dateTrunc(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名称或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表年月日的字段名或常量,支持year,day,month") String datepart) { + @FunctionParamter(value = "string", comment = "代表时间的字段名称或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表年月日的字段名或常量,支持year,day,month") String datepart) { return dateTrunc(message, context, datetime, datepart, "'" + DateUtil.DEFAULT_FORMAT + "'"); } @@ -80,9 +80,9 @@ public String dateTrunc(IMessage message, FunctionContext context, */ @FunctionMethod(value = "datetrunc", alias = "datefirst", comment = "获取标准时间格式的年月日的起始时间") public String dateTrunc(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名称或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表年月日的字段名或常量,支持year,day,month") String datepart, - @FunctionParamter(value = "string", comment = "代表时间格式的字段名或常量") String format) { + @FunctionParamter(value = "string", comment = "代表时间的字段名称或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表年月日的字段名或常量,支持year,day,month") String datepart, + @FunctionParamter(value = "string", comment = "代表时间格式的字段名或常量") String format) { if (datetime == null || datepart == null) { return null; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateUtilFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateUtilFunction.java index bd922d69..5eee0d58 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateUtilFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateUtilFunction.java @@ -30,14 +30,14 @@ public class DateUtilFunction { @FunctionMethod(value = "isValidTime", alias = "isDate", comment = "当前字符串是否是日期") public Boolean isValidateDate(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime) { return isValidateDate(message, context, datetime, "'" + DateUtil.DEFAULT_FORMAT + "'"); } @FunctionMethod(value = "isValidTime", alias = "isDate", comment = "当前字符串是否是日期") public Boolean isValidateDate(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { datetime = FunctionUtils.getValueString(message, context, datetime); format = FunctionUtils.getValueString(message, context, format); return DateUtil.isValidTime(datetime, format); @@ -45,16 +45,16 @@ public Boolean isValidateDate(IMessage message, FunctionContext context, @FunctionMethod(value = "addDay", alias = "dayAdd", comment = "给当前时间增加n天") public String addDay(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表待增加时间的字段名,数字或常量") String months) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表待增加时间的字段名,数字或常量") String months) { return addHour(message, context, datetime, "'" + DateUtil.DEFAULT_FORMAT + "'", months); } @FunctionMethod(value = "addDay", alias = "dayAdd", comment = "给当前时间增加n天") public String addDay(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format, - @FunctionParamter(value = "string", comment = "代表待增加时间的字段名,数字或常量") String months) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format, + @FunctionParamter(value = "string", comment = "代表待增加时间的字段名,数字或常量") String months) { datetime = FunctionUtils.getValueString(message, context, datetime); format = FunctionUtils.getValueString(message, context, format); months = FunctionUtils.getValueString(message, context, months); @@ -66,16 +66,16 @@ public String addDay(IMessage message, FunctionContext context, @FunctionMethod(value = "addMonth", alias = "monthAdd", comment = "给当前时间增加n月") public String addMonth(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表待增加时间的字段名,数字或常量") String months) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表待增加时间的字段名,数字或常量") String months) { return addHour(message, context, datetime, "'" + DateUtil.DEFAULT_FORMAT + "'", months); } @FunctionMethod(value = "addMonth", alias = "monthAdd", comment = "给当前时间增加n月") public String addMonth(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format, - @FunctionParamter(value = "string", comment = "代表待增加时间的字段名,数字或常量") String months) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format, + @FunctionParamter(value = "string", comment = "代表待增加时间的字段名,数字或常量") String months) { datetime = FunctionUtils.getValueString(message, context, datetime); format = FunctionUtils.getValueString(message, context, format); months = FunctionUtils.getValueString(message, context, months); @@ -87,16 +87,16 @@ public String addMonth(IMessage message, FunctionContext context, @FunctionMethod(value = "addYear", alias = "yearAdd", comment = "给当前时间增加n年") public String addYear(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表待增加时间的字段名,数字或常量") String years) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表待增加时间的字段名,数字或常量") String years) { return addHour(message, context, datetime, "'" + DateUtil.DEFAULT_FORMAT + "'", years); } @FunctionMethod(value = "addYear", alias = "yearAdd", comment = "给当前时间增加n年") public String addYear(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format, - @FunctionParamter(value = "string", comment = "代表待增加时间的字段名,数字或常量") String years) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format, + @FunctionParamter(value = "string", comment = "代表待增加时间的字段名,数字或常量") String years) { datetime = FunctionUtils.getValueString(message, context, datetime); format = FunctionUtils.getValueString(message, context, format); years = FunctionUtils.getValueString(message, context, years); @@ -108,16 +108,16 @@ public String addYear(IMessage message, FunctionContext context, @FunctionMethod(value = "addHour", alias = "hourAdd", comment = "给当前时间增加n个小时") public String addHour(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表待增加时间的字段名,数字或常量") String hours) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表待增加时间的字段名,数字或常量") String hours) { return addHour(message, context, datetime, "'" + DateUtil.DEFAULT_FORMAT + "'", hours); } @FunctionMethod(value = "addHour", alias = "hourAdd", comment = "给当前时间增加n个小时") public String addHour(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format, - @FunctionParamter(value = "string", comment = "代表待增加时间的字段名,数字或常量") String hours) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format, + @FunctionParamter(value = "string", comment = "代表待增加时间的字段名,数字或常量") String hours) { datetime = FunctionUtils.getValueString(message, context, datetime); format = FunctionUtils.getValueString(message, context, format); hours = FunctionUtils.getValueString(message, context, hours); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateUtileFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateUtileFunction.java index 9b4eb15f..385ce176 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateUtileFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DateUtileFunction.java @@ -26,7 +26,6 @@ import org.apache.rocketmq.streams.script.context.FunctionContext; import org.apache.rocketmq.streams.script.utils.FunctionUtils; - @Function public class DateUtileFunction { @@ -66,7 +65,6 @@ public String addDay(IMessage message, FunctionContext context, return DateUtil.format(date, format); } - @FunctionMethod(value = "addMonth", alias = "monthAdd", comment = "给当前时间增加n月") public String addMonth(IMessage message, FunctionContext context, @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, @@ -88,7 +86,6 @@ public String addMonth(IMessage message, FunctionContext context, return DateUtil.format(date, format); } - @FunctionMethod(value = "addYear", alias = "yearAdd", comment = "给当前时间增加n年") public String addYear(IMessage message, FunctionContext context, @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, @@ -110,7 +107,6 @@ public String addYear(IMessage message, FunctionContext context, return DateUtil.format(date, format); } - @FunctionMethod(value = "addHour", alias = "hourAdd", comment = "给当前时间增加n个小时") public String addHour(IMessage message, FunctionContext context, @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, @@ -132,20 +128,21 @@ public String addHour(IMessage message, FunctionContext context, return DateUtil.format(date, format); } - @FunctionMethod(value="to_date") + @FunctionMethod(value = "to_date") public Date create(IMessage message, FunctionContext context, @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { datetime = FunctionUtils.getValueString(message, context, datetime); - if(format==null){ + if (format == null) { return DateUtil.parse(datetime); } format = FunctionUtils.getValueString(message, context, format); - return DateUtil.parse(datetime,format); + return DateUtil.parse(datetime, format); } - @FunctionMethod(value="to_date") + + @FunctionMethod(value = "to_date") public Date create(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime){ - return create(message,context,datetime,null); + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime) { + return create(message, context, datetime, null); } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DayFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DayFunction.java index 6ee5af18..001619b9 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DayFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/DayFunction.java @@ -30,7 +30,7 @@ public class DayFunction { @FunctionMethod(value = "dayBegin", alias = "begin", comment = "获取指定日期的开始时间") public Long dayBegin(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表往前推几天,0表示当天的时间") String datepart) { + @FunctionParamter(value = "string", comment = "代表往前推几天,0表示当天的时间") String datepart) { String datepartStr = FunctionUtils.getValueString(message, context, datepart); Date date = new Date(); Calendar dateStart = Calendar.getInstance(); @@ -46,7 +46,7 @@ public Long dayBegin(IMessage message, FunctionContext context, @FunctionMethod(value = "dayEnd", alias = "end", comment = "获取指定日期的结束时间") public Long dayEnd(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表往前推几天,0表示当天的时间") String datepart) { + @FunctionParamter(value = "string", comment = "代表往前推几天,0表示当天的时间") String datepart) { String datepartStr = FunctionUtils.getValueString(message, context, datepart); Date date = new Date(); Calendar dateEnd = Calendar.getInstance(); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/FromUnixTimeFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/FromUnixTimeFunction.java index 3aea5dd4..67510aaa 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/FromUnixTimeFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/FromUnixTimeFunction.java @@ -29,6 +29,12 @@ @Function public class FromUnixTimeFunction { + public static void main(String[] args) { + Date date = new Date(1608624595 * 1000); + String value = DateUtil.format(date, DateUtil.DEFAULT_FORMAT); + System.out.println(value); + } + /** * @param message * @param context @@ -37,8 +43,8 @@ public class FromUnixTimeFunction { */ @FunctionMethod(value = "fromunixtime", comment = "把unixtime转换成自定义日期格式") public String formUnixTime(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表unixtime的字段名或常量") String unixtime, - @FunctionParamter(value = "string", comment = "代表日期格式的字段名或常量") String format) { + @FunctionParamter(value = "string", comment = "代表unixtime的字段名或常量") String unixtime, + @FunctionParamter(value = "string", comment = "代表日期格式的字段名或常量") String format) { Timestamp timestamp = null; if (unixtime == null) { return null; @@ -56,14 +62,8 @@ public String formUnixTime(IMessage message, FunctionContext context, @FunctionMethod(value = "fromunixtime", comment = "把unixtime转换成标准格式") public String formUnixTime(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表unixtime的字段名或常量") String unixtim) { + @FunctionParamter(value = "string", comment = "代表unixtime的字段名或常量") String unixtim) { return formUnixTime(message, context, unixtim, "'" + DateUtil.DEFAULT_FORMAT + "'"); } - - public static void main(String[] args) { - Date date = new Date(1608624595 * 1000); - String value = DateUtil.format(date, DateUtil.DEFAULT_FORMAT); - System.out.println(value); - } } \ No newline at end of file diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/IsDateFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/IsDateFunction.java index b530d71e..0542db9c 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/IsDateFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/IsDateFunction.java @@ -38,8 +38,8 @@ public class IsDateFunction { */ @FunctionMethod(value = "isdate", alias = "isValidTime", comment = "判断传递的数据是否是指定格式的时间字符串") public boolean isDate(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String date, - @FunctionParamter(value = "string", comment = "代表时间格式的字段名或常量") String format) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String date, + @FunctionParamter(value = "string", comment = "代表时间格式的字段名或常量") String format) { boolean flag = false; date = FunctionUtils.getValueString(message, context, date); format = FunctionUtils.getValueString(message, context, format); @@ -48,7 +48,7 @@ public boolean isDate(IMessage message, FunctionContext context, @FunctionMethod(value = "isdate", alias = "isValidTime", comment = "字符串是否是格式为’yyyy-MM-dd HH:mm:ss‘的时间字符串") public boolean isDate(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String date) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String date) { return isDate(message, context, date, "'" + DateUtil.DEFAULT_FORMAT + "'"); } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/LastDayFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/LastDayFunction.java index 27d3f52f..dee1fcbe 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/LastDayFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/LastDayFunction.java @@ -40,8 +40,8 @@ public class LastDayFunction { */ @FunctionMethod(value = "lastday", comment = "当前日期对应的月份中的最后一个天") public String lstDay(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表日期的字段名或常量") String date, - @FunctionParamter(value = "string", comment = "代表时间格式的字段名或常量") String format) { + @FunctionParamter(value = "string", comment = "代表日期的字段名或常量") String date, + @FunctionParamter(value = "string", comment = "代表时间格式的字段名或常量") String format) { Timestamp timestamp = null; date = FunctionUtils.getValueString(message, context, date); format = FunctionUtils.getValueString(message, context, format); @@ -59,7 +59,7 @@ public String lstDay(IMessage message, FunctionContext context, @FunctionMethod(value = "lastday", comment = "当前日期对应的月份中的最后一个天") public String lstDay(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表日期的字段名或常量") String date) { + @FunctionParamter(value = "string", comment = "代表日期的字段名或常量") String date) { return lstDay(message, context, date, "'" + DateUtil.DEFAULT_FORMAT + "'"); } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/QuarterFuction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/QuarterFuction.java index 8cbb27fc..68c0c7e8 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/QuarterFuction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/QuarterFuction.java @@ -35,7 +35,6 @@ public Integer quarter(IMessage message, FunctionContext context, return quarter(message, context, datetime, "'" + DateUtil.DEFAULT_FORMAT + "'"); } - /** * 获取当前日期所属季度 * diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/QuarterFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/QuarterFunction.java index c74bf91b..127d2e7c 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/QuarterFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/QuarterFunction.java @@ -30,7 +30,7 @@ public class QuarterFunction { @FunctionMethod(value = "quarter", comment = "获取当前的季度值,用数字表示") public Integer quarter(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime) { return quarter(message, context, datetime, "'" + DateUtil.DEFAULT_FORMAT + "'"); } @@ -44,8 +44,8 @@ public Integer quarter(IMessage message, FunctionContext context, */ @FunctionMethod(value = "quarter", comment = "获取当前的季度值,用数字表示") public Integer quarter(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { Integer quarter = null; datetime = FunctionUtils.getValueString(message, context, datetime); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/UnixTimeStampFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/UnixTimeStampFunction.java index 6b7dc36a..4cefd808 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/UnixTimeStampFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/UnixTimeStampFunction.java @@ -38,8 +38,8 @@ public class UnixTimeStampFunction { */ @FunctionMethod(value = "unixtimestamp", alias = "unixtime", comment = "获取unix时间") public Long unixStam(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { Long unixStap = null; datetime = FunctionUtils.getValueString(message, context, datetime); if (datetime == null) { @@ -66,7 +66,7 @@ public Long unixStam(IMessage message, FunctionContext context, @FunctionMethod(value = "unixtimestamp", alias = "unixtime", comment = "获取unix时间") public Long unixStam(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime) { //将"'"+DateUtil.DEFAULT_FORMAT+"'"两边的引号去除,在下面DateUtil.parse解析的时候会由于引号解析失败。 return unixStam(message, context, datetime, DateUtil.DEFAULT_FORMAT); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/WeekOfYearFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/WeekOfYearFunction.java index 980d92c8..9694d955 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/WeekOfYearFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/WeekOfYearFunction.java @@ -38,8 +38,8 @@ public class WeekOfYearFunction { */ @FunctionMethod(value = "weekofyear", alias = "week", comment = "获取日期dateTime是那一年的第几周") public Integer weekofyear(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { Integer day = null; datetime = FunctionUtils.getValueString(message, context, datetime); format = FunctionUtils.getValueString(message, context, format); @@ -52,7 +52,7 @@ public Integer weekofyear(IMessage message, FunctionContext context, @FunctionMethod(value = "weekofyear", alias = "week", comment = "获取当前日期dateTime是那一年的第几周") public Integer weekofyear(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime) { return weekofyear(message, context, datetime, "'" + DateUtil.DEFAULT_FORMAT + "'"); } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/WeekdayFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/WeekdayFunction.java index a89ed3c2..2f50eaaf 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/WeekdayFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/date/WeekdayFunction.java @@ -37,8 +37,8 @@ public class WeekdayFunction { */ @FunctionMethod(value = "weekday", alias = "dayofweek", comment = "获取星期几") public Integer weekday(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, - @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime, + @FunctionParamter(value = "string", comment = "代表格式的字段名或常量") String format) { Integer day = null; datetime = FunctionUtils.getValueString(message, context, datetime); format = FunctionUtils.getValueString(message, context, format); @@ -51,7 +51,7 @@ public Integer weekday(IMessage message, FunctionContext context, @FunctionMethod(value = "weekday", alias = "dayofweek", comment = "获取星期几") public Integer weekday(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime) { + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String datetime) { return weekday(message, context, datetime, "'" + DateUtil.DEFAULT_FORMAT + "'"); } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/env/ENVFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/env/ENVFunction.java deleted file mode 100644 index 01d8fd96..00000000 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/env/ENVFunction.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.script.function.impl.env; - -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.context.AbstractContext; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.utils.ENVUtile; -import org.apache.rocketmq.streams.script.annotation.Function; -import org.apache.rocketmq.streams.script.annotation.FunctionMethod; -import org.apache.rocketmq.streams.script.annotation.FunctionParamter; -import org.apache.rocketmq.streams.script.utils.FunctionUtils; - -@Function -public class ENVFunction { - public static final String ORIG_MESSAGE = "inner_message"; - - @FunctionMethod(value = "env", comment = "取得字段evnKey的值") - public String doENV(IMessage message, AbstractContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String envKey) { - envKey = FunctionUtils.getValueString(message, context, envKey); - message.getMessageBody().getString(envKey); - if (envKey.equals(ORIG_MESSAGE)) { - return message.getMessageBody().toJSONString(); - } - String value = ComponentCreator.getProperties().getProperty(envKey); - if (value != null) { - return value; - } - return ENVUtile.getENVParameter(value); - } - -} diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/eval/EvalFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/eval/EvalFunction.java index d9cd4178..ae98bf17 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/eval/EvalFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/eval/EvalFunction.java @@ -38,6 +38,26 @@ public static boolean isFunction(String name) { return "exec_function".equals(name) || "eval_function".equals(name); } + public static void main(String[] args) { + String value = "now()"; + + Object object = ScriptComponent.getInstance().getFunctionService().directExecuteFunction("eval_function", "now()"); + System.out.println(object); + } + + @FunctionMethod(value = "eval") + public Object eval(IMessage message, FunctionContext context, + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String script) { + String scriptValue = FunctionUtils.getValueString(message, context, script); + FunctionScript functionScript = functionScriptICache.get(scriptValue); + if (functionScript == null) { + functionScript = new FunctionScript(scriptValue); + functionScript.init(); + functionScriptICache.put(scriptValue, functionScript); + } + return functionScript.executeScriptAsFunction(message, context); + } + @FunctionMethod(value = "exec_function") public Object execFunction(IMessage message, FunctionContext context, @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String script) { @@ -79,11 +99,4 @@ public Object udf(IMessage message, FunctionContext context) { "execFunction", "你的脚本"); return object; } - - public static void main(String[] args) { - String value = "now()"; - - Object object = ScriptComponent.getInstance().getFunctionService().directExecuteFunction("eval_function", "now()"); - System.out.println(object); - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/AdditionalFiledFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/AdditionalFiledFunction.java index 633da3bb..da81d355 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/AdditionalFiledFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/AdditionalFiledFunction.java @@ -20,7 +20,7 @@ import java.util.Date; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.utils.DateUtil; -import org.apache.rocketmq.streams.common.utils.Ip2LongUtils; +import org.apache.rocketmq.streams.common.utils.Ip2LongUtil; import org.apache.rocketmq.streams.common.utils.RandomStrUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.script.annotation.Function; @@ -43,8 +43,8 @@ public class AdditionalFiledFunction { @FunctionMethod(value = "addRandomId", alias = "addRandom", comment = "增加一个随机数的新字段") public String addRandomId(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "新字段名称") String newFieldName, - @FunctionParamter(value = "string", comment = "字段长度,不需要单引号或双引号") String strLength) { + @FunctionParamter(value = "string", comment = "新字段名称") String newFieldName, + @FunctionParamter(value = "string", comment = "字段长度,不需要单引号或双引号") String strLength) { String name = FunctionUtils.getValueString(message, context, newFieldName); if (StringUtil.isEmpty(name)) { name = newFieldName; @@ -56,12 +56,12 @@ public String addRandomId(IMessage message, FunctionContext context, @FunctionMethod(value = "random", alias = "createRandom", comment = "产生一个随机数") public String random(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字段长度的字段名,数字或常量") String strLength) { + @FunctionParamter(value = "string", comment = "代表字段长度的字段名,数字或常量") String strLength) { Long length = 10L; if (!StringUtil.isEmpty(strLength) && !"null".equals(strLength)) { Object object = FunctionUtils.getValue(message, context, strLength); if (FunctionUtils.isNumberObject(object)) { - length = (Long)object; + length = (Long) object; } else { length = Long.valueOf(object.toString()); } @@ -72,8 +72,8 @@ public String random(IMessage message, FunctionContext context, @FunctionMethod(value = "copyField", alias = "copy", comment = "copy一个字段的值到一个新的字段(不建议使用,建议使用直接赋值方式)") public Object extra(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表新字段名的字段名") String newFieldName, - @FunctionParamter(value = "string", comment = "代表字段值的字段名或常量") String oldFieldName) { + @FunctionParamter(value = "string", comment = "代表新字段名的字段名") String newFieldName, + @FunctionParamter(value = "string", comment = "代表字段值的字段名或常量") String oldFieldName) { Object value = FunctionUtils.getValue(message, context, oldFieldName); if (value == null) { return null; @@ -88,7 +88,7 @@ public Object extra(IMessage message, FunctionContext context, @FunctionMethod(value = "copyField", alias = "copy", comment = "copy一个字段的值到一个新的字段") public Object extra(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字段值的字段名或常量") String oldFieldName) { + @FunctionParamter(value = "string", comment = "代表字段值的字段名或常量") String oldFieldName) { Object value = FunctionUtils.getValue(message, context, oldFieldName); return value; } @@ -104,8 +104,8 @@ public String addReceiveTime(IMessage message, FunctionContext context) { @FunctionMethod(alias = "addField", value = "newField", comment = "增加一个新字段,字段值为value") public String addField(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "新字段的名称,不需要引号") String newFieldName, - @FunctionParamter(value = "string", comment = "代表新字段值的字段名或常量") String value) { + @FunctionParamter(value = "string", comment = "新字段的名称,不需要引号") String newFieldName, + @FunctionParamter(value = "string", comment = "代表新字段值的字段名或常量") String value) { Object newFieldValue = FunctionUtils.getValue(message, context, value); String name = FunctionUtils.getValueString(message, context, newFieldName); if (StringUtil.isEmpty(name)) { @@ -118,8 +118,8 @@ public String addField(IMessage message, FunctionContext context, @Deprecated @FunctionMethod(value = "addConstant", alias = "constant", comment = "增加一个常量") public String addConstant(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "新字段的名次,不需要引号") String newFieldName, - @FunctionParamter(value = "string", comment = "新字段的值,默认为常量,不需要引号") String value) { + @FunctionParamter(value = "string", comment = "新字段的名次,不需要引号") String newFieldName, + @FunctionParamter(value = "string", comment = "新字段的值,默认为常量,不需要引号") String value) { String name = FunctionUtils.getValueString(message, context, newFieldName); if (StringUtil.isEmpty(name)) { name = newFieldName; @@ -148,7 +148,7 @@ public String addConstant(IMessage message, FunctionContext context, @FunctionMethod(value = "addByEqual", comment = "建议用if(compareValue==b)" + "then{newFieldName=equalValue}else{newFieldName=notEqualValue}方式") public String addByEqual(IMessage message, FunctionContext context, String newFieldName, String msgFielName, - String compareValue, String equalValue, String notEqualValue) { + String compareValue, String equalValue, String notEqualValue) { String value = null; String msgValue = FunctionUtils.getValueString(message, context, msgFielName); if (compareValue.equals(msgValue)) { @@ -199,8 +199,8 @@ public String addVarFromProperties(IMessage message, FunctionContext context, St */ @FunctionMethod(value = "addByIp2Long", comment = "把ip转换成long,并增加到新字段") public String addByIp2Long(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "新字段的名次,不需要引号") String newFieldName, - @FunctionParamter(value = "string", comment = "代表ip的字段名或常量") String msgFielName) { + @FunctionParamter(value = "string", comment = "新字段的名次,不需要引号") String newFieldName, + @FunctionParamter(value = "string", comment = "代表ip的字段名或常量") String msgFielName) { long ipLong = ip2Long(message, context, msgFielName); String name = FunctionUtils.getValueString(message, context, newFieldName); @@ -220,25 +220,25 @@ public String addByIp2Long(IMessage message, FunctionContext context, */ @FunctionMethod(value = "ip2Long", comment = "把ip转换成long") public Long ip2Long(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表ip的字段名或常量") String msgFielName) { + @FunctionParamter(value = "string", comment = "代表ip的字段名或常量") String msgFielName) { Object o = FunctionUtils.getValue(message, context, msgFielName); if (o == null) { return null; } - String ip = (String)o; - long ipLong = Ip2LongUtils.ipDotDec2Long(ip); + String ip = (String) o; + long ipLong = Ip2LongUtil.ipDotDec2Long(ip); return ipLong; } @FunctionMethod(value = "timeLong2String", comment = "把时间戳转换成标准格式,赋值给新字段") public String timeLong2String(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "新字段的名次,不需要引号") String newFieldName, - @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String msgFielName) { + @FunctionParamter(value = "string", comment = "新字段的名次,不需要引号") String newFieldName, + @FunctionParamter(value = "string", comment = "代表时间的字段名或常量") String msgFielName) { Object o = FunctionUtils.getValue(message, context, msgFielName); if (o == null) { return null; } - String time = (String)o; + String time = (String) o; if (MatchUtil.isNumber(time)) { time = DateUtil.longToString(Long.parseLong(time)); } @@ -258,7 +258,7 @@ public String timeLong2String(IMessage message, FunctionContext context, * @return */ private boolean matchOccureTime(Object occureTime, String cacheDateFormat, String msgOccureTime, - String msgDateFormate) { + String msgDateFormate) { Date openTime = createTime(occureTime, cacheDateFormat); Date logTime = createTime(msgOccureTime, msgDateFormate); long timeDiff = DateUtil.dateDiff(openTime, logTime); @@ -270,11 +270,11 @@ private boolean matchOccureTime(Object occureTime, String cacheDateFormat, Strin private Date createTime(Object time, String format) { if (String.class.isInstance(time)) { - return createTime((String)time, format); + return createTime((String) time, format); } else if (Date.class.isInstance(time)) { - return createTime((Date)time); + return createTime((Date) time); } else if (Long.class.isInstance(time)) { - return createTime((Long)time); + return createTime((Long) time); } return null; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/CoalesceFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/CoalesceFunction.java index 606af1c6..0920ebf0 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/CoalesceFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/CoalesceFunction.java @@ -28,16 +28,6 @@ @Function public class CoalesceFunction { - @FunctionMethod("coalesce") - public Object coalesce(IMessage message, FunctionContext context, String fieldName, String nullValue) { - Object o = message.getMessageBody().get(fieldName); - nullValue = FunctionUtils.getConstant(nullValue); - if (o == null) { - return nullValue; - } - return o; - } - public static void main(String[] args) { ScriptComponent scriptComponent = ScriptComponent.getInstance(); JSONObject msg = new JSONObject(); @@ -47,4 +37,14 @@ public static void main(String[] args) { System.out.println(message.getMessageBody()); } } + + @FunctionMethod("coalesce") + public Object coalesce(IMessage message, FunctionContext context, String fieldName, String nullValue) { + Object o = message.getMessageBody().get(fieldName); + nullValue = FunctionUtils.getConstant(nullValue); + if (o == null) { + return nullValue; + } + return o; + } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/ExtraFieldFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/ExtraFieldFunction.java index e9700f03..4e3ec690 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/ExtraFieldFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/ExtraFieldFunction.java @@ -29,8 +29,8 @@ public class ExtraFieldFunction { @FunctionMethod(value = "rename", alias = "rn", comment = "修改字段名称") public String extra(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表新字段的名称,不需要引号") String newFieldName, - @FunctionParamter(value = "string", comment = "代表旧字段的字段,不需要引号") String oldFieldName) { + @FunctionParamter(value = "string", comment = "代表新字段的名称,不需要引号") String newFieldName, + @FunctionParamter(value = "string", comment = "代表旧字段的字段,不需要引号") String oldFieldName) { Object value = FunctionUtils.getValue(message, context, oldFieldName); if (value == null) { return null; @@ -51,9 +51,9 @@ public String extra(IMessage message, FunctionContext context, * @param context * @param expandFieldName */ - @FunctionMethod(value = "expandField" ,alias = "expand_json", comment = "json字段展开到外层数据结") + @FunctionMethod(value = "expandField", alias = "expand_json", comment = "json字段展开到外层数据结") public void expandField(IMessage channelMessage, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String expandFieldName) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String expandFieldName) { if (StringUtil.isEmpty(expandFieldName)) { return; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/FieldFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/FieldFunction.java index 4ca7c645..db6d5bd0 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/FieldFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/FieldFunction.java @@ -19,6 +19,7 @@ import com.alibaba.fastjson.JSONObject; import com.google.common.collect.Lists; import java.util.List; +import java.util.Map; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.script.annotation.Function; @@ -40,6 +41,28 @@ public T getFieldValue(IMessage message, FunctionContext context, return (T) message.getMessageBody().get(name); } + @FunctionMethod(value = "map_field", alias = "get_map_field", comment = "获取字段值") + public Object getMapFieldValue(IMessage message, FunctionContext context, + @FunctionParamter(value = "string", comment = "字段的名称,不需要引号") String fieldName, String jsonField) { + Map fieldValue = (Map) message.getMessageBody().get(FunctionUtils.getConstant(fieldName)); + if (fieldValue == null) { + return null; + } + jsonField = FunctionUtils.getValueString(message, context, jsonField); + return fieldValue.get(jsonField); + } + + @FunctionMethod(value = "put_map_field", alias = "put_map_field", comment = "获取字段值") + public Object putMapFieldValue(IMessage message, FunctionContext context, + @FunctionParamter(value = "string", comment = "字段的名称,不需要引号") String mapFieldName, String fieldName, Object value) { + Map fieldValue = (Map) message.getMessageBody().get(mapFieldName); + if (fieldValue == null) { + return null; + } + fieldName = FunctionUtils.getValueString(message, context, fieldName); + return fieldValue.put(fieldName, value); + } + @FunctionMethod(value = "json_field", alias = "get_json_field", comment = "获取字段值") public Object getJsonFieldValue(IMessage message, FunctionContext context, @FunctionParamter(value = "string", comment = "字段的名称,不需要引号") String fieldName, String jsonField) { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/RemoveFieldFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/RemoveFieldFunction.java index c47c6647..62cfeeee 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/RemoveFieldFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/RemoveFieldFunction.java @@ -28,11 +28,11 @@ public class RemoveFieldFunction { public static boolean isFunction(String functionName) { - if(functionName==null){ + if (functionName == null) { return false; } - functionName=functionName.toLowerCase(); - if("rm".equals(functionName)||"removeField".equals(functionName)||"delete".equals(functionName)||"del".equals(functionName)){ + functionName = functionName.toLowerCase(); + if ("rm".equals(functionName) || "removeField".equals(functionName) || "delete".equals(functionName) || "del".equals(functionName)) { return true; } return false; @@ -40,15 +40,15 @@ public static boolean isFunction(String functionName) { @FunctionMethod(value = "rm", alias = "removeField", comment = "删除某个字段") public T remove(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名称,不需要引号") String fieldName) { + @FunctionParamter(value = "string", comment = "字段名称,不需要引号") String fieldName) { Object value = message.getMessageBody().get(fieldName); message.getMessageBody().remove(fieldName); - return (T)value; + return (T) value; } @FunctionMethod(value = "rm", alias = "removeField", comment = "删除某个字段") public T remove(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名称,不需要引号") String... fieldNames) { + @FunctionParamter(value = "string", comment = "字段名称,不需要引号") String... fieldNames) { if (fieldNames == null) { return null; } @@ -64,7 +64,7 @@ public T remove(IMessage message, FunctionContext context, @FunctionMethod(value = "delete", alias = "del", comment = "删除某个字段") public T delete(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名称,不需要引号") String... fieldNames) { + @FunctionParamter(value = "string", comment = "字段名称,不需要引号") String... fieldNames) { return remove(message, context, fieldNames); } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/RetainFieldFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/RetainFieldFunction.java index 177fc1b7..ab41bbf8 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/RetainFieldFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/RetainFieldFunction.java @@ -34,7 +34,7 @@ public class RetainFieldFunction { @FunctionMethod(value = "retainField", alias = "retain", comment = "需要保留的字段") public void retainField(IMessage message, FunctionContext context, - @FunctionParamter(value = "array", comment = "字段名,不需要引号") String... msgFieldNames) { + @FunctionParamter(value = "array", comment = "字段名,不需要引号") String... msgFieldNames) { Map map = new HashMap<>(); for (String field : msgFieldNames) { if (FunctionUtils.isConstant(field)) { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/filter/BreakFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/filter/BreakFunction.java index 66943102..cafdc342 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/filter/BreakFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/filter/BreakFunction.java @@ -24,7 +24,7 @@ @Function public class BreakFunction { - @FunctionMethod(value = "break", alias = "return", comment = "终止消息处理") + @FunctionMethod(value = "break", comment = "终止消息处理") public String breakExecute(IMessage message, FunctionContext context) { IMessage m = context.breakExecute(); return m.getMessageBody().toJSONString(); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/flatmap/SplitArrayFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/flatmap/SplitArrayFunction.java index 11a825e1..b18bb2a6 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/flatmap/SplitArrayFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/flatmap/SplitArrayFunction.java @@ -20,8 +20,6 @@ import com.alibaba.fastjson.JSONObject; import java.util.List; import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.utils.CollectionUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; @@ -31,12 +29,13 @@ import org.apache.rocketmq.streams.script.context.FunctionContext; import org.apache.rocketmq.streams.script.function.model.FunctionType; import org.apache.rocketmq.streams.script.utils.FunctionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @Function public class SplitArrayFunction { - private static final Log LOG = LogFactory.getLog(SplitArrayFunction.class); - public static final String SPLIT_LAST_FLAG = "__split_last_flag";//对于拆分的消息,会在最后一条加一个标识别。 public static final String FUNTION_NAME = "splitArray"; + private static final Logger LOGGER = LoggerFactory.getLogger(SplitArrayFunction.class); /** * 把一个json结构中包含的数组平铺起来 @@ -92,7 +91,7 @@ public void splitArray(IMessage channelMessage, FunctionContext context, copyMessage.setMessageBody(jsonObject); newMessage = copyMessage; } else { - LOG.warn("can not support split item , the value is " + value.getClass().getName()); + LOGGER.warn("can not support split item , the value is " + value.getClass().getName()); continue; } if (i < jsonArray.size() - 1) { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/mock/MockFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/flow/ReturnFunction.java similarity index 61% rename from rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/mock/MockFunction.java rename to rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/flow/ReturnFunction.java index e68d32ca..2a9cc478 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/mock/MockFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/flow/ReturnFunction.java @@ -14,26 +14,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.script.function.impl.mock; +package org.apache.rocketmq.streams.script.function.impl.flow; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; +import org.apache.rocketmq.streams.script.context.FunctionContext; import org.apache.rocketmq.streams.script.utils.FunctionUtils; @Function -public class MockFunction { +public class ReturnFunction { - @FunctionMethod("mock") - public void mock(IMessage message, AbstractContext context, String key, String value) { - key = FunctionUtils.getValueString(message, context, key); - value = FunctionUtils.getValueString(message, context, value); - if (StringUtil.isEmpty(key) || StringUtil.isEmpty(value)) { - return; + @FunctionMethod(value = "return", alias = "return", comment = "返回消息值") + public void breakExecute(IMessage message, FunctionContext context, String fieldName) { + Object value = FunctionUtils.getValue(message, context, fieldName); + if (value != null) { + context.setReturnValue(value); } - ComponentCreator.getProperties().put(key, value); } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonArrayFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonArrayFunction.java index 5eaed7c0..64d8f9aa 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonArrayFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonArrayFunction.java @@ -70,17 +70,17 @@ public String convertRow2Column(IMessage message, FunctionContext context, Strin @FunctionMethod(value = "in_max", alias = "max_from_array", comment = "获取jsonArray中的最大元素") public Object extraMax(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表json的字段名或常量") String jsonValueOrFieldName, - @FunctionParamter(value = "string", comment = "获取json的模式,支持name.name的方式,支持数组$.name[index].name的方式") - String path) { + @FunctionParamter(value = "string", comment = "代表json的字段名或常量") String jsonValueOrFieldName, + @FunctionParamter(value = "string", comment = "获取json的模式,支持name.name的方式,支持数组$.name[index].name的方式") + String path) { return extraValue(message, context, jsonValueOrFieldName, path, true); } @FunctionMethod(value = "remove_max", alias = "rm_max", comment = "获取jsonArray中的最大元素") public Object removeMax(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表json的字段名或常量") String jsonValueOrFieldName, - @FunctionParamter(value = "string", comment = "获取json的模式,支持name.name的方式,支持数组$.name[index].name的方式") String path, - @FunctionParamter(value = "boolean", comment = "去除最大值是否是排序") Boolean isOrder) { + @FunctionParamter(value = "string", comment = "代表json的字段名或常量") String jsonValueOrFieldName, + @FunctionParamter(value = "string", comment = "获取json的模式,支持name.name的方式,支持数组$.name[index].name的方式") String path, + @FunctionParamter(value = "boolean", comment = "去除最大值是否是排序") Boolean isOrder) { String maxValue = null; if (isOrder) { @@ -91,15 +91,15 @@ public Object removeMax(IMessage message, FunctionContext context, } String fieldName = FunctionUtils.getValueString(message, context, path); - List jsonArray = (List)message.getMessageBody().get(jsonValueOrFieldName); + List jsonArray = (List) message.getMessageBody().get(jsonValueOrFieldName); JSONArray result = new JSONArray(); for (int i = 0; i < jsonArray.size(); i++) { Object o = jsonArray.get(i); Map row = null; if (IMessage.class.isInstance(o)) { - row = ((IMessage)o).getMessageBody(); + row = ((IMessage) o).getMessageBody(); } else { - row = (Map)o; + row = (Map) o; } Object rowValue = row.get(fieldName); String fieldValue = null; @@ -119,20 +119,20 @@ public Object removeMax(IMessage message, FunctionContext context, @FunctionMethod(value = "in_min", alias = "min_from_array", comment = "获取jsonArray中的最小元素") public Object extraMin(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表json的字段名或常量") String jsonValueOrFieldName, - @FunctionParamter(value = "string", comment = "获取json的模式,支持name.name的方式,支持数组$.name[index].name的方式") - String path) { + @FunctionParamter(value = "string", comment = "代表json的字段名或常量") String jsonValueOrFieldName, + @FunctionParamter(value = "string", comment = "获取json的模式,支持name.name的方式,支持数组$.name[index].name的方式") + String path) { return extraValue(message, context, jsonValueOrFieldName, path, false); } public String extraValue(IMessage message, FunctionContext context, String jsonValueOrFieldName, String path, - boolean isMax) { + boolean isMax) { if (StringUtil.isEmpty(jsonValueOrFieldName) || StringUtil.isEmpty(path)) { return null; } String fieldName = FunctionUtils.getValueString(message, context, path); - List jsonArray = (List)message.getMessageBody().get(jsonValueOrFieldName); + List jsonArray = (List) message.getMessageBody().get(jsonValueOrFieldName); if (jsonArray == null) { return null; } @@ -141,9 +141,9 @@ public String extraValue(IMessage message, FunctionContext context, String jsonV Object o = jsonArray.get(i); Map row = null; if (IMessage.class.isInstance(o)) { - row = ((IMessage)o).getMessageBody(); + row = ((IMessage) o).getMessageBody(); } else { - row = (Map)o; + row = (Map) o; } Object rowValue = row.get(fieldName); String fieldValue = null; @@ -174,7 +174,7 @@ public String extraValueOrder(IMessage message, FunctionContext context, String return null; } String fieldName = FunctionUtils.getValueString(message, context, path); - List jsonArray = (List)message.getMessageBody().get(jsonValueOrFieldName); + List jsonArray = (List) message.getMessageBody().get(jsonValueOrFieldName); if (jsonArray == null) { return null; } @@ -197,9 +197,9 @@ private String getListElementValue(List jsonArray, int i, String fieldName) { Object o = jsonArray.get(i); Map row = null; if (IMessage.class.isInstance(o)) { - row = ((IMessage)o).getMessageBody(); + row = ((IMessage) o).getMessageBody(); } else { - row = (Map)o; + row = (Map) o; } Object rowValue = row.get(fieldName); String fieldValue = null; @@ -211,7 +211,7 @@ private String getListElementValue(List jsonArray, int i, String fieldName) { @FunctionMethod(value = "array_size", alias = "size_array", comment = "获取jsonArray的大小") public Integer arraySize(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表json的字段名或常量") String jsonValueOrFieldName) { + @FunctionParamter(value = "string", comment = "代表json的字段名或常量") String jsonValueOrFieldName) { if (StringUtil.isEmpty(jsonValueOrFieldName)) { return 0; } @@ -220,7 +220,7 @@ public Integer arraySize(IMessage message, FunctionContext context, if (message.getMessageBody().get(jsonValueOrFieldName) == null) { jsonValueOrFieldName = tmp; } - List value = (List)FunctionUtils.getValue(message, context, jsonValueOrFieldName); + List value = (List) FunctionUtils.getValue(message, context, jsonValueOrFieldName); if (value == null) { return null; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonCreatorFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonCreatorFunction.java index 800fc3ce..6d394707 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonCreatorFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonCreatorFunction.java @@ -17,8 +17,8 @@ package org.apache.rocketmq.streams.script.function.impl.json; import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson.JSONArray; import com.alibaba.fastjson.JSONObject; -import com.google.gson.JsonArray; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -31,8 +31,8 @@ public class JsonCreatorFunction { @FunctionMethod(value = "json_merge", alias = "jsonMerge", comment = "根据字段来组合json") public String extraJsonByField(IMessage message, FunctionContext context, - @FunctionParamter(value = "json", comment = "字段名列表") String jsonFieldName) { - jsonFieldName = FunctionUtils.getConstant( jsonFieldName); + @FunctionParamter(value = "json", comment = "字段名列表") String jsonFieldName) { + jsonFieldName = FunctionUtils.getConstant(jsonFieldName); JSONObject msg = message.getMessageBody().getJSONObject(jsonFieldName); message.getMessageBody().putAll(msg); return null; @@ -40,14 +40,14 @@ public String extraJsonByField(IMessage message, FunctionContext context, @FunctionMethod(value = "json", alias = "toJson", comment = "根据字段来组合json") public Object convertJson(IMessage message, FunctionContext context, - @FunctionParamter(value = "array", comment = "字段名列表") String fieldName) { + @FunctionParamter(value = "array", comment = "字段名列表") String fieldName) { Object value = FunctionUtils.getValue(message, context, fieldName); if (JSONObject.class.isInstance(value)) { - return (JSONObject)value; - } else if (JsonArray.class.isInstance(value)) { - return (JsonArray)value; + return (JSONObject) value; + } else if (JSONArray.class.isInstance(value)) { + return (JSONArray) value; } else { - String temp = (String)value; + String temp = (String) value; if (temp.startsWith("[") && temp.endsWith("]")) { return JSONObject.parseArray(temp); } else if (temp.startsWith("{") && temp.endsWith("}")) { @@ -61,7 +61,7 @@ public Object convertJson(IMessage message, FunctionContext context, @FunctionMethod(value = "json_generate_field", alias = "json_create_field", comment = "根据字段来组合json") public String extraJsonByField(IMessage message, FunctionContext context, - @FunctionParamter(value = "array", comment = "字段名列表") String... fieldNames) { + @FunctionParamter(value = "array", comment = "字段名列表") String... fieldNames) { if (fieldNames == null) { return null; } @@ -81,7 +81,7 @@ public String extraJsonByField(IMessage message, FunctionContext context, @FunctionMethod(value = "json_create", alias = "json_generate", comment = "根据两个字段定义一个," + "如name:yuanxiaodong,age:18对应的json为{name:yuanxiaodong,age:18}") public String extraJson(IMessage message, FunctionContext context, - @FunctionParamter(value = "array", comment = "格式如下,加引号代表常量否则代表字段名name:yuanxiaodong,age:18") String... kvs) { + @FunctionParamter(value = "array", comment = "格式如下,加引号代表常量否则代表字段名name:yuanxiaodong,age:18") String... kvs) { if (kvs == null) { return null; } @@ -111,8 +111,8 @@ public String extraJson(IMessage message, FunctionContext context, */ @FunctionMethod(value = "jsonAdd", alias = "json_add", comment = "给已有的json增加元素") public String addElement(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "现有json对应的字段名或常量") String jsonFiledName, - @FunctionParamter(value = "array", comment = "格式如下,加引号代表常量否则代表字段名name:yuanxiaodong,age:18") String... kvs) { + @FunctionParamter(value = "string", comment = "现有json对应的字段名或常量") String jsonFiledName, + @FunctionParamter(value = "array", comment = "格式如下,加引号代表常量否则代表字段名name:yuanxiaodong,age:18") String... kvs) { if (kvs == null) { return null; } @@ -146,8 +146,8 @@ public String addElement(IMessage message, FunctionContext context, */ @FunctionMethod(value = "jsonRemove", alias = "json_remove", comment = "移除现有json的元素") public String removeElement(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "现有json对应的字段名或常量") String jsonFiledName, - @FunctionParamter(value = "array", comment = "代表要移除key的字段名或常量列表") String... keyNames) { + @FunctionParamter(value = "string", comment = "现有json对应的字段名或常量") String jsonFiledName, + @FunctionParamter(value = "array", comment = "代表要移除key的字段名或常量列表") String... keyNames) { if (keyNames == null) { return null; } @@ -176,7 +176,7 @@ public String removeElement(IMessage message, FunctionContext context, */ @FunctionMethod(value = "jsonExpand", alias = "json_expand", comment = "展开一个json中的json") public void expandElement(IMessage message, FunctionContext context, - @FunctionParamter(value = "array", comment = "代表要移除key的字段名或常量列表") String jsonSubFieldName) { + @FunctionParamter(value = "array", comment = "代表要移除key的字段名或常量列表") String jsonSubFieldName) { jsonSubFieldName = FunctionUtils.getConstant(jsonSubFieldName); String jsonValue = message.getMessageBody().getString(jsonSubFieldName); if (jsonValue == null) { @@ -198,8 +198,8 @@ public void expandElement(IMessage message, FunctionContext context, */ @FunctionMethod(value = "jsonExpand", alias = "json_expand", comment = "展开一个json中的json") public String expandElement(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "现有json对应的字段名或常量") String jsonFiledName, - @FunctionParamter(value = "array", comment = "代表要移除key的字段名或常量列表") String jsonSubFieldName) { + @FunctionParamter(value = "string", comment = "现有json对应的字段名或常量") String jsonFiledName, + @FunctionParamter(value = "array", comment = "代表要移除key的字段名或常量列表") String jsonSubFieldName) { jsonFiledName = FunctionUtils.getValueString(message, context, jsonSubFieldName); jsonSubFieldName = FunctionUtils.getValueString(message, context, jsonSubFieldName); String jsonValue = message.getMessageBody().getString(jsonFiledName); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonValueFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonValueFunction.java index d6bc1baa..6f691291 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonValueFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonValueFunction.java @@ -13,7 +13,8 @@ * 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.script.function.impl.json; + */ +package org.apache.rocketmq.streams.script.function.impl.json; import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.JSONObject; @@ -33,9 +34,21 @@ @Function public class JsonValueFunction { - @FunctionMethod(value = "json",comment = "获取msg中的json数据") + public static void main(String[] args) { + JSONObject msg = new JSONObject(); + JSONObject metaConf = new JSONObject(); + metaConf.put("asset.type", "ECS"); + msg.put("_input", metaConf); + ScriptComponent scriptComponent = ScriptComponent.getInstance(); + List msgs = scriptComponent.getService().executeScript(msg, "a=EQUALS(json_field(_input, 'asset.type'),'ECS');"); + for (IMessage message : msgs) { + System.out.println(message.getMessageBody()); + } + } + + @FunctionMethod(value = "json", comment = "获取msg中的json数据") public JSONObject getJson(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表json的字段名或常量") String fieldName){ + @FunctionParamter(value = "string", comment = "代表json的字段名或常量") String fieldName) { fieldName = FunctionUtils.getValueString(message, context, fieldName); return message.getMessageBody().getJSONObject(fieldName); } @@ -44,14 +57,14 @@ public JSONObject getJson(IMessage message, FunctionContext context, public Object extra(IMessage message, FunctionContext context, @FunctionParamter(value = "string", comment = "代表json的字段名或常量") String jsonValueOrFieldName, @FunctionParamter(value = "string", comment = "获取json的模式,支持name.name的方式,支持数组$.name[index].name的方式") - String path) { + String path) { if (StringUtil.isEmpty(jsonValueOrFieldName) || StringUtil.isEmpty(path)) { return null; } String value = FunctionUtils.getValueString(message, context, jsonValueOrFieldName); String pattern = FunctionUtils.getValueString(message, context, path); - if(pattern==null){ - pattern=path; + if (pattern == null) { + pattern = path; } if (StringUtil.isEmpty(value) || StringUtil.isEmpty(pattern)) { return null; @@ -65,39 +78,26 @@ public Object extra(IMessage message, FunctionContext context, } else { bean = JSON.parseObject(value); } - if(bean==null){ + if (bean == null) { return null; } return ReflectUtil.getBeanFieldOrJsonValue(bean, pattern); } - - public static void main(String[] args) { - JSONObject msg=new JSONObject(); - JSONObject metaConf=new JSONObject(); - metaConf.put("asset.type","ECS"); - msg.put("_input",metaConf); - ScriptComponent scriptComponent=ScriptComponent.getInstance(); - List msgs=scriptComponent.getService().executeScript(msg,"a=EQUALS(json_field(_input, 'asset.type'),'ECS');"); - for(IMessage message:msgs){ - System.out.println(message.getMessageBody()); - } - } - @FunctionMethod(value = "for_field", alias = "forField", comment = "循环所有的msg字段") public Object extra(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表json的字段名或常量") String scriptValue){ - scriptValue=FunctionUtils.getValueString(message,context,scriptValue); + @FunctionParamter(value = "string", comment = "代表json的字段名或常量") String scriptValue) { + scriptValue = FunctionUtils.getValueString(message, context, scriptValue); Iterator> it = message.getMessageBody().entrySet().iterator(); - JSONObject msg=new JSONObject(); + JSONObject msg = new JSONObject(); msg.putAll(message.getMessageBody()); - while (it.hasNext()){ - Map.Entry entry=it.next(); - String key=entry.getKey(); - Object value=entry.getValue(); - msg.put("iterator.key",key); - msg.put("iterator.value",value); - ScriptComponent.getInstance().getService().executeScript(msg,scriptValue); + while (it.hasNext()) { + Map.Entry entry = it.next(); + String key = entry.getKey(); + Object value = entry.getValue(); + msg.put("iterator.key", key); + msg.put("iterator.value", value); + ScriptComponent.getInstance().getService().executeScript(msg, scriptValue); } msg.remove("iterator.key"); msg.remove("iterator.value"); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/UDTFFieldNameFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/UDTFFieldNameFunction.java index 621aefc0..30c8af38 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/UDTFFieldNameFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/UDTFFieldNameFunction.java @@ -16,11 +16,7 @@ */ package org.apache.rocketmq.streams.script.function.impl.json; - import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.utils.ReflectUtil; -import org.apache.rocketmq.streams.common.utils.StringUtil; -import org.apache.rocketmq.streams.script.ScriptComponent; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; import org.apache.rocketmq.streams.script.annotation.FunctionParamter; @@ -30,21 +26,19 @@ @Function public class UDTFFieldNameFunction { - @FunctionMethod(value = "addAliasForNewField",comment = "获取msg中的json数据") + @FunctionMethod(value = "addAliasForNewField", comment = "获取msg中的json数据") public Object addAliasForNewField(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表json的字段名或常量") String fieldName,String alias,int i){ - fieldName=FunctionUtils.getConstant(fieldName); - alias=FunctionUtils.getConstant(alias); - Object object=message.getMessageBody().get(fieldName); - if(message.getMessageBody().containsKey("f"+i)) { - object=message.getMessageBody().get("f"+i); - message.getMessageBody().put(alias+fieldName,object); - } - else if(message.getMessageBody().containsKey(fieldName)&&!message.getMessageBody().containsKey("f"+i)){ - message.getMessageBody().put(alias+fieldName,object); + @FunctionParamter(value = "string", comment = "代表json的字段名或常量") String fieldName, String alias, int i) { + fieldName = FunctionUtils.getConstant(fieldName); + alias = FunctionUtils.getConstant(alias); + Object object = message.getMessageBody().get(fieldName); + if (message.getMessageBody().containsKey("f" + i)) { + object = message.getMessageBody().get("f" + i); + message.getMessageBody().put(alias + fieldName, object); + } else if (message.getMessageBody().containsKey(fieldName) && !message.getMessageBody().containsKey("f" + i)) { + message.getMessageBody().put(alias + fieldName, object); } return object; } - } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AbsFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AbsFunction.java index ad46d3f3..182e2e87 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AbsFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AbsFunction.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -37,9 +36,9 @@ public class AbsFunction { */ @FunctionMethod(value = "abs", alias = "abs", comment = "求数值的绝对值") public Double abs(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = - "Double或bigint类型或Decimal类型,输入为bigint时返回bigint,输入为double时返回double类型。输入decimal类型时返回decimal类型") - String number) { + @FunctionParamter(value = "string", comment = + "Double或bigint类型或Decimal类型,输入为bigint时返回bigint,输入为double时返回double类型。输入decimal类型时返回decimal类型") + String number) { Double result = null; Double numberTem = Double.parseDouble(FunctionUtils.getValueString(message, context, number)); if (numberTem == null) { @@ -59,9 +58,9 @@ public Double abs(IMessage message, FunctionContext context, */ @FunctionMethod(value = "abs", alias = "abs", comment = "求数值的绝对值") public Double abs(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = - "Double或bigint类型或Decimal类型,输入为bigint时返回bigint,输入为double时返回double类型。输入decimal类型时返回decimal类型") - Double number) { + @FunctionParamter(value = "Double", comment = + "Double或bigint类型或Decimal类型,输入为bigint时返回bigint,输入为double时返回double类型。输入decimal类型时返回decimal类型") + Double number) { Double result = null; if (number == null) { return result; @@ -80,9 +79,9 @@ public Double abs(IMessage message, FunctionContext context, */ @FunctionMethod(value = "abs", alias = "abs", comment = "求数值的绝对值") public Integer abs(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = - "Double或bigint类型或Decimal类型,输入为bigint时返回bigint,输入为double时返回double类型。输入decimal类型时返回decimal类型") - Integer number) { + @FunctionParamter(value = "Integer", comment = + "Double或bigint类型或Decimal类型,输入为bigint时返回bigint,输入为double时返回double类型。输入decimal类型时返回decimal类型") + Integer number) { Integer result = null; if (number == null) { return result; @@ -91,25 +90,4 @@ public Integer abs(IMessage message, FunctionContext context, return result; } - /** - * 求数值的绝对值 - * - * @param message - * @param context - * @param number - * @return - */ - @FunctionMethod(value = "abs", alias = "abs", comment = "求数值的绝对值") - public BigDecimal abs(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = - "Double或bigint类型或Decimal类型,输入为bigint时返回bigint,输入为double时返回double类型。输入decimal类型时返回decimal类型") - BigDecimal number) { - BigDecimal result = null; - if (number == null) { - return result; - } - result = number.abs(); - return result; - } - } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AcosFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AcosFunction.java index 5df1706b..f0733666 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AcosFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AcosFunction.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -37,7 +36,7 @@ public class AcosFunction { */ @FunctionMethod(value = "acos", alias = "acos", comment = "求反余弦函数") public Double acos(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求反余弦的字段名或常量") String number) { + @FunctionParamter(value = "String", comment = "代表要求反余弦的字段名或常量") String number) { Double result = null; Double numberTem = Double.parseDouble(FunctionUtils.getValueString(message, context, number)); if (numberTem == null || numberTem < -1.0 || numberTem > 1.0) { @@ -57,7 +56,7 @@ public Double acos(IMessage message, FunctionContext context, */ @FunctionMethod(value = "acos", alias = "acos", comment = "求反余弦") public Double acos(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "Double常量") Double number) { + @FunctionParamter(value = "Double", comment = "Double常量") Double number) { Double result = null; if (number == null || number < -1 || number > 1) { return result; @@ -66,25 +65,6 @@ public Double acos(IMessage message, FunctionContext context, return result; } - /** - * 求反余弦函数 - * - * @param message - * @param context - * @param number - * @return - */ - @FunctionMethod(value = "acos", alias = "acos", comment = "求反余弦") - public BigDecimal acos(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "BigDecimal常量") BigDecimal number) { - BigDecimal result = null; - if (number == null || number.intValue() < -1 || number.intValue() > 1) { - return result; - } - result = new BigDecimal(Math.acos(number.intValue())); - return result; - } - /** * 求反余弦函数 * @@ -95,7 +75,7 @@ public BigDecimal acos(IMessage message, FunctionContext context, */ @FunctionMethod(value = "acos", alias = "acos", comment = "求反余弦") public Double acos(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "Integer常量") Integer number) { + @FunctionParamter(value = "Integer", comment = "Integer常量") Integer number) { Double result = null; if (number == null || number < -1 || number > 1) { return result; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AdditionFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AdditionFunction.java index 6854d1d0..d2f3c95e 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AdditionFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AdditionFunction.java @@ -39,22 +39,42 @@ public class AdditionFunction { */ @FunctionMethod(value = "addition", alias = "addition", comment = "两个数值的加法返回结果") public Object addition(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求乘法的第一个参数") String x, - @FunctionParamter(value = "String", comment = "代表要求乘法的第二个参数") String y) { + @FunctionParamter(value = "String", comment = "代表要求乘法的第一个参数") String x, + @FunctionParamter(value = "String", comment = "代表要求乘法的第二个参数") String y) { Double result = null; String paramX = FunctionUtils.getValueString(message, context, x); String paramY = FunctionUtils.getValueString(message, context, y); if (paramX == null || paramY == null) { return result; } - Double baseTem = Double.parseDouble(paramX); - Double xTem = Double.parseDouble(paramY); - - BigDecimal b1 = new BigDecimal(String.valueOf(baseTem)); - BigDecimal b2 = new BigDecimal(String.valueOf(xTem)); - BigDecimal bb = b1.add(b2); - Double bbb = bb.doubleValue(); - return bbb; + if (FunctionUtils.isLong(paramX) && FunctionUtils.isLong(paramY)) { + Long baseTem = Long.parseLong(paramX); + Long xTem = Long.parseLong(paramY); + + BigDecimal b1 = new BigDecimal(String.valueOf(baseTem)); + BigDecimal b2 = new BigDecimal(String.valueOf(xTem)); + BigDecimal bb = b1.add(b2); + long bbb = bb.longValue(); + return bbb; + } + + if (FunctionUtils.isDouble(paramX) && FunctionUtils.isDouble(paramY)) { + Double baseTem = Double.parseDouble(paramX); + Double xTem = Double.parseDouble(paramY); + + BigDecimal b1 = new BigDecimal(String.valueOf(baseTem)); + BigDecimal b2 = new BigDecimal(String.valueOf(xTem)); + BigDecimal bb = b1.add(b2); + Double bbb = bb.doubleValue(); + return bbb; + } + if (paramX == null) { + return paramY; + } + if (paramY == null) { + return paramX; + } + return paramX + paramY; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AsinFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AsinFunction.java index 224f17fc..9b2e5ff2 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AsinFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AsinFunction.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -37,7 +36,7 @@ public class AsinFunction { */ @FunctionMethod(value = "asin", alias = "asin", comment = "求反正弦函数") public Double asin(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求反正弦的字段名或常量") String number) { + @FunctionParamter(value = "String", comment = "代表要求反正弦的字段名或常量") String number) { Double result = null; Double numberTem = Double.parseDouble(FunctionUtils.getValueString(message, context, number)); if (numberTem == null || numberTem < -1.0 || numberTem > 1.0) { @@ -57,7 +56,7 @@ public Double asin(IMessage message, FunctionContext context, */ @FunctionMethod(value = "asin", alias = "asin", comment = "求反正弦函数") public Double asin(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "Double常量") Double number) { + @FunctionParamter(value = "Double", comment = "Double常量") Double number) { Double result = null; if (number == null || number < -1.0 || number > 1.0) { return result; @@ -66,25 +65,6 @@ public Double asin(IMessage message, FunctionContext context, return result; } - /** - * 求反正弦函数 - * - * @param message - * @param context - * @param number - * @return - */ - @FunctionMethod(value = "asin", alias = "asin", comment = "求反正弦函数") - public BigDecimal asin(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "BigDecimal常量") BigDecimal number) { - BigDecimal result = null; - if (number == null || number.intValue() < -1 || number.intValue() > 1) { - return result; - } - result = new BigDecimal(Math.asin(number.intValue())); - return result; - } - /** * 求反正弦函数 * @@ -95,7 +75,7 @@ public BigDecimal asin(IMessage message, FunctionContext context, */ @FunctionMethod(value = "asin", alias = "asin", comment = "求反正弦函数") public Double asin(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "Integer常量") Integer number) { + @FunctionParamter(value = "Integer", comment = "Integer常量") Integer number) { Double result = null; if (number == null || number < -1.0 || number > 1.0) { return result; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AtanFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AtanFunction.java index 097fa1a4..8fb0e5d3 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AtanFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/AtanFunction.java @@ -36,7 +36,7 @@ public class AtanFunction { */ @FunctionMethod(value = "atan", alias = "atan", comment = "求反正切函数") public Double atan(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求反余弦的字段名或常量") String number) { + @FunctionParamter(value = "String", comment = "代表要求反余弦的字段名或常量") String number) { Double result = null; Double numberTem = Double.parseDouble(FunctionUtils.getValueString(message, context, number)); if (numberTem == null || numberTem < -1.0 || numberTem > 1.0) { @@ -56,7 +56,7 @@ public Double atan(IMessage message, FunctionContext context, */ @FunctionMethod(value = "atan", alias = "atan", comment = "求反正切函数") public Double atan(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "Double常量") Double number) { + @FunctionParamter(value = "Double", comment = "Double常量") Double number) { Double result = null; if (number == null || number < -1 || number > 1) { return result; @@ -75,7 +75,7 @@ public Double atan(IMessage message, FunctionContext context, */ @FunctionMethod(value = "atan", alias = "atan", comment = "求反正切函数") public Double atan(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "Integer常量") Integer number) { + @FunctionParamter(value = "Integer", comment = "Integer常量") Integer number) { Double result = null; if (number == null || number < -1 || number > 1) { return result; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/BinFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/BinFunction.java index 9dd294cc..9d4a746d 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/BinFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/BinFunction.java @@ -36,7 +36,7 @@ public class BinFunction { */ @FunctionMethod(value = "bin", alias = "bin", comment = "返回number的二进制代码表示") public String bin(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量") String number) { + @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量") String number) { String result = null; Integer numberTem = Integer.parseInt(FunctionUtils.getValueString(message, context, number)); if (numberTem == null) { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/BitFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/BitFunction.java index d0688d5f..1ea2121e 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/BitFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/BitFunction.java @@ -37,8 +37,8 @@ public class BitFunction { */ @FunctionMethod(value = "bitor", alias = "bitor", comment = "位取或,输入和输出类型均为整型,且类型一致") public Integer or(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "取位或的第一个值") Integer a, - @FunctionParamter(value = "Integer", comment = "取位或的第二个值") Integer b) { + @FunctionParamter(value = "Integer", comment = "取位或的第一个值") Integer a, + @FunctionParamter(value = "Integer", comment = "取位或的第二个值") Integer b) { Integer result = null; if (a == null || b == null) { return result; @@ -58,8 +58,8 @@ public Integer or(IMessage message, FunctionContext context, */ @FunctionMethod(value = "bitor", alias = "bitor", comment = "位取或,输入和输出类型均为整型,且类型一致") public Integer or(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量") String a, - @FunctionParamter(value = "String", comment = "代表要求反余弦的字段名或常量") String b) { + @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量") String a, + @FunctionParamter(value = "String", comment = "代表要求反余弦的字段名或常量") String b) { Integer result = null; Integer aTem = Integer.parseInt(FunctionUtils.getValueString(message, context, a)); Integer bTem = Integer.parseInt(FunctionUtils.getValueString(message, context, a)); @@ -81,8 +81,8 @@ public Integer or(IMessage message, FunctionContext context, */ @FunctionMethod(value = "bitxor", alias = "bitxor", comment = "按位取异或") public Integer xor(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "取位或的第一个值") Integer a, - @FunctionParamter(value = "Integer", comment = "取位或的第一个值") Integer b) { + @FunctionParamter(value = "Integer", comment = "取位或的第一个值") Integer a, + @FunctionParamter(value = "Integer", comment = "取位或的第一个值") Integer b) { Integer result = null; if (a == null || b == null) { return result; @@ -102,8 +102,8 @@ public Integer xor(IMessage message, FunctionContext context, */ @FunctionMethod(value = "bitxor", alias = "bitxor", comment = "按位取异或") public Integer xor(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量") String a, - @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量") String b) { + @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量") String a, + @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量") String b) { Integer result = null; Integer aTem = Integer.parseInt(FunctionUtils.getValueString(message, context, a)); Integer bTem = Integer.parseInt(FunctionUtils.getValueString(message, context, a)); @@ -124,7 +124,7 @@ public Integer xor(IMessage message, FunctionContext context, */ @FunctionMethod(value = "bitnot", comment = "按位取反", alias = "bitnot") public Integer not(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "取反的Integer常量") Integer a) { + @FunctionParamter(value = "Integer", comment = "取反的Integer常量") Integer a) { Integer result = null; if (a == null) { return result; @@ -143,7 +143,7 @@ public Integer not(IMessage message, FunctionContext context, */ @FunctionMethod(value = "bitnot", comment = "按位取反", alias = "bitnot") public Integer not(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "取反的字段或常量") String a) { + @FunctionParamter(value = "String", comment = "取反的字段或常量") String a) { Integer result = null; Integer aTem = Integer.parseInt(FunctionUtils.getValueString(message, context, a)); if (a == null) { @@ -164,8 +164,8 @@ public Integer not(IMessage message, FunctionContext context, */ @FunctionMethod(value = "bitand", alias = "bitand", comment = "运算符按位“与”操作") public Integer and(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "按位取反的第一个常量值") Integer a, - @FunctionParamter(value = "Integer", comment = "按位取反的第二个常量值") Integer b) { + @FunctionParamter(value = "Integer", comment = "按位取反的第一个常量值") Integer a, + @FunctionParamter(value = "Integer", comment = "按位取反的第二个常量值") Integer b) { Integer result = null; if (a == null || b == null) { return result; @@ -185,8 +185,8 @@ public Integer and(IMessage message, FunctionContext context, */ @FunctionMethod(value = "bitAnd", alias = "bitAnd", comment = "运算符按位“与”操作") public Integer and(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "按位取反的第一个常量值") String a, - @FunctionParamter(value = "String", comment = "按位取反的第二个常量值") String b) { + @FunctionParamter(value = "String", comment = "按位取反的第一个常量值") String a, + @FunctionParamter(value = "String", comment = "按位取反的第二个常量值") String b) { Integer result = null; Integer aTem = Integer.parseInt(FunctionUtils.getValueString(message, context, a)); Integer bTem = Integer.parseInt(FunctionUtils.getValueString(message, context, a)); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/BroundFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/BroundFunction.java index e86eb2a6..0bb8f78e 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/BroundFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/BroundFunction.java @@ -37,7 +37,7 @@ public class BroundFunction { */ @FunctionMethod(value = "bround", alias = "bround", comment = "返回银行家舍入法的值") public Double bround(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "代表要求值的integer常量") Integer base) { + @FunctionParamter(value = "Integer", comment = "代表要求值的integer常量") Integer base) { Double result = null; if (base == null) { return result; @@ -56,7 +56,7 @@ public Double bround(IMessage message, FunctionContext context, */ @FunctionMethod(value = "bround", alias = "bround", comment = "返回银行家舍入法的值") public Double bround(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量") String base) { + @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量") String base) { Double result = null; Double numberTem = Double.parseDouble(FunctionUtils.getValueString(message, context, base)); if (base == null) { @@ -76,8 +76,8 @@ public Double bround(IMessage message, FunctionContext context, */ @FunctionMethod(value = "bround", alias = "bround", comment = "返回银行家舍入法的值") public Double bround(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "代表要求值的integer常量") Integer base, - @FunctionParamter(value = "Integer", comment = "银行家舍入法计算后保留的小数位数") Integer x) { + @FunctionParamter(value = "Integer", comment = "代表要求值的integer常量") Integer base, + @FunctionParamter(value = "Integer", comment = "银行家舍入法计算后保留的小数位数") Integer x) { Double result = null; if (base == null || x == null) { return result; @@ -96,8 +96,8 @@ public Double bround(IMessage message, FunctionContext context, */ @FunctionMethod(value = "bround", alias = "bround", comment = "返回银行家舍入法的值") public Double bround(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量") String base, - @FunctionParamter(value = "Integer", comment = "银行家舍入法计算后保留的小数位数") Integer x) { + @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量") String base, + @FunctionParamter(value = "Integer", comment = "银行家舍入法计算后保留的小数位数") Integer x) { Double result = null; Double numberTem = Double.parseDouble(FunctionUtils.getValueString(message, context, base)); if (base == null || x == null) { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CardinalityFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CardinalityFunction.java index 1e5ddcd5..e0f75f3a 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CardinalityFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CardinalityFunction.java @@ -35,7 +35,7 @@ public class CardinalityFunction { */ @FunctionMethod(value = "cardinality", alias = "cardinality", comment = "返回一个集合中的元素数量") public Integer cardinality(IMessage message, FunctionContext context, - @FunctionParamter(value = "Array", comment = "待求值的集合") Object[] objects) { + @FunctionParamter(value = "Array", comment = "待求值的集合") Object[] objects) { Integer result = null; if (objects == null) { return result; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CbrtFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CbrtFunction.java index 0e8d7c60..ea83fbc4 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CbrtFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CbrtFunction.java @@ -35,7 +35,7 @@ public class CbrtFunction { */ @FunctionMethod(value = "cbrt", alias = "cbrt", comment = "返回立方根") public Double cbrt(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "待求值的常量") Integer number) { + @FunctionParamter(value = "Integer", comment = "待求值的常量") Integer number) { Double result = null; if (number == null || number < 0 || number > 20) { return result; @@ -53,7 +53,7 @@ public Double cbrt(IMessage message, FunctionContext context, */ @FunctionMethod(value = "cbrt", alias = "cbrt", comment = "返回立方根") public Double cbrt(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "待求值的字段名或常量") String number) { + @FunctionParamter(value = "String", comment = "待求值的字段名或常量") String number) { Double result = null; Integer numberTem = Integer.parseInt(FunctionUtils.getValueString(message, context, number)); if (number == null || numberTem < 0 || numberTem > 20) { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CeilFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CeilFunction.java index c7ce3a2c..9f54022c 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CeilFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CeilFunction.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -37,7 +36,7 @@ public class CeilFunction { */ @FunctionMethod(value = "ceil", alias = "ceil", comment = "返回不小于输入值value的最小整数") public Double ceil(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { + @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { Double result = null; Double numberTem = Double.parseDouble(FunctionUtils.getValueString(message, context, number)); if (numberTem == null) { @@ -57,7 +56,7 @@ public Double ceil(IMessage message, FunctionContext context, */ @FunctionMethod(value = "ceil", alias = "ceil", comment = "返回不小于输入值value的最小整数") public Double ceil(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "代表要求值的Double常量") Double number) { + @FunctionParamter(value = "Double", comment = "代表要求值的Double常量") Double number) { Double result = null; if (number == null) { return result; @@ -66,25 +65,6 @@ public Double ceil(IMessage message, FunctionContext context, return result; } - /** - * 返回不小于输入值value的最小整数 - * - * @param message - * @param context - * @param number - * @return - */ - @FunctionMethod(value = "ceil", alias = "ceil", comment = "返回不小于输入值value的最小整数") - public BigDecimal ceil(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "代表要求值的BigDecimal常量") BigDecimal number) { - BigDecimal result = null; - if (number == null) { - return result; - } - result = new BigDecimal(Math.ceil(number.intValue())); - return result; - } - /** * 返回不小于输入值value的最小整数 * @@ -95,7 +75,7 @@ public BigDecimal ceil(IMessage message, FunctionContext context, */ @FunctionMethod(value = "ceil", alias = "ceil", comment = "返回不小于输入值value的最小整数") public Double ceil(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "代表要求值的Integer常量") Integer number) { + @FunctionParamter(value = "Integer", comment = "代表要求值的Integer常量") Integer number) { Double result = null; if (number == null) { return result; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ConvFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ConvFunction.java index 4f0f98f5..bea97ad9 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ConvFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ConvFunction.java @@ -39,9 +39,9 @@ public class ConvFunction { @FunctionMethod(value = "conv", alias = "conv", comment = "进制转换函数") public String conv(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "要转换的字段名称或常量值") String input, - @FunctionParamter(value = "String", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") String from_base, - @FunctionParamter(value = "String", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") String to_base) { + @FunctionParamter(value = "String", comment = "要转换的字段名称或常量值") String input, + @FunctionParamter(value = "String", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") String from_base, + @FunctionParamter(value = "String", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") String to_base) { String result = null; String inputTem = FunctionUtils.getValueString(message, context, input); Integer formTem = Integer.parseInt(FunctionUtils.getValueString(message, context, from_base)); @@ -66,9 +66,9 @@ public String conv(IMessage message, FunctionContext context, @FunctionMethod(value = "conv", alias = "conv", comment = "进制转换函数") public String conv(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "要转换的字段名称或常量值") Double input, - @FunctionParamter(value = "String", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") String from_base, - @FunctionParamter(value = "String", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") String to_base) { + @FunctionParamter(value = "Double", comment = "要转换的字段名称或常量值") Double input, + @FunctionParamter(value = "String", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") String from_base, + @FunctionParamter(value = "String", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") String to_base) { String result = null; Integer formTem = Integer.parseInt(FunctionUtils.getValueString(message, context, from_base)); Integer toTem = Integer.parseInt(FunctionUtils.getValueString(message, context, to_base)); @@ -92,9 +92,9 @@ public String conv(IMessage message, FunctionContext context, @FunctionMethod(value = "conv", alias = "conv", comment = "进制转换函数") public String conv(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "要转换的字段名称或常量值") Double input, - @FunctionParamter(value = "Double", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") Double from_base, - @FunctionParamter(value = "String", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") String to_base) { + @FunctionParamter(value = "Double", comment = "要转换的字段名称或常量值") Double input, + @FunctionParamter(value = "Double", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") Double from_base, + @FunctionParamter(value = "String", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") String to_base) { String result = null; Integer toTem = Integer.parseInt(FunctionUtils.getValueString(message, context, to_base)); if (input == null || from_base == null || toTem == null) { @@ -117,9 +117,9 @@ public String conv(IMessage message, FunctionContext context, @FunctionMethod(value = "conv", alias = "conv", comment = "进制转换函数") public String conv(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "要转换的字段名称或常量值") Double input, - @FunctionParamter(value = "Double", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") Double from_base, - @FunctionParamter(value = "Double", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") Double to_base) { + @FunctionParamter(value = "Double", comment = "要转换的字段名称或常量值") Double input, + @FunctionParamter(value = "Double", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") Double from_base, + @FunctionParamter(value = "Double", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") Double to_base) { String result = null; if (input == null || from_base == null || to_base == null) { return result; @@ -141,9 +141,9 @@ public String conv(IMessage message, FunctionContext context, @FunctionMethod(value = "conv", alias = "conv", comment = "进制转换函数") public String conv(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "要转换的字段名称或常量值") String input, - @FunctionParamter(value = "Double", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") Double from_base, - @FunctionParamter(value = "String", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") String to_base) { + @FunctionParamter(value = "String", comment = "要转换的字段名称或常量值") String input, + @FunctionParamter(value = "Double", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") Double from_base, + @FunctionParamter(value = "String", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") String to_base) { String result = null; String inputTem = FunctionUtils.getValueString(message, context, input); Integer toTem = Integer.parseInt(FunctionUtils.getValueString(message, context, to_base)); @@ -156,9 +156,9 @@ public String conv(IMessage message, FunctionContext context, @FunctionMethod(value = "conv", alias = "conv", comment = "进制转换函数") public String conv(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "要转换的字段名称或常量值") String input, - @FunctionParamter(value = "String", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") String from_base, - @FunctionParamter(value = "Double", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") Double to_base) { + @FunctionParamter(value = "String", comment = "要转换的字段名称或常量值") String input, + @FunctionParamter(value = "String", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") String from_base, + @FunctionParamter(value = "Double", comment = "以十进制表示的进制的值,可接受的的值为2,8,10,16") Double to_base) { String result = null; String inputTem = FunctionUtils.getValueString(message, context, input); Integer formTem = Integer.parseInt(FunctionUtils.getValueString(message, context, from_base)); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CosFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CosFunction.java index 4866fd10..311bf3fd 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CosFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CosFunction.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -37,7 +36,7 @@ public class CosFunction { */ @FunctionMethod(value = "cos", alias = "cos", comment = "余弦函数,输入为弧度值") public Double cos(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的字段名称或常量值") String number) { + @FunctionParamter(value = "String", comment = "代表要求值的字段名称或常量值") String number) { Double result = null; Double numberTem = Double.parseDouble(FunctionUtils.getValueString(message, context, number)); if (numberTem == null) { @@ -57,7 +56,7 @@ public Double cos(IMessage message, FunctionContext context, */ @FunctionMethod(value = "cos", alias = "cos", comment = "余弦函数,输入为弧度值") public Double cos(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "代表要求值的常量值") Double number) { + @FunctionParamter(value = "Double", comment = "代表要求值的常量值") Double number) { Double result = null; if (number == null) { return result; @@ -76,7 +75,7 @@ public Double cos(IMessage message, FunctionContext context, */ @FunctionMethod(value = "cos", alias = "cos", comment = "余弦函数,输入为弧度值") public Double cos(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { + @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { Double result = null; if (number == null) { return result; @@ -85,22 +84,4 @@ public Double cos(IMessage message, FunctionContext context, return result; } - /** - * 余弦函数,输入为弧度值 - * - * @param message - * @param context - * @param number - * @return - */ - @FunctionMethod(value = "cos", alias = "cos", comment = "余弦函数,输入为弧度值") - public BigDecimal cos(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "代表要求值的常量值") BigDecimal number) { - BigDecimal result = null; - if (number == null) { - return result; - } - result = new BigDecimal(Math.cos(number.intValue())); - return result; - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CosHFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CosHFunction.java index 6abdead9..f1cb8a36 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CosHFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CosHFunction.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -37,7 +36,7 @@ public class CosHFunction { */ @FunctionMethod(value = "cosh", alias = "cosh", comment = "双曲余弦函数") public Double cosh(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { + @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { Double result = null; Double numberTem = Double.parseDouble(FunctionUtils.getValueString(message, context, number)); if (numberTem == null) { @@ -57,7 +56,7 @@ public Double cosh(IMessage message, FunctionContext context, */ @FunctionMethod(value = "cosh", alias = "cosh", comment = "双曲余弦函数") public Double cosh(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "代表要求值的常量值") Double number) { + @FunctionParamter(value = "Double", comment = "代表要求值的常量值") Double number) { Double result = null; if (number == null) { return result; @@ -76,7 +75,7 @@ public Double cosh(IMessage message, FunctionContext context, */ @FunctionMethod(value = "cosh", alias = "cosh", comment = "双曲余弦函数") public Double cosh(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { + @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { Double result = null; if (number == null) { return result; @@ -85,22 +84,4 @@ public Double cosh(IMessage message, FunctionContext context, return result; } - /** - * 双曲余弦函数 - * - * @param message - * @param context - * @param number - * @return - */ - @FunctionMethod(value = "cosh", alias = "cosh", comment = "双曲余弦函数") - public BigDecimal cosh(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "代表要求值的常量值") BigDecimal number) { - BigDecimal result = null; - if (number == null) { - return result; - } - result = new BigDecimal(Math.cosh(number.intValue())); - return result; - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CotFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CotFunction.java index aae3e21c..9692ab61 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CotFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/CotFunction.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -37,7 +36,7 @@ public class CotFunction { */ @FunctionMethod(value = "cot", alias = "cot", comment = "余切函数,输入为弧度值") public Double cot(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { + @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { Double result = null; Double numberTem = Double.parseDouble(FunctionUtils.getValueString(message, context, number)); if (numberTem == null) { @@ -57,7 +56,7 @@ public Double cot(IMessage message, FunctionContext context, */ @FunctionMethod(value = "cot", alias = "cot", comment = "余切函数,输入为弧度值") public Double cot(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "代表要求值的常量值") Double number) { + @FunctionParamter(value = "Double", comment = "代表要求值的常量值") Double number) { Double result = null; if (number == null) { return result; @@ -76,7 +75,7 @@ public Double cot(IMessage message, FunctionContext context, */ @FunctionMethod(value = "cot", alias = "cot", comment = "余切函数,输入为弧度值") public Double cot(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { + @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { Double result = null; if (number == null) { return result; @@ -85,22 +84,4 @@ public Double cot(IMessage message, FunctionContext context, return result; } - /** - * 余切函数,输入为弧度值 - * - * @param message - * @param context - * @param number - * @return - */ - @FunctionMethod(value = "cot", alias = "cot", comment = "余切函数,输入为弧度值") - public BigDecimal cot(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "代表要求值的常量值") BigDecimal number) { - BigDecimal result = null; - if (number == null) { - return result; - } - result = new BigDecimal(1 / Math.tan(number.intValue())); - return result; - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/DegreesFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/DegreesFunction.java index dfe58831..dfd391b8 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/DegreesFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/DegreesFunction.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -37,7 +36,7 @@ public class DegreesFunction { */ @FunctionMethod(value = "degrees", alias = "degrees", comment = "将弧度转换为角度") public Double degrees(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "代表要求值的常量值") Double base) { + @FunctionParamter(value = "Double", comment = "代表要求值的常量值") Double base) { Double result = null; if (base == null) { return result; @@ -56,7 +55,7 @@ public Double degrees(IMessage message, FunctionContext context, */ @FunctionMethod(value = "degrees", alias = "degrees", comment = "将弧度转换为角度") public Double degrees(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String base) { + @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String base) { Double numberTem = Double.parseDouble(FunctionUtils.getValueString(message, context, base)); Double result = null; if (base == null) { @@ -66,22 +65,4 @@ public Double degrees(IMessage message, FunctionContext context, return result; } - /** - * 将弧度转换为角度 - * - * @param message - * @param context - * @param base - * @return - */ - @FunctionMethod(value = "degrees", alias = "degrees", comment = "将弧度转换为角度") - public Double degrees(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "代表要求值的常量值") BigDecimal base) { - Double result = null; - if (base == null) { - return result; - } - result = Math.toDegrees(base.intValue()); - return result; - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/DivisionFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/DivisionFunction.java index e6e46537..b66ac5f5 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/DivisionFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/DivisionFunction.java @@ -39,8 +39,8 @@ public class DivisionFunction { */ @FunctionMethod(value = "division", alias = "division", comment = "两个数值的除法返回结果") public Object division(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求乘法的第一个参数") String x, - @FunctionParamter(value = "String", comment = "代表要求乘法的第二个参数") String y) { + @FunctionParamter(value = "String", comment = "代表要求乘法的第一个参数") String x, + @FunctionParamter(value = "String", comment = "代表要求乘法的第二个参数") String y) { Double result = null; String paramX = FunctionUtils.getValueString(message, context, x); String paramY = FunctionUtils.getValueString(message, context, y); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ExpFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ExpFunction.java index b728591c..0aebbc00 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ExpFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ExpFunction.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -37,7 +36,7 @@ public class ExpFunction { */ @FunctionMethod(value = "exp", alias = "exp", comment = "指数函数返回number的指数值") public Double exp(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { + @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { Double result = null; Double numberTem = Double.parseDouble(FunctionUtils.getValueString(message, context, number)); if (numberTem == null) { @@ -57,7 +56,7 @@ public Double exp(IMessage message, FunctionContext context, */ @FunctionMethod(value = "exp", alias = "exp", comment = "指数函数返回number的指数值") public Double exp(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "代表要求值的字段名或常量值") Double number) { + @FunctionParamter(value = "Double", comment = "代表要求值的字段名或常量值") Double number) { Double result = null; if (number == null) { return result; @@ -76,7 +75,7 @@ public Double exp(IMessage message, FunctionContext context, */ @FunctionMethod(value = "exp", alias = "exp", comment = "指数函数返回number的指数值") public Double exp(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { + @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { Double result = null; if (number == null) { return result; @@ -85,22 +84,4 @@ public Double exp(IMessage message, FunctionContext context, return result; } - /** - * 指数函数返回number的指数值 - * - * @param message - * @param context - * @param number - * @return - */ - @FunctionMethod(value = "exp", alias = "exp", comment = "指数函数返回number的指数值") - public BigDecimal exp(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "代表要求值的常量值") BigDecimal number) { - BigDecimal result = null; - if (number == null) { - return result; - } - result = new BigDecimal(Math.exp(number.intValue())); - return result; - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/FactorialFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/FactorialFunction.java index 6f95c78b..ced617e6 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/FactorialFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/FactorialFunction.java @@ -35,7 +35,7 @@ public class FactorialFunction { */ @FunctionMethod(value = "factorial", alias = "factorial", comment = "返回值的阶乘") public Long factorial(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { + @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { Long result = null; if (number == null || number < 0 || number > 20) { return result; @@ -59,7 +59,7 @@ public Long factorial(IMessage message, FunctionContext context, */ @FunctionMethod(value = "factorial", alias = "factorial", comment = "返回值的阶乘") public Long factorial(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { + @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { Long result = null; Integer numberTem = Integer.parseInt(FunctionUtils.getValueString(message, context, number)); if (number == null || numberTem < 0 || numberTem > 20) { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/FloorFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/FloorFunction.java index fd90a428..9e23ecb0 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/FloorFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/FloorFunction.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -37,7 +36,7 @@ public class FloorFunction { */ @FunctionMethod(value = "floor", alias = "floor", comment = "向下取整,返回比当前值小的整数值") public Double floor(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { + @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { Double result = null; String value = FunctionUtils.getValueString(message, context, number); if (value == null) { @@ -65,7 +64,7 @@ public Double floor(IMessage message, FunctionContext context, */ @FunctionMethod(value = "floor", alias = "floor", comment = "向下取整,返回比当前值小的整数值") public Double floor(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "代表要求值的常量值") Double number) { + @FunctionParamter(value = "Double", comment = "代表要求值的常量值") Double number) { Double result = null; if (number == null) { return result; @@ -84,7 +83,7 @@ public Double floor(IMessage message, FunctionContext context, */ @FunctionMethod(value = "floor", alias = "floor", comment = "向下取整,返回比当前值小的整数值") public Double floor(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { + @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { Double result = null; if (number == null) { return result; @@ -93,22 +92,4 @@ public Double floor(IMessage message, FunctionContext context, return result; } - /** - * 向下取整,返回比number小的整数值 - * - * @param message - * @param context - * @param number - * @return - */ - @FunctionMethod(value = "floor", alias = "floor", comment = "向下取整,返回比当前值小的整数值") - public BigDecimal floor(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "代表要求值的常量值") BigDecimal number) { - BigDecimal result = null; - if (number == null) { - return result; - } - result = new BigDecimal(Math.floor(number.intValue())); - return result; - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/HexFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/HexFunction.java index 8b49ce28..b54205ea 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/HexFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/HexFunction.java @@ -36,7 +36,7 @@ public class HexFunction { */ @FunctionMethod(value = "hex", alias = "hex", comment = "将整数或字符转换为十六进制格式") public String hex(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { + @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { String result = null; Integer numberTem = Integer.parseInt(FunctionUtils.getValueString(message, context, number)); if (numberTem == null) { @@ -56,7 +56,7 @@ public String hex(IMessage message, FunctionContext context, */ @FunctionMethod(value = "hex", alias = "hex", comment = "将整数或字符转换为十六进制格式") public String hex(IMessage message, FunctionContext context, - @FunctionParamter(value = "Long", comment = "代表要求的常量值") Long number) { + @FunctionParamter(value = "Long", comment = "代表要求的常量值") Long number) { String result = null; if (number == null) { return result; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/IsDecimalFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/IsDecimalFunction.java index 3c9c5e35..6d64f839 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/IsDecimalFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/IsDecimalFunction.java @@ -26,6 +26,16 @@ @Function public class IsDecimalFunction { + /** + * 将整数或字符转换为十六进制格式 + * + * @param message + * @param context + * @param number + * @return + */ + String regex = "^[a-zA-Z]+$"; + /** * 将整数或字符转换为十六进制格式 * @@ -37,12 +47,13 @@ public class IsDecimalFunction { @FunctionMethod("IS_DECIMAL") public boolean isDecimal(IMessage message, FunctionContext context, @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { - if(number==null){ + if (number == null) { return false; } - number= FunctionUtils.getValueString(message, context, number); + number = FunctionUtils.getValueString(message, context, number); return FunctionUtils.isDouble(number); } + /** * 将整数或字符转换为十六进制格式 * @@ -54,30 +65,20 @@ public boolean isDecimal(IMessage message, FunctionContext context, @FunctionMethod(value = "is_number", comment = "将整数或字符转换为十六进制格式") public boolean isNumber(IMessage message, FunctionContext context, @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { - if(number==null){ + if (number == null) { return false; } - number= FunctionUtils.getValueString(message, context, number); + number = FunctionUtils.getValueString(message, context, number); return FunctionUtils.isLong(number); } - - /** - * 将整数或字符转换为十六进制格式 - * - * @param message - * @param context - * @param number - * @return - */ - String regex="^[a-zA-Z]+$"; @FunctionMethod(value = "IS_ALPHA", comment = "将整数或字符转换为十六进制格式") public boolean isAlpha(IMessage message, FunctionContext context, @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { - if(number==null){ + if (number == null) { return false; } - number= FunctionUtils.getValueString(message, context, number); - return StringUtil.matchRegex(number,regex); + number = FunctionUtils.getValueString(message, context, number); + return StringUtil.matchRegex(number, regex); } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/LnFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/LnFunction.java index 8f8abde5..acb48127 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/LnFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/LnFunction.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -37,7 +36,7 @@ public class LnFunction { */ @FunctionMethod(value = "ln", alias = "ln", comment = "返回值的自然对数") public Double ln(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { + @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { Double result = null; Double numberTem = Double.parseDouble(FunctionUtils.getValueString(message, context, number)); if (numberTem == null) { @@ -57,7 +56,7 @@ public Double ln(IMessage message, FunctionContext context, */ @FunctionMethod(value = "ln", alias = "ln", comment = "返回值的自然对数") public Double ln(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "代表要求值的常量值") Double number) { + @FunctionParamter(value = "Double", comment = "代表要求值的常量值") Double number) { Double result = null; if (number == null) { return result; @@ -76,7 +75,7 @@ public Double ln(IMessage message, FunctionContext context, */ @FunctionMethod(value = "ln", alias = "ln", comment = "返回值的自然对数") public Double ln(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { + @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { Double result = null; if (number == null) { return result; @@ -85,22 +84,4 @@ public Double ln(IMessage message, FunctionContext context, return result; } - /** - * 返回number的自然对数 - * - * @param message - * @param context - * @param number - * @return - */ - @FunctionMethod(value = "ln", alias = "ln", comment = "返回值的自然对数") - public BigDecimal ln(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "代表要求值的常量值") BigDecimal number) { - BigDecimal result = null; - if (number == null) { - return result; - } - result = new BigDecimal(Math.log(2.71828) / Math.log(number.intValue())); - return result; - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/Log10Function.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/Log10Function.java index d428fead..71cc53b1 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/Log10Function.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/Log10Function.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -36,7 +35,7 @@ public class Log10Function { */ @FunctionMethod(value = "log10", alias = "log10", comment = "以10为底返回值的对数") public Double log10(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { + @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { Double result = null; Double baseTem = Double.parseDouble(FunctionUtils.getValueString(message, context, number)); if (baseTem == null) { @@ -55,7 +54,7 @@ public Double log10(IMessage message, FunctionContext context, */ @FunctionMethod(value = "log10", alias = "log10", comment = "以10为底返回值的对数") public Double log10(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "代表要求值的常量值") Double number) { + @FunctionParamter(value = "Double", comment = "代表要求值的常量值") Double number) { Double result = null; if (number == null) { return result; @@ -73,7 +72,7 @@ public Double log10(IMessage message, FunctionContext context, */ @FunctionMethod(value = "log10", alias = "log10", comment = "以10为底返回值的对数") public Double log10(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { + @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { Double result = null; if (number == null) { return result; @@ -82,21 +81,4 @@ public Double log10(IMessage message, FunctionContext context, return result; } - /** - * 以10为底返回number的对数 - * - * @param message - * @param context - * @return - */ - @FunctionMethod(value = "log10", alias = "log10", comment = "以10为底返回值的对数") - public BigDecimal log10(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "代表要求值的常量值") BigDecimal number) { - BigDecimal result = null; - if (number == null) { - return result; - } - result = new BigDecimal(Math.log10(number.doubleValue())); - return result; - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/Log2Function.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/Log2Function.java index 1d6841b2..e0b4a492 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/Log2Function.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/Log2Function.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -36,7 +35,7 @@ public class Log2Function { */ @FunctionMethod(value = "log2", alias = "log2", comment = "以2为底返回number的对数") public Double log2(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { + @FunctionParamter(value = "String", comment = "代表要求值的字段名或常量值") String number) { Double result = null; Double baseTem = Double.parseDouble(FunctionUtils.getValueString(message, context, number)); if (baseTem == 0) { @@ -55,7 +54,7 @@ public Double log2(IMessage message, FunctionContext context, */ @FunctionMethod(value = "log2", alias = "log2", comment = "以2为底返回number的对数") public Double log2(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "代表要求值的常量值") Double number) { + @FunctionParamter(value = "Double", comment = "代表要求值的常量值") Double number) { Double result = null; if (number == null) { return result; @@ -73,7 +72,7 @@ public Double log2(IMessage message, FunctionContext context, */ @FunctionMethod(value = "log2", alias = "log2", comment = "以2为底返回number的对数") public Double log2(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { + @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { Double result = null; if (number == null) { return result; @@ -82,21 +81,4 @@ public Double log2(IMessage message, FunctionContext context, return result; } - /** - * 以2为底返回number的对数 - * - * @param message - * @param context - * @return - */ - @FunctionMethod(value = "log2", alias = "log2", comment = "以2为底返回number的对数") - public BigDecimal log2(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "代表要求值的常量值") BigDecimal number) { - BigDecimal result = null; - if (number == null) { - return result; - } - result = new BigDecimal(Math.log(2) / Math.log(number.doubleValue())); - return result; - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/LogFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/LogFunction.java index 31e22003..d6ac9a63 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/LogFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/LogFunction.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -37,8 +36,8 @@ public class LogFunction { */ @FunctionMethod(value = "log", alias = "log", comment = "返回以base为底的值的对数") public Double log(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "底数值") String base, - @FunctionParamter(value = "String", comment = "要求对数的字段名或常量值") String x) { + @FunctionParamter(value = "String", comment = "底数值") String base, + @FunctionParamter(value = "String", comment = "要求对数的字段名或常量值") String x) { Double result = null; Double baseTem = Double.parseDouble(FunctionUtils.getValueString(message, context, base)); Double xTem = Double.parseDouble(FunctionUtils.getValueString(message, context, x)); @@ -59,8 +58,8 @@ public Double log(IMessage message, FunctionContext context, */ @FunctionMethod(value = "log", alias = "log", comment = "返回以base为底的值的对数") public Double log(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "底数值") Double base, - @FunctionParamter(value = "Double", comment = "要求对数值的常量") Double x) { + @FunctionParamter(value = "Double", comment = "底数值") Double base, + @FunctionParamter(value = "Double", comment = "要求对数值的常量") Double x) { Double result = null; if (base == null || x == null) { return result; @@ -79,8 +78,8 @@ public Double log(IMessage message, FunctionContext context, */ @FunctionMethod(value = "log", alias = "log", comment = "返回以base为底的值的对数") public Double log(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "底数值") Integer base, - @FunctionParamter(value = "Integer", comment = "要求对数值的常量") Integer x) { + @FunctionParamter(value = "Integer", comment = "底数值") Integer base, + @FunctionParamter(value = "Integer", comment = "要求对数值的常量") Integer x) { Double result = null; if (base == null || x == null) { return result; @@ -89,23 +88,4 @@ public Double log(IMessage message, FunctionContext context, return result; } - /** - * 返回以base为底的x的对数 - * - * @param message - * @param context - * @param x - * @return - */ - @FunctionMethod(value = "log", alias = "log", comment = "返回以base为底的值的对数") - public BigDecimal log(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "底数值") BigDecimal base, - @FunctionParamter(value = "BigDecimal", comment = "要求对数值的常量") BigDecimal x) { - BigDecimal result = null; - if (base == null || x == null) { - return result; - } - result = new BigDecimal(Math.log(base.doubleValue()) / Math.log(x.doubleValue())); - return result; - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/MathFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/MathFunction.java index 4c7b3cfe..c8574006 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/MathFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/MathFunction.java @@ -34,16 +34,16 @@ public class MathFunction { */ @FunctionMethod(value = "math", comment = "将输入值number截取到指定小数点位置") public Object mathOperator(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String operator, - @FunctionParamter(value = "String", comment = "代表要截取小数点的位置") String a, String b) { + @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String operator, + @FunctionParamter(value = "String", comment = "代表要截取小数点的位置") String a, String b) { Object aValue = FunctionUtils.getValue(message, context, a); Object bValue = FunctionUtils.getValue(message, context, b); if (Double.class.isInstance(aValue)) { - return mathOperator(message, context, operator, (Double)aValue, (Double)bValue); + return mathOperator(message, context, operator, (Double) aValue, (Double) bValue); } else if (Integer.class.isInstance(aValue)) { - return mathOperator(message, context, operator, (Integer)aValue, (Integer)bValue); + return mathOperator(message, context, operator, (Integer) aValue, (Integer) bValue); } else { - return mathOperator(message, context, operator, (Long)aValue, (Long)bValue); + return mathOperator(message, context, operator, (Long) aValue, (Long) bValue); } } @@ -57,8 +57,8 @@ public Object mathOperator(IMessage message, FunctionContext context, */ @FunctionMethod(value = "mathl", comment = "将输入值number截取到指定小数点位置") public Long mathOperator(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String operator, - @FunctionParamter(value = "String", comment = "代表要截取小数点的位置") Long a, Long b) { + @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String operator, + @FunctionParamter(value = "String", comment = "代表要截取小数点的位置") Long a, Long b) { operator = FunctionUtils.getValueString(message, context, operator); if ("+".equals(operator)) { return a + b; @@ -84,8 +84,8 @@ public Long mathOperator(IMessage message, FunctionContext context, */ @FunctionMethod(value = "mathi", comment = "将输入值number截取到指定小数点位置") public Integer mathOperator(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String operator, - @FunctionParamter(value = "String", comment = "代表要截取小数点的位置") Integer a, Integer b) { + @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String operator, + @FunctionParamter(value = "String", comment = "代表要截取小数点的位置") Integer a, Integer b) { operator = FunctionUtils.getValueString(message, context, operator); if ("+".equals(operator)) { return a + b; @@ -111,8 +111,8 @@ public Integer mathOperator(IMessage message, FunctionContext context, */ @FunctionMethod(value = "mathd", comment = "将输入值number截取到指定小数点位置") public Double mathOperator(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String operator, - @FunctionParamter(value = "String", comment = "代表要截取小数点的位置") Double a, Double b) { + @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String operator, + @FunctionParamter(value = "String", comment = "代表要截取小数点的位置") Double a, Double b) { operator = FunctionUtils.getValueString(message, context, operator); if ("+".equals(operator)) { return a + b; @@ -129,7 +129,6 @@ public Double mathOperator(IMessage message, FunctionContext context, } } - /** * 将输入值number截取到指定小数点位置 * @@ -143,11 +142,11 @@ public long mathOperator(IMessage message, FunctionContext context, @FunctionParamter(value = "String", comment = "代表要截取小数点的位置") String mod) { operator = FunctionUtils.getValueString(message, context, operator); mod = FunctionUtils.getValueString(message, context, mod); - if(operator==null||mod==null){ + if (operator == null || mod == null) { return -1; } - Long operatorLong=Long.valueOf(operator); - Long modLong=Long.valueOf(mod); - return operatorLong%modLong; + Double operatorLong = Double.valueOf(operator); + Double modLong = Double.valueOf(mod); + return operatorLong.longValue() % modLong.longValue(); } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/MultiplicationFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/MultiplicationFunction.java index 5779044b..a0c0bdc4 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/MultiplicationFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/MultiplicationFunction.java @@ -39,8 +39,8 @@ public class MultiplicationFunction { */ @FunctionMethod(value = "multiplication", alias = "multiplication", comment = "两个数值的乘法返回结果") public Object multiplication(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求乘法的第一个参数") String x, - @FunctionParamter(value = "String", comment = "代表要求乘法的第二个参数") String y) { + @FunctionParamter(value = "String", comment = "代表要求乘法的第一个参数") String x, + @FunctionParamter(value = "String", comment = "代表要求乘法的第二个参数") String y) { Double result = null; String paramX = FunctionUtils.getValueString(message, context, x); String paramY = FunctionUtils.getValueString(message, context, y); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/OperationFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/OperationFunction.java index dfc6336c..b490bd9a 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/OperationFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/OperationFunction.java @@ -37,8 +37,8 @@ public class OperationFunction { */ @FunctionMethod(value = "add", alias = "add", comment = "返回A + B的结果") public Integer add(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "第一个操作数") String a, - @FunctionParamter(value = "String", comment = "第二个操作数") String b) { + @FunctionParamter(value = "String", comment = "第一个操作数") String a, + @FunctionParamter(value = "String", comment = "第二个操作数") String b) { Integer result = null; Integer aTem = Integer.parseInt(FunctionUtils.getValueString(message, context, a)); Integer bTem = Integer.parseInt(FunctionUtils.getValueString(message, context, b)); @@ -60,8 +60,8 @@ public Integer add(IMessage message, FunctionContext context, */ @FunctionMethod(value = "minus ", alias = "minus", comment = "返回A – B的结果") public Integer minus(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "第一个操作数") String a, - @FunctionParamter(value = "String", comment = "第二个操作数") String b) { + @FunctionParamter(value = "String", comment = "第一个操作数") String a, + @FunctionParamter(value = "String", comment = "第二个操作数") String b) { Integer result = null; Integer aTem = Integer.parseInt(FunctionUtils.getValueString(message, context, a)); Integer bTem = Integer.parseInt(FunctionUtils.getValueString(message, context, b)); @@ -83,8 +83,8 @@ public Integer minus(IMessage message, FunctionContext context, */ @FunctionMethod(value = "multiply", alias = "multiply", comment = "返回A * B的结果") public Integer multiply(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "第一个操作数") String a, - @FunctionParamter(value = "String", comment = "第二个操作数") String b) { + @FunctionParamter(value = "String", comment = "第一个操作数") String a, + @FunctionParamter(value = "String", comment = "第二个操作数") String b) { Integer result = null; Integer aTem = Integer.parseInt(FunctionUtils.getValueString(message, context, a)); Integer bTem = Integer.parseInt(FunctionUtils.getValueString(message, context, b)); @@ -106,8 +106,8 @@ public Integer multiply(IMessage message, FunctionContext context, */ @FunctionMethod(value = "divide", alias = "divide", comment = "返回A / B的结果") public Integer divide(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "第一个操作数") String a, - @FunctionParamter(value = "String", comment = "第二个操作数") String b) { + @FunctionParamter(value = "String", comment = "第一个操作数") String a, + @FunctionParamter(value = "String", comment = "第二个操作数") String b) { Integer result = null; Integer aTem = Integer.parseInt(FunctionUtils.getValueString(message, context, a)); Integer bTem = Integer.parseInt(FunctionUtils.getValueString(message, context, b)); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/PowerFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/PowerFunction.java index 7e0d0640..2575293a 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/PowerFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/PowerFunction.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -37,8 +36,8 @@ public class PowerFunction { */ @FunctionMethod(value = "power", alias = "power", comment = "返回x的y次方") public Double power(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "Double常量代表底数") String base, - @FunctionParamter(value = "String", comment = "Double常量代表底数") String x) { + @FunctionParamter(value = "String", comment = "Double常量代表底数") String base, + @FunctionParamter(value = "String", comment = "Double常量代表底数") String x) { Double result = null; Double baseTem = Double.parseDouble(FunctionUtils.getValueString(message, context, base)); Double xTem = Double.parseDouble(FunctionUtils.getValueString(message, context, x)); @@ -59,8 +58,8 @@ public Double power(IMessage message, FunctionContext context, */ @FunctionMethod(value = "power", alias = "power", comment = "返回x的y次方") public Double power(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "Double常量代表底数") Double base, - @FunctionParamter(value = "Double", comment = "Double常量代表底数") Double x) { + @FunctionParamter(value = "Double", comment = "Double常量代表底数") Double base, + @FunctionParamter(value = "Double", comment = "Double常量代表底数") Double x) { Double result = null; if (base == null || x == null) { return result; @@ -79,8 +78,8 @@ public Double power(IMessage message, FunctionContext context, */ @FunctionMethod(value = "power", alias = "power", comment = "返回x的y次方") public Double power(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "Integer常量代表底数") Integer base, - @FunctionParamter(value = "Integer", comment = "Integer常量代表底数") Integer x) { + @FunctionParamter(value = "Integer", comment = "Integer常量代表底数") Integer base, + @FunctionParamter(value = "Integer", comment = "Integer常量代表底数") Integer x) { Double result = null; if (base == null || x == null) { return result; @@ -88,24 +87,4 @@ public Double power(IMessage message, FunctionContext context, result = Math.pow(base, x); return result; } - - /** - * 返回x的y次方 - * - * @param message - * @param context - * @param x - * @return - */ - @FunctionMethod(value = "power", alias = "power", comment = "返回x的y次方") - public BigDecimal power(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "BigDecimal常量代表底数") BigDecimal base, - @FunctionParamter(value = "BigDecimal", comment = "BigDecimal常量代表幂数") BigDecimal x) { - BigDecimal result = null; - if (base == null || x == null) { - return result; - } - result = new BigDecimal(Math.pow(base.doubleValue(), x.doubleValue())); - return result; - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/RoundFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/RoundFunction.java index 6bcdec12..79bbdd4e 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/RoundFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/RoundFunction.java @@ -37,8 +37,8 @@ public class RoundFunction { */ @FunctionMethod(value = "round", comment = "四舍五入到指定小数点位置") public Double round(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表要求值的列名或常量") String base, - @FunctionParamter(value = "string", comment = "四舍五入计算到小数点后的位置") String x) { + @FunctionParamter(value = "string", comment = "代表要求值的列名或常量") String base, + @FunctionParamter(value = "string", comment = "四舍五入计算到小数点后的位置") String x) { Double result = null; Double baseTem = Double.parseDouble(FunctionUtils.getValueString(message, context, base)); Integer xTem = Integer.parseInt(FunctionUtils.getValueString(message, context, x)); @@ -60,8 +60,8 @@ public Double round(IMessage message, FunctionContext context, */ @FunctionMethod(value = "round", comment = "四舍五入到指定小数点位置") public Double round(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "代表要求值的常量") Double base, - @FunctionParamter(value = "Double", comment = "四舍五入计算到小数点后的位置") Double x) { + @FunctionParamter(value = "Double", comment = "代表要求值的常量") Double base, + @FunctionParamter(value = "Double", comment = "四舍五入计算到小数点后的位置") Double x) { Double result = null; if (base == null || x == null) { return result; @@ -81,8 +81,8 @@ public Double round(IMessage message, FunctionContext context, */ @FunctionMethod(value = "round", comment = "四舍五入到指定小数点位置") public Double round(IMessage message, FunctionContext context, - @FunctionParamter(value = "integer", comment = "代表要求值的常量") Integer base, - @FunctionParamter(value = "integer", comment = "四舍五入计算到小数点后的位置") Integer x) { + @FunctionParamter(value = "integer", comment = "代表要求值的常量") Integer base, + @FunctionParamter(value = "integer", comment = "四舍五入计算到小数点后的位置") Integer x) { Double result = null; if (base == null || x == null) { return result; @@ -92,23 +92,4 @@ public Double round(IMessage message, FunctionContext context, return result; } - /** - * 四舍五入到指定小数点位置 - * - * @param message - * @param context - * @param x - * @return - */ - @FunctionMethod(value = "round", comment = "四舍五入到指定小数点位置") - public BigDecimal round(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "代表要求值的常量") BigDecimal base, - @FunctionParamter(value = "BigDecimal", comment = "四舍五入计算到小数点后的位置") BigDecimal x) { - BigDecimal result = null; - if (base == null || x == null) { - return result; - } - result = new BigDecimal(base.setScale(x.intValue(), BigDecimal.ROUND_HALF_UP).doubleValue()); - return result; - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ShiftLeftFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ShiftLeftFunction.java index 771756cf..8155e6cd 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ShiftLeftFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ShiftLeftFunction.java @@ -35,8 +35,8 @@ public class ShiftLeftFunction { */ @FunctionMethod(value = "shiftleft", comment = "按位左移(<<)") public Integer shiftleft(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要操作的列名称或常量") String number, - @FunctionParamter(value = "String", comment = "代表要移动的位数") String number2) { + @FunctionParamter(value = "String", comment = "代表要操作的列名称或常量") String number, + @FunctionParamter(value = "String", comment = "代表要移动的位数") String number2) { Integer result = null; if (number == null || number2 == null) { return result; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ShiftRightFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ShiftRightFunction.java index 22de0b34..7c82edef 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ShiftRightFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ShiftRightFunction.java @@ -35,8 +35,8 @@ public class ShiftRightFunction { */ @FunctionMethod(value = "shiftright", comment = "按位右移(>>)") public Integer shiftright(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要操作的列名称或常量") String number, - @FunctionParamter(value = "String", comment = "代表要移动的位数") String number2) { + @FunctionParamter(value = "String", comment = "代表要操作的列名称或常量") String number, + @FunctionParamter(value = "String", comment = "代表要移动的位数") String number2) { Integer result = null; if (number == null || number2 == null) { return result; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ShiftRightUnSignedFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ShiftRightUnSignedFunction.java index 48b7f089..f15ead9e 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ShiftRightUnSignedFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/ShiftRightUnSignedFunction.java @@ -35,8 +35,8 @@ public class ShiftRightUnSignedFunction { */ @FunctionMethod(value = "shiftrightunsigned", comment = "无符号按位右移(>>>)") public Integer shiftrightunsigned(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要操作的列名称或常量") String number, - @FunctionParamter(value = "String", comment = "代表要移动的位数") String number2) { + @FunctionParamter(value = "String", comment = "代表要操作的列名称或常量") String number, + @FunctionParamter(value = "String", comment = "代表要移动的位数") String number2) { Integer result = null; if (number == null || number2 == null) { return result; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SignFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SignFunction.java index aad09d5c..96fc5705 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SignFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SignFunction.java @@ -36,7 +36,7 @@ public class SignFunction { */ @FunctionMethod(value = "sign", comment = "取输入数据的符号") public Double sign(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String base) { + @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String base) { Double result = null; if (base == null) { return result; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SinFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SinFunction.java index 298714a1..1a3abe49 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SinFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SinFunction.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -37,7 +36,7 @@ public class SinFunction { */ @FunctionMethod(value = "sin", comment = "求正弦函数值") public Double sin(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求的值的列名称或常量值") String number) { + @FunctionParamter(value = "String", comment = "代表要求的值的列名称或常量值") String number) { Double result = null; Double numberTem = Double.parseDouble(FunctionUtils.getValueString(message, context, number)); if (numberTem == null) { @@ -57,7 +56,7 @@ public Double sin(IMessage message, FunctionContext context, */ @FunctionMethod(value = "sin", comment = "求正弦函数值") public Double sin(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "代表要求的值的常量值") Double number) { + @FunctionParamter(value = "Double", comment = "代表要求的值的常量值") Double number) { Double result = null; if (number == null) { return result; @@ -76,7 +75,7 @@ public Double sin(IMessage message, FunctionContext context, */ @FunctionMethod(value = "sin", comment = "求正弦函数值") public Double sin(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "代表要求的值的常量值") Integer number) { + @FunctionParamter(value = "Integer", comment = "代表要求的值的常量值") Integer number) { Double result = null; if (number == null) { return result; @@ -85,22 +84,4 @@ public Double sin(IMessage message, FunctionContext context, return result; } - /** - * 正弦函数 - * - * @param message - * @param context - * @param number - * @return - */ - @FunctionMethod(value = "sin", comment = "求正弦函数值") - public BigDecimal sin(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "代表要求的值的常量值") BigDecimal number) { - BigDecimal result = null; - if (number == null) { - return result; - } - result = new BigDecimal(Math.sin(number.intValue())); - return result; - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SinHFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SinHFunction.java index 074e8e13..12cf6296 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SinHFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SinHFunction.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -37,7 +36,7 @@ public class SinHFunction { */ @FunctionMethod(value = "sinh", comment = "双曲正弦函数") public Double sinh(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String number) { + @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String number) { Double result = null; Double numberTem = Double.parseDouble(FunctionUtils.getValueString(message, context, number)); if (numberTem == null) { @@ -57,7 +56,7 @@ public Double sinh(IMessage message, FunctionContext context, */ @FunctionMethod(value = "sinh", comment = "双曲正弦函数") public Double sinh(IMessage message, FunctionContext context, - @FunctionParamter(value = "Double", comment = "代表要求值的常量值") Double number) { + @FunctionParamter(value = "Double", comment = "代表要求值的常量值") Double number) { Double result = null; if (number == null) { return result; @@ -76,7 +75,7 @@ public Double sinh(IMessage message, FunctionContext context, */ @FunctionMethod(value = "sinh", comment = "双曲正弦函数") public Double sinh(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { + @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { Double result = null; if (number == null) { return result; @@ -85,22 +84,4 @@ public Double sinh(IMessage message, FunctionContext context, return result; } - /** - * 双曲正弦函数 - * - * @param message - * @param context - * @param number - * @return - */ - @FunctionMethod(value = "sinh", comment = "双曲正弦函数") - public BigDecimal sinh(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "代表要求值的常量值") BigDecimal number) { - BigDecimal result = null; - if (number == null) { - return result; - } - result = new BigDecimal(Math.sinh(number.intValue())); - return result; - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SqrtFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SqrtFunction.java index f14010a5..19ede9cf 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SqrtFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SqrtFunction.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -37,7 +36,7 @@ public class SqrtFunction { */ @FunctionMethod(value = "sqrt", comment = "计算平方根") public Double sqrt(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String number) { + @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String number) { Double result = null; Double numberTem = Double.parseDouble(FunctionUtils.getValueString(message, context, number)); if (numberTem == null) { @@ -57,7 +56,7 @@ public Double sqrt(IMessage message, FunctionContext context, */ @FunctionMethod(value = "sqrt", comment = "计算平方根") public Double sqrt(IMessage message, FunctionContext context, - @FunctionParamter(value = "double", comment = "代表要求值的常量值") Double number) { + @FunctionParamter(value = "double", comment = "代表要求值的常量值") Double number) { Double result = null; if (number == null) { return result; @@ -76,7 +75,7 @@ public Double sqrt(IMessage message, FunctionContext context, */ @FunctionMethod(value = "sqrt", comment = "计算平方根") public Double sqrt(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { + @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { Double result = null; if (number == null) { return result; @@ -85,22 +84,4 @@ public Double sqrt(IMessage message, FunctionContext context, return result; } - /** - * 计算平方根 - * - * @param message - * @param context - * @param number - * @return - */ - @FunctionMethod(value = "sqrt", comment = "计算平方根") - public BigDecimal sqrt(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "代表要求值的常量值") BigDecimal number) { - BigDecimal result = null; - if (number == null) { - return result; - } - result = new BigDecimal(Math.sqrt(number.intValue())); - return result; - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SubtractionFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SubtractionFunction.java index 122b18d1..5ef85109 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SubtractionFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/SubtractionFunction.java @@ -39,8 +39,8 @@ public class SubtractionFunction { */ @FunctionMethod(value = "subtraction", alias = "subtraction", comment = "两个数值的减法返回结果") public Object subtraction(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求乘法的第一个参数") String x, - @FunctionParamter(value = "String", comment = "代表要求乘法的第二个参数") String y) { + @FunctionParamter(value = "String", comment = "代表要求乘法的第一个参数") String x, + @FunctionParamter(value = "String", comment = "代表要求乘法的第二个参数") String y) { Double result = null; String paramX = FunctionUtils.getValueString(message, context, x); String paramY = FunctionUtils.getValueString(message, context, y); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/TanFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/TanFunction.java index f9075a4d..282fad9a 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/TanFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/TanFunction.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -37,7 +36,7 @@ public class TanFunction { */ @FunctionMethod(value = "tan", comment = "正切函数,输入为弧度值") public Double tan(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String number) { + @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String number) { Double result = null; Double numberTem = Double.parseDouble(FunctionUtils.getValueString(message, context, number)); if (numberTem == null) { @@ -57,7 +56,7 @@ public Double tan(IMessage message, FunctionContext context, */ @FunctionMethod(value = "tan", comment = "正切函数,输入为弧度值") public Double tan(IMessage message, FunctionContext context, - @FunctionParamter(value = "double", comment = "代表要求值的常量值") Double number) { + @FunctionParamter(value = "double", comment = "代表要求值的常量值") Double number) { Double result = null; if (number == null) { return result; @@ -76,7 +75,7 @@ public Double tan(IMessage message, FunctionContext context, */ @FunctionMethod(value = "tan", comment = "正切函数,输入为弧度值") public Double tan(IMessage message, FunctionContext context, - @FunctionParamter(value = "integer", comment = "代表要求值的常量值") Integer number) { + @FunctionParamter(value = "integer", comment = "代表要求值的常量值") Integer number) { Double result = null; if (number == null) { return result; @@ -85,22 +84,4 @@ public Double tan(IMessage message, FunctionContext context, return result; } - /** - * 正切函数,输入为弧度值 - * - * @param message - * @param context - * @param number - * @return - */ - @FunctionMethod(value = "tan", comment = "正切函数,输入为弧度值") - public BigDecimal tan(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "代表要求值的常量值") BigDecimal number) { - BigDecimal result = null; - if (number == null) { - return result; - } - result = new BigDecimal(Math.tan(number.intValue())); - return result; - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/TanHFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/TanHFunction.java index 7685ceb1..5d025f1e 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/TanHFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/TanHFunction.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.math; -import java.math.BigDecimal; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -37,7 +36,7 @@ public class TanHFunction { */ @FunctionMethod(value = "tanh", comment = "双曲正切函数") public Double tanh(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String number) { + @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String number) { Double result = null; Double numberTem = Double.parseDouble(FunctionUtils.getValueString(message, context, number)); if (numberTem == null) { @@ -57,7 +56,7 @@ public Double tanh(IMessage message, FunctionContext context, */ @FunctionMethod(value = "tanh", comment = "双曲正切函数") public Double tanh(IMessage message, FunctionContext context, - @FunctionParamter(value = "double", comment = "代表要求值的常量值") Double number) { + @FunctionParamter(value = "double", comment = "代表要求值的常量值") Double number) { Double result = null; if (number == null) { return result; @@ -76,7 +75,7 @@ public Double tanh(IMessage message, FunctionContext context, */ @FunctionMethod(value = "tanh", comment = "双曲正切函数") public Double tanh(IMessage message, FunctionContext context, - @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { + @FunctionParamter(value = "Integer", comment = "代表要求值的常量值") Integer number) { Double result = null; if (number == null) { return result; @@ -85,22 +84,4 @@ public Double tanh(IMessage message, FunctionContext context, return result; } - /** - * 双曲正切函数 - * - * @param message - * @param context - * @param number - * @return - */ - @FunctionMethod(value = "tanh", comment = "双曲正切函数") - public BigDecimal tanh(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "代表要求值的常量值") BigDecimal number) { - BigDecimal result = null; - if (number == null) { - return result; - } - result = new BigDecimal(Math.tanh(number.intValue())); - return result; - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/TruncFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/TruncFunction.java index 3d0c4707..e7e0d9f7 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/TruncFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/TruncFunction.java @@ -37,8 +37,8 @@ public class TruncFunction { */ @FunctionMethod(value = "trunc", comment = "将输入值number截取到指定小数点位置") public Double trunc(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String base, - @FunctionParamter(value = "String", comment = "代表要截取小数点的位置") String x) { + @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String base, + @FunctionParamter(value = "String", comment = "代表要截取小数点的位置") String x) { Double result = null; if (base == null || x == null) { return result; @@ -60,8 +60,8 @@ public Double trunc(IMessage message, FunctionContext context, */ @FunctionMethod(value = "trunc", comment = "将输入值number截取到指定小数点位置") public Double trunc(IMessage message, FunctionContext context, - @FunctionParamter(value = "double", comment = "代表要求值的常量值") Double base, - @FunctionParamter(value = "double", comment = "代表要截取小数点的位置") Double x) { + @FunctionParamter(value = "double", comment = "代表要求值的常量值") Double base, + @FunctionParamter(value = "double", comment = "代表要截取小数点的位置") Double x) { Double result = null; if (base == null || x == null) { return result; @@ -81,8 +81,8 @@ public Double trunc(IMessage message, FunctionContext context, */ @FunctionMethod(value = "trunc", comment = "将输入值number截取到指定小数点位置") public Double trunc(IMessage message, FunctionContext context, - @FunctionParamter(value = "integer", comment = "代表要求值的常量值") Integer base, - @FunctionParamter(value = "integer", comment = "代表要截取小数点的位置") Integer x) { + @FunctionParamter(value = "integer", comment = "代表要求值的常量值") Integer base, + @FunctionParamter(value = "integer", comment = "代表要截取小数点的位置") Integer x) { Double result = null; if (base == null || x == null) { return result; @@ -92,24 +92,4 @@ public Double trunc(IMessage message, FunctionContext context, return result; } - /** - * 将输入值number截取到指定小数点位置 - * - * @param message - * @param context - * @param x - * @return - */ - @FunctionMethod(value = "trunc", comment = "将输入值number截取到指定小数点位置") - public BigDecimal trunc(IMessage message, FunctionContext context, - @FunctionParamter(value = "BigDecimal", comment = "代表要求值的常量值") BigDecimal base, - @FunctionParamter(value = "BigDecimal", comment = "代表要截取小数点的位置") BigDecimal x) { - BigDecimal result = null; - if (base == null || x == null) { - return result; - } - ; - result = new BigDecimal(base.setScale(x.intValue(), BigDecimal.ROUND_FLOOR).doubleValue()); - return result; - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/UnHexFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/UnHexFunction.java index 592a28a9..24073d5a 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/UnHexFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/math/UnHexFunction.java @@ -36,7 +36,7 @@ public class UnHexFunction { */ @FunctionMethod(value = "unhex", comment = "返回十六进制字符串所代表的字符串") public String unhex(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String base) { + @FunctionParamter(value = "String", comment = "代表要求值的列名称或常量值") String base) { String result = ""; if (base == null) { return result; @@ -45,7 +45,7 @@ public String unhex(IMessage message, FunctionContext context, byte[] baKeyword = new byte[s.length() / 2]; for (int i = 0; i < baKeyword.length; i++) { try { - baKeyword[i] = (byte)(0xff & Integer.parseInt(s.substring(i * 2, i * 2 + 2), 16)); + baKeyword[i] = (byte) (0xff & Integer.parseInt(s.substring(i * 2, i * 2 + 2), 16)); } catch (Exception e) { throw new RuntimeException("转换错误 " + s, e); } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/offset/OffsetFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/offset/OffsetFunction.java index 87c4705e..5ed6f311 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/offset/OffsetFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/offset/OffsetFunction.java @@ -38,7 +38,7 @@ public class OffsetFunction { */ @FunctionMethod(value = "offset", alias = "progress", comment = "设置offset") public void extractOffset(IMessage message, AbstractContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String... fieldNames) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String... fieldNames) { if (fieldNames == null) { return; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/GrokFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/GrokFunction.java index babbe393..6e09771d 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/GrokFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/GrokFunction.java @@ -37,31 +37,6 @@ public class GrokFunction { grokCompiler.registerDefaultPatterns(); } - @FunctionMethod(value = "grok", alias = "GROK") - public JSONObject doGrok(IMessage message, AbstractContext context, String fieldName, String grokStr) { - /* - 传入自定义的pattern, 会从已注册的patterns里面进行配对, 例如: TIMESTAMP_ISO8601:timestamp1, TIMESTAMP_ISO8601在注册的 - patterns里面有对应的解析格式, 配对成功后, 会在match时按照固定的解析格式将解析结果存入map中, 此处timestamp1作为输出的key - */ - grokStr = FunctionUtils.getConstant( grokStr); - Grok grok = grokCompiler.compile(grokStr); - fieldName = FunctionUtils.getConstant(fieldName); - String logMsg = message.getMessageBody().getString(fieldName); - // 通过match()方法进行匹配, 对log进行解析, 按照指定的格式进行输出 - Match grokMatch = grok.match(logMsg); - // 获取结果 - Map resultMap = grokMatch.capture(); - message.getMessageBody().putAll(resultMap); - return message.getMessageBody(); - } - - @FunctionMethod(value = "add_grok", alias = "addGrok") - public void addGrok(IMessage message, AbstractContext context, String name, String pattern) { - name = FunctionUtils.getValueString(message, context, name); - pattern = FunctionUtils.getValueString(message, context, pattern); - grokCompiler.register(name, pattern); - } - public static void main(String[] args) { String log = "localhost GET /index.html 1024 0.016"; String grok = "%{IPORHOST:client} %{WORD:method} %{URIPATHPARAM:request} %{INT:size} %{NUMBER:duration}"; @@ -87,4 +62,29 @@ public static void main(String[] args) { // value.remove("log"); System.out.println(value); } + + @FunctionMethod(value = "grok", alias = "GROK") + public JSONObject doGrok(IMessage message, AbstractContext context, String fieldName, String grokStr) { + /* + 传入自定义的pattern, 会从已注册的patterns里面进行配对, 例如: TIMESTAMP_ISO8601:timestamp1, TIMESTAMP_ISO8601在注册的 + patterns里面有对应的解析格式, 配对成功后, 会在match时按照固定的解析格式将解析结果存入map中, 此处timestamp1作为输出的key + */ + grokStr = FunctionUtils.getConstant(grokStr); + Grok grok = grokCompiler.compile(grokStr); + fieldName = FunctionUtils.getConstant(fieldName); + String logMsg = message.getMessageBody().getString(fieldName); + // 通过match()方法进行匹配, 对log进行解析, 按照指定的格式进行输出 + Match grokMatch = grok.match(logMsg); + // 获取结果 + Map resultMap = grokMatch.capture(); + message.getMessageBody().putAll(resultMap); + return message.getMessageBody(); + } + + @FunctionMethod(value = "add_grok", alias = "addGrok") + public void addGrok(IMessage message, AbstractContext context, String name, String pattern) { + name = FunctionUtils.getValueString(message, context, name); + pattern = FunctionUtils.getValueString(message, context, pattern); + grokCompiler.register(name, pattern); + } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/LogParserFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/LogParserFunction.java index 51e56973..439af2d9 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/LogParserFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/LogParserFunction.java @@ -33,9 +33,9 @@ public class LogParserFunction { @FunctionMethod(value = "val", alias = "const", comment = "把数据做常量处理") public String doConstants(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名") String fieldName) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名") String fieldName) { String log = FunctionUtils.getValueString(message, context, fieldName); - Map flags = (Map)context.get(CONST_MAP_KEY); + Map flags = (Map) context.get(CONST_MAP_KEY); if (flags == null) { flags = new HashMap<>(); context.put(CONST_MAP_KEY, flags); @@ -48,7 +48,7 @@ public String doConstants(IMessage message, FunctionContext context, @FunctionMethod(value = "backet", alias = "backet", comment = "把带括号的换成换位符") public String doBacket(IMessage message, FunctionContext context, String fieldName, String... signs) { String log = FunctionUtils.getValueString(message, context, fieldName); - Map flags = (Map)context.get(CONST_MAP_KEY); + Map flags = (Map) context.get(CONST_MAP_KEY); if (flags == null) { flags = new HashMap<>(); context.put(CONST_MAP_KEY, flags); @@ -71,8 +71,8 @@ public String doBacket(IMessage message, FunctionContext context, String fieldNa @FunctionMethod(value = "parse_express", alias = "express", comment = "通过分割符来进行日志解析") public JSONObject parseExpression(IMessage message, FunctionContext context, - @FunctionParamter(value = "String", comment = "代表字符串的字段名或常量") String fieldName, - @FunctionParamter(value = "String", comment = "代表分隔符的字段名或常量") String sign) { + @FunctionParamter(value = "String", comment = "代表字符串的字段名或常量") String fieldName, + @FunctionParamter(value = "String", comment = "代表分隔符的字段名或常量") String sign) { sign = FunctionUtils.getValueString(message, context, sign); String log = FunctionUtils.getValueString(message, context, fieldName); Map result = LogParserUtil.parseExpression(log, sign); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/Paser2JsonFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/Paser2JsonFunction.java index 767153be..cf961582 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/Paser2JsonFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/Paser2JsonFunction.java @@ -22,8 +22,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; @@ -32,8 +30,6 @@ @Function public class Paser2JsonFunction { - private static final Log LOG = LogFactory.getLog(Paser2JsonFunction.class); - /** * 如果原始数据是json,且希望能够自动展开成单层,可以调用这个方法,无论嵌套了几层json或jsonarray都会被展开 * @@ -41,8 +37,8 @@ public class Paser2JsonFunction { * @param context */ @FunctionMethod(value = "spread_json", alias = "autoJson", comment = "原始数据是嵌套json或jsonArray调用此方法会自动展开成单层") - public void spread2Json(IMessage message, FunctionContext context,String fieldName) { - fieldName= FunctionUtils.getConstant(fieldName); + public void spread2Json(IMessage message, FunctionContext context, String fieldName) { + fieldName = FunctionUtils.getConstant(fieldName); JSONObject jsonObject = message.getMessageBody().getJSONObject(fieldName); // if (message.isJsonMessage()) { @@ -152,7 +148,7 @@ protected JSONObject spreadJson(JSONObject jsonObject, List jsonArrayFie jsonArrayFieldNames.add(key); } } else if (JSONObject.class.isInstance(object)) { - JSONObject value = (JSONObject)object; + JSONObject value = (JSONObject) object; value = spreadJson(value, jsonArrayFieldNames); hasJson = true; result.putAll(value); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/PaserBySplitFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/PaserBySplitFunction.java index 219e6f2e..ce82bb1a 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/PaserBySplitFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/PaserBySplitFunction.java @@ -19,8 +19,6 @@ import com.alibaba.fastjson.JSONObject; import java.util.HashMap; import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.utils.LogParserUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; @@ -33,14 +31,30 @@ @Function public class PaserBySplitFunction { - private static final Log LOG = LogFactory.getLog(PaserBySplitFunction.class); private static final String CONST_MAP_KEY = "_const_flags";//存放常量替换的map + /** + * 按分隔符分割日志,根据名字做命名 + * + * @param message + * @param context + * @param log + * @param fieldName + * @param sign + * @param flags + * @param names + * @return + */ + private static Map signs = new HashMap(); + + static { + signs.put("|", "\\|"); + } @FunctionMethod(value = "paserByComma", comment = "根据英文逗号分割字符串") public JSONObject paserByComma(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名") String fieldName) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名") String fieldName) { String log = FunctionUtils.getValueString(message, context, fieldName); - Map flags = (Map)context.get(CONST_MAP_KEY); + Map flags = (Map) context.get(CONST_MAP_KEY); if (flags == null) { flags = new HashMap<>(); } @@ -57,11 +71,11 @@ public JSONObject paserByComma(IMessage message, FunctionContext context, */ @FunctionMethod(value = "paserBySign", comment = "根据char分割字符串,其中char通过ascii码转换过来,常用于使用不可见字符做分割") public JSONObject paserByAsciiSplit(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名") String fieldName, - @FunctionParamter(value = "string", comment = "代表分割符") String asciiDec) { - char splitSign = (char)Integer.parseInt(asciiDec); + @FunctionParamter(value = "string", comment = "代表字符串的字段名") String fieldName, + @FunctionParamter(value = "string", comment = "代表分割符") String asciiDec) { + char splitSign = (char) Integer.parseInt(asciiDec); String log = FunctionUtils.getValueString(message, context, fieldName); - Map flags = (Map)context.get(CONST_MAP_KEY); + Map flags = (Map) context.get(CONST_MAP_KEY); if (flags == null) { flags = new HashMap<>(); } @@ -70,11 +84,11 @@ public JSONObject paserByAsciiSplit(IMessage message, FunctionContext context, @FunctionMethod(value = "split", alias = "paserBySplit", comment = "通过分割符来进行日志解析") public JSONObject parseBySign(IMessage message, FunctionContext context, - @FunctionParamter(value = "boolean", comment = "是否需要预先处理常量类型") boolean needConstants, - @FunctionParamter(value = "boolean", comment = "是否需要预先处理带括号的数据") boolean needBacket, - @FunctionParamter(value = "boolean", comment = "是否预先处理时间类型的数据") boolean needDate, - @FunctionParamter(value = "string", comment = "代表字符串的字段名") String fieldName, - @FunctionParamter(value = "string", comment = "代表分割符") String sign) { + @FunctionParamter(value = "boolean", comment = "是否需要预先处理常量类型") boolean needConstants, + @FunctionParamter(value = "boolean", comment = "是否需要预先处理带括号的数据") boolean needBacket, + @FunctionParamter(value = "boolean", comment = "是否预先处理时间类型的数据") boolean needDate, + @FunctionParamter(value = "string", comment = "代表字符串的字段名") String fieldName, + @FunctionParamter(value = "string", comment = "代表分割符") String sign) { Map flags = new HashMap<>(); fieldName = FunctionUtils.getValueString(message, context, fieldName); String log = FunctionUtils.getValueString(message, context, fieldName); @@ -93,35 +107,17 @@ public JSONObject parseBySign(IMessage message, FunctionContext context, @FunctionMethod(value = "split", alias = "paserBySplit", comment = "通过分割符来进行日志解析") public JSONObject parseBySign(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名") String fieldName, - @FunctionParamter(value = "string", comment = "代表分割符") String sign) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名") String fieldName, + @FunctionParamter(value = "string", comment = "代表分割符") String sign) { sign = FunctionUtils.getConstant(sign); - String log = FunctionUtils.getValueString(message,context,fieldName); - Map flags = (Map)context.get(CONST_MAP_KEY); + String log = FunctionUtils.getValueString(message, context, fieldName); + Map flags = (Map) context.get(CONST_MAP_KEY); if (flags == null) { flags = new HashMap<>(); } return parseBySplit(message, context, log, fieldName, sign, flags); } - /** - * 按分隔符分割日志,根据名字做命名 - * - * @param message - * @param context - * @param log - * @param fieldName - * @param sign - * @param flags - * @param names - * @return - */ - private static Map signs = new HashMap(); - - static { - signs.put("|", "\\|"); - } - /** * 通过分割符进行解析 * @@ -140,7 +136,7 @@ private JSONObject parseBySplit(IMessage message, FunctionContext context, Strin String[] values = log.split(sign); Map jsonObject = new HashMap(); for (int i = 0; i < values.length; i++) { - String name = FunctionType.UDTF.getName() +i; + String name = FunctionType.UDTF.getName() + i; String value = values[i]; String tmp = flags.get(value); if (StringUtil.isNotEmpty(tmp)) { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/RegexParserFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/RegexParserFunction.java index 18df0581..483d304a 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/RegexParserFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/RegexParserFunction.java @@ -19,41 +19,25 @@ import com.alibaba.fastjson.JSONObject; import java.util.regex.Matcher; import java.util.regex.Pattern; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; import org.apache.rocketmq.streams.script.annotation.FunctionParamter; import org.apache.rocketmq.streams.script.context.FunctionContext; import org.apache.rocketmq.streams.script.function.model.FunctionType; import org.apache.rocketmq.streams.script.utils.FunctionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @Function public class RegexParserFunction { - private static final Log LOG = LogFactory.getLog(RegexParserFunction.class); - - @FunctionMethod(value = "paserByRegex", comment = "通过正则解析实例日志") - public String paserByRegex(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名") String fieldName, - @FunctionParamter(value = "string", comment = "正则表达式") String regex) { - - String log = FunctionUtils.getValueString(message, context, fieldName); - regex=FunctionUtils.getConstant(regex); - JSONObject jsonObject = parseLog(regex, fieldName, log); - if (jsonObject == null) { - context.breakExecute(); - } - message.setMessageBody(jsonObject); - return jsonObject.toJSONString(); - } + private static final Logger LOGGER = LoggerFactory.getLogger(RegexParserFunction.class); /** * 解析实例日志 * - * @param regex 正则表达式 - * @param log 日志 + * @param regex 正则表达式 + * @param log 日志 * @return regex和解析的字段和内容的对应关系 */ public static JSONObject parseLog(String regex, String fieldName, String log) { @@ -61,14 +45,29 @@ public static JSONObject parseLog(String regex, String fieldName, String log) { Pattern pattern = Pattern.compile(regex); Matcher matcher = pattern.matcher(log); if (!matcher.matches()) { - LOG.error("parseLog error: log not match regex!" + regex + ":" + log); + LOGGER.error("parseLog error: log not match regex!" + regex + ":" + log); return null; } for (int i = 1; i <= matcher.groupCount(); i++) { - String name = FunctionType.UDTF.getName()+i; + String name = FunctionType.UDTF.getName() + i; result.put(name, matcher.group(i)); } return result; } + @FunctionMethod(value = "paserByRegex", comment = "通过正则解析实例日志") + public String paserByRegex(IMessage message, FunctionContext context, + @FunctionParamter(value = "string", comment = "代表字符串的字段名") String fieldName, + @FunctionParamter(value = "string", comment = "正则表达式") String regex) { + + String log = FunctionUtils.getValueString(message, context, fieldName); + regex = FunctionUtils.getConstant(regex); + JSONObject jsonObject = parseLog(regex, fieldName, log); + if (jsonObject == null) { + context.breakExecute(); + } + message.setMessageBody(jsonObject); + return jsonObject.toJSONString(); + } + } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/python/GroovyFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/python/GroovyFunction.java index 81730749..a8846636 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/python/GroovyFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/python/GroovyFunction.java @@ -34,7 +34,7 @@ public class GroovyFunction { @FunctionMethod(value = "groovy", alias = "groovy", comment = "执行一个指定名称的python脚本") public JSONObject doPython(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "python名称") String scriptValue) { + @FunctionParamter(value = "string", comment = "python名称") String scriptValue) { scriptValue = FunctionUtils.getValueString(message, context, scriptValue); GroovyScriptOperator operator = cache.get(scriptValue); if (operator == null) { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/python/PythonFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/python/PythonFunction.java index 49a38d51..ceeaf78b 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/python/PythonFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/python/PythonFunction.java @@ -17,9 +17,16 @@ package org.apache.rocketmq.streams.script.function.impl.python; import com.alibaba.fastjson.JSONObject; +import java.io.BufferedReader; +import java.io.InputStreamReader; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.rocketmq.streams.common.cache.softreference.ICache; import org.apache.rocketmq.streams.common.cache.softreference.impl.SoftReferenceCache; +import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.configuration.SystemContext; import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.utils.FileUtil; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; import org.apache.rocketmq.streams.script.annotation.FunctionParamter; @@ -29,11 +36,11 @@ @Function public class PythonFunction { + protected static AtomicInteger PROC_COUNT = new AtomicInteger(5); private ICache cache = new SoftReferenceCache<>(); - @FunctionMethod(value = "py", alias = "python", comment = "执行一个指定名称的python脚本") public JSONObject doPython(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "python名称") String scriptValue) { + @FunctionParamter(value = "string", comment = "python名称") String scriptValue) { scriptValue = FunctionUtils.getValueString(message, context, scriptValue); JPythonScriptOperator pythonScriptOperator = cache.get(scriptValue); if (pythonScriptOperator == null) { @@ -46,4 +53,54 @@ public JSONObject doPython(IMessage message, FunctionContext context, pythonScriptOperator.doMessage(message, context); return message.getMessageBody(); } + + @FunctionMethod(value = "py", alias = "python", comment = "执行一个指定名称的python脚本") + public Object pythonUDF(String fileName, String... args) { + + Process proc; + BufferedReader in = null; + + try { + + while (PROC_COUNT.decrementAndGet() < 0) { + synchronized (this) { + PROC_COUNT.incrementAndGet(); + this.notify(); + if (PROC_COUNT.get() > 0) { + break; + } else { + this.wait(); + } + } + } + proc = Runtime.getRuntime().exec("python " + createFilePath(fileName) + " " + MapKeyUtil.createKeyBySign(" ", args));// 执行py文件 + //用输入输出流来截取结果 + in = new BufferedReader(new InputStreamReader(proc.getInputStream())); + StringBuilder result = new StringBuilder(); + String line = null; + while ((line = in.readLine()) != null) { + result.append(line); + } + proc.waitFor(); + return result; + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } finally { + try { + in.close(); + synchronized (this) { + PROC_COUNT.incrementAndGet(); + this.wait(); + } + } catch (Exception e) { + e.printStackTrace(); + } + } + + } + + private String createFilePath(String fileName) { + return FileUtil.concatFilePath(SystemContext.getProperty(ComponentCreator.UDF_JAR_PATH), fileName); + } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/relation/AndFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/relation/AndFunction.java index dd02052e..a02c6ced 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/relation/AndFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/relation/AndFunction.java @@ -24,6 +24,16 @@ @Function public class AndFunction { + public static void main(String[] args) { + ScriptComponent scriptComponent = ScriptComponent.getInstance(); + JSONObject msg = new JSONObject(); + msg.put("name", "chris"); + msg.put("age", 18); + + scriptComponent.getService().executeScript(msg, "x=or(equals(name,'chris'),>(age,19),!(equals(age,1))))"); + System.out.println(msg); + } + @FunctionMethod(value = "and", alias = "&", comment = "支持内嵌函数") public Boolean and(Boolean... values) { if (values == null) { @@ -52,14 +62,4 @@ public Boolean or(Boolean... values) { return false; } - - public static void main(String[] args) { - ScriptComponent scriptComponent = ScriptComponent.getInstance(); - JSONObject msg = new JSONObject(); - msg.put("name", "chris"); - msg.put("age", 18); - - scriptComponent.getService().executeScript(msg, "x=or(equals(name,'chris'),>(age,19),!(equals(age,1))))"); - System.out.println(msg); - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/sql/SQLFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/sql/SQLFunction.java index fccfd962..7cb4c0d0 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/sql/SQLFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/sql/SQLFunction.java @@ -30,9 +30,9 @@ public class SQLFunction { @FunctionMethod(value = "inSQL", alias = "create_inSQL", comment = "把字段对应的数组转化成sql中的in变量") public String createInSQL(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String msgFieldName, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String columnName, - @FunctionParamter(value = "boolean", comment = "是否是String类型") boolean isString) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String msgFieldName, + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String columnName, + @FunctionParamter(value = "boolean", comment = "是否是String类型") boolean isString) { msgFieldName = FunctionUtils.getValueString(message, context, msgFieldName); columnName = FunctionUtils.getValueString(message, context, columnName); JSONArray jsonArray = message.getMessageBody().getJSONArray(msgFieldName); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/AsciiFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/AsciiFunction.java index 44cbe490..e7f5b246 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/AsciiFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/AsciiFunction.java @@ -37,13 +37,13 @@ public class AsciiFunction { */ @FunctionMethod(value = "ascii", comment = "返回字符串第一个字符的ascii码") public Integer ascii(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String str) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String str) { Integer result = null; String ori = FunctionUtils.getValueString(message, context, str); if (StringUtil.isEmpty(ori)) { return result; } - result = (int)ori.charAt(0); + result = (int) ori.charAt(0); return result; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/Base64Function.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/Base64Function.java index 93822102..162e3cc5 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/Base64Function.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/Base64Function.java @@ -41,7 +41,7 @@ public class Base64Function { */ @FunctionMethod(value = "tobase64", comment = "把字节转换成base64") public String tobase64(IMessage message, FunctionContext context, - @FunctionParamter(value = "array", comment = "字节数组") byte fieldName[]) { + @FunctionParamter(value = "array", comment = "字节数组") byte fieldName[]) { if (fieldName == null) { return null; } @@ -58,7 +58,7 @@ public String tobase64(IMessage message, FunctionContext context, */ @FunctionMethod(value = "frombase64", comment = "把代表字符串的字段或常量转换为字节数组") public byte[] frombase64(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段或常量") String fieldName) { + @FunctionParamter(value = "string", comment = "代表字符串的字段或常量") String fieldName) { byte[] bt = null; String ori = FunctionUtils.getValueString(message, context, fieldName); bt = decoder.decode(ori); @@ -75,7 +75,7 @@ public byte[] frombase64(IMessage message, FunctionContext context, */ @FunctionMethod(value = "decode16", comment = "把代表字符串的字段或常量转换为字节数组") public String decode16(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段或常量") String fieldName) { + @FunctionParamter(value = "string", comment = "代表字符串的字段或常量") String fieldName) { String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori)) { return null; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/BlinkStringFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/BlinkStringFunction.java index 80c6b6e5..e73d6cec 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/BlinkStringFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/BlinkStringFunction.java @@ -29,9 +29,9 @@ public class BlinkStringFunction { @FunctionMethod(value = "substring", alias = "substr", comment = "截取从index开始len长度的字符串,-1代表截取从index后的全部") public String doStrig(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名或常量") String filedName, - @FunctionParamter(value = "string", comment = "字段名,常量或数字") String startIndex, - @FunctionParamter(value = "string", comment = "字段名,常量或数字") String length) { + @FunctionParamter(value = "string", comment = "字段名或常量") String filedName, + @FunctionParamter(value = "string", comment = "字段名,常量或数字") String startIndex, + @FunctionParamter(value = "string", comment = "字段名,常量或数字") String length) { String value = FunctionUtils.getValueString(message, context, filedName); int index = FunctionUtils.getValueInteger(message, context, startIndex); if (StringUtil.isEmpty(length)) { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/CharMatchCountFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/CharMatchCountFunction.java index 46984cd7..c44cb50c 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/CharMatchCountFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/CharMatchCountFunction.java @@ -30,8 +30,8 @@ public class CharMatchCountFunction { @FunctionMethod(value = "charmatchcount", alias = "char_matchcount", comment = "用于计算str1中有多少个字符出现在str2中") public Integer charmatchcount(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名或常量") String str1, - @FunctionParamter(value = "string", comment = "字段名或常量") String str2) { + @FunctionParamter(value = "string", comment = "字段名或常量") String str1, + @FunctionParamter(value = "string", comment = "字段名或常量") String str2) { Integer number = 0; String str1Tem = FunctionUtils.getValueString(message, context, str1); String str2Tem = FunctionUtils.getValueString(message, context, str2); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ChrFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ChrFunction.java index 30efabd7..3b140f38 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ChrFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ChrFunction.java @@ -36,13 +36,13 @@ public class ChrFunction { */ @FunctionMethod(value = "chr", alias = "char", comment = "将给定ASCII码ascii转换成字符") public String chr(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名或常量") String ascii) { + @FunctionParamter(value = "string", comment = "字段名或常量") String ascii) { String number = null; int asciiTem = Integer.parseInt(FunctionUtils.getValueString(message, context, ascii)); if (ascii == null || asciiTem < 0 || asciiTem > 255) { return number; } - number = (char)asciiTem + ""; + number = (char) asciiTem + ""; return number; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/HashCodeFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/HashCodeFunction.java index 6d168523..e61e86e2 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/HashCodeFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/HashCodeFunction.java @@ -37,7 +37,7 @@ public class HashCodeFunction { */ @FunctionMethod(value = "hashcode", comment = "获取字符串的hashcode") public String hashcode(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名或常量") String fieldName) { + @FunctionParamter(value = "string", comment = "字段名或常量") String fieldName) { String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori)) { return null; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/InitCapFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/InitCapFunction.java index 49a864f5..128fbfb5 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/InitCapFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/InitCapFunction.java @@ -37,7 +37,7 @@ public class InitCapFunction { */ @FunctionMethod(value = "initcap", comment = "返回字符串,每个字转换器的第一个字母大写,其余为小写") public String initcap(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名或常量") String fieldName) { + @FunctionParamter(value = "string", comment = "字段名或常量") String fieldName) { String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori)) { return null; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/InstrFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/InstrFunction.java index a4859135..4e552ee3 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/InstrFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/InstrFunction.java @@ -28,8 +28,8 @@ public class InstrFunction { @FunctionMethod(value = "contains", comment = "返回字符串str2在str1中的位置") public boolean contains(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名或常量") String str1, - @FunctionParamter(value = "string", comment = "字段名或常量") String str2) { + @FunctionParamter(value = "string", comment = "字段名或常量") String str1, + @FunctionParamter(value = "string", comment = "字段名或常量") String str2) { Integer index = instr(message, context, str1, str2); return index > -1; } @@ -45,8 +45,8 @@ public boolean contains(IMessage message, FunctionContext context, */ @FunctionMethod(value = "instr", alias = "indexOf", comment = "返回字符串str2在str1中的位置") public Integer instr(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名或常量") String str1, - @FunctionParamter(value = "string", comment = "字段名或常量") String str2) { + @FunctionParamter(value = "string", comment = "字段名或常量") String str1, + @FunctionParamter(value = "string", comment = "字段名或常量") String str2) { Integer number = null; str1 = FunctionUtils.getValueString(message, context, str1); str2 = FunctionUtils.getValueString(message, context, str2); @@ -68,8 +68,8 @@ public Integer instr(IMessage message, FunctionContext context, */ @FunctionMethod(value = "blink_instr", alias = "blink_indexOf", comment = "返回字符串str2在str1中的位置") public Integer blinkInstr(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名或常量") String str1, - @FunctionParamter(value = "string", comment = "字段名或常量") String str2) { + @FunctionParamter(value = "string", comment = "字段名或常量") String str1, + @FunctionParamter(value = "string", comment = "字段名或常量") String str2) { int index = instr(message, context, str1, str2); return index + 1; } @@ -85,9 +85,9 @@ public Integer blinkInstr(IMessage message, FunctionContext context, */ @FunctionMethod(value = "instr", alias = "indexOf", comment = "返回字符串str2在str1中从index后的位置") public Integer instr(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名或常量") String str1, - @FunctionParamter(value = "string", comment = "字段名或常量") String str2, - @FunctionParamter(value = "string", comment = "起始位置,数字,字段名或常量") String positionStr) { + @FunctionParamter(value = "string", comment = "字段名或常量") String str1, + @FunctionParamter(value = "string", comment = "字段名或常量") String str2, + @FunctionParamter(value = "string", comment = "起始位置,数字,字段名或常量") String positionStr) { Integer number = null; str1 = FunctionUtils.getValueString(message, context, str1); str2 = FunctionUtils.getValueString(message, context, str2); @@ -110,10 +110,10 @@ public Integer instr(IMessage message, FunctionContext context, */ @FunctionMethod(value = "instr", alias = "indexOf", comment = "返回字符串str2在str1中从index后出现n次的位置") public Integer instr(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名或常量") String str1, - @FunctionParamter(value = "string", comment = "字段名或常量") String str2, - @FunctionParamter(value = "string", comment = "起始位置,数字,字段名或常量") String positionStr, - @FunctionParamter(value = "string", comment = "出现的次数,数字,字段名或常量") String appearanceStr) { + @FunctionParamter(value = "string", comment = "字段名或常量") String str1, + @FunctionParamter(value = "string", comment = "字段名或常量") String str2, + @FunctionParamter(value = "string", comment = "起始位置,数字,字段名或常量") String positionStr, + @FunctionParamter(value = "string", comment = "出现的次数,数字,字段名或常量") String appearanceStr) { Integer number = null; str1 = FunctionUtils.getValueString(message, context, str1); str2 = FunctionUtils.getValueString(message, context, str2); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/InstrtFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/InstrtFunction.java index 424c61a2..1af7e21c 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/InstrtFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/InstrtFunction.java @@ -27,14 +27,14 @@ @Function public class InstrtFunction { - @FunctionMethod(value = "contains", comment = "返回字符串str2在str1中的位置") public boolean contains(IMessage message, FunctionContext context, @FunctionParamter(value = "string", comment = "字段名或常量") String str1, - @FunctionParamter(value = "string", comment = "字段名或常量") String str2){ - Integer index=instr(message,context,str1,str2); - return index>-1; + @FunctionParamter(value = "string", comment = "字段名或常量") String str2) { + Integer index = instr(message, context, str1, str2); + return index > -1; } + /** * 返回字符串str2在str1中的位置 * @@ -60,6 +60,7 @@ public Integer instr(IMessage message, FunctionContext context, /** * blink sql中的位置是从1开始的,所以在原有的位置上加1 + * * @param message * @param context * @param str1 @@ -70,8 +71,8 @@ public Integer instr(IMessage message, FunctionContext context, public Integer blinkInstr(IMessage message, FunctionContext context, @FunctionParamter(value = "string", comment = "字段名或常量") String str1, @FunctionParamter(value = "string", comment = "字段名或常量") String str2) { - int index=instr(message,context,str1,str2); - return index+1; + int index = instr(message, context, str1, str2); + return index + 1; } /** diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/IsEncodingFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/IsEncodingFunction.java index 56cc4c29..8b7a575e 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/IsEncodingFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/IsEncodingFunction.java @@ -40,9 +40,9 @@ public class IsEncodingFunction { */ @FunctionMethod(value = "isencoding", alias = "is_encoding", comment = "编码判断") public Boolean isencoding(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String param, - @FunctionParamter(value = "string", comment = "代表输入编码格式的字段名或常量") String from, - @FunctionParamter(value = "string", comment = "代表输出编码格式的字段名或常量") String to) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String param, + @FunctionParamter(value = "string", comment = "代表输入编码格式的字段名或常量") String from, + @FunctionParamter(value = "string", comment = "代表输出编码格式的字段名或常量") String to) { String ori = FunctionUtils.getValueString(message, context, param); from = FunctionUtils.getValueString(message, context, from); to = FunctionUtils.getValueString(message, context, to); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/KeyValueFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/KeyValueFunction.java index 1d0c8634..4b1a59dc 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/KeyValueFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/KeyValueFunction.java @@ -41,10 +41,10 @@ public class KeyValueFunction { @FunctionMethod(value = "keyvalue", alias = "kv", comment = "将srcStr(源字符串)按split1分成“key-value”对,按split2将key-value对分开,返回“key”所对应的value") public String keyvalue(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名或常量") String str, - @FunctionParamter(value = "string", comment = "第一分隔符,字段名或常量") String split1, - @FunctionParamter(value = "string", comment = "第二分隔符,字段名或常量") String split2, - @FunctionParamter(value = "string", comment = "待返回value对应的key,字段名或常量") String key) { + @FunctionParamter(value = "string", comment = "字段名或常量") String str, + @FunctionParamter(value = "string", comment = "第一分隔符,字段名或常量") String split1, + @FunctionParamter(value = "string", comment = "第二分隔符,字段名或常量") String split2, + @FunctionParamter(value = "string", comment = "待返回value对应的key,字段名或常量") String key) { JSONObject jsonObject = strtomap(message, context, str, split1, split2); return jsonObject.getString(key); } @@ -61,8 +61,8 @@ public String keyvalue(IMessage message, FunctionContext context, @FunctionMethod(value = "keyvalue", alias = "kv", comment = "将srcStr(源字符串)按;" + "分成“key-value”对,按:将key-value对分开,返回“key”所对应的value") public String keyvalue(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名或常量") String str, - @FunctionParamter(value = "string", comment = "待返回value对应的key,字段名或常量") String key) { + @FunctionParamter(value = "string", comment = "字段名或常量") String str, + @FunctionParamter(value = "string", comment = "待返回value对应的key,字段名或常量") String key) { return keyvalue(message, context, str, ";", ":", key); } @@ -76,7 +76,7 @@ public String keyvalue(IMessage message, FunctionContext context, */ @FunctionMethod(value = "jsoncreate", alias = "str2json", comment = "解析'name'='yuanxiaodong','age'=18的字符串为json") public JSONObject strtomap(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "格式为'name'='yuanxiaodong','age'=18的字符串,支持字段名和常量") String str) { + @FunctionParamter(value = "string", comment = "格式为'name'='yuanxiaodong','age'=18的字符串,支持字段名和常量") String str) { return strtomap(message, context, str, ",", "="); } @@ -92,9 +92,9 @@ public JSONObject strtomap(IMessage message, FunctionContext context, */ @FunctionMethod(value = "strtomap", alias = "str2json", comment = "解析'name'='yuanxiaodong','age'=18的字符串为json") public JSONObject strtomap(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "格式为'name'='yuanxiaodong','age'=18的字符串,支持字段名和常量") String str, - @FunctionParamter(value = "string", comment = "第一分隔符,字段名或常量") String split1, - @FunctionParamter(value = "string", comment = "第二分隔符,字段名或常量") String split2) { + @FunctionParamter(value = "string", comment = "格式为'name'='yuanxiaodong','age'=18的字符串,支持字段名和常量") String str, + @FunctionParamter(value = "string", comment = "第一分隔符,字段名或常量") String split1, + @FunctionParamter(value = "string", comment = "第二分隔符,字段名或常量") String split2) { JSONObject result = null; str = FunctionUtils.getValueString(message, context, str); split1 = FunctionUtils.getValueString(message, context, split1); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/LTrimFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/LTrimFunction.java index d82ad3bf..a374f6b3 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/LTrimFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/LTrimFunction.java @@ -37,7 +37,7 @@ public class LTrimFunction { */ @FunctionMethod(value = "ltrim", comment = "去除左侧空格") public String ltrim(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名或常量") String fieldName) { + @FunctionParamter(value = "string", comment = "字段名或常量") String fieldName) { String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori)) { return null; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/LengthBFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/LengthBFunction.java index 67f3d67c..a2bd9b4a 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/LengthBFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/LengthBFunction.java @@ -36,13 +36,13 @@ public class LengthBFunction { */ @FunctionMethod(value = "lengthb", alias = "lenb", comment = "字符串对应的字节长度") public Long lenghtb(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名或常量") String param) { + @FunctionParamter(value = "string", comment = "字段名或常量") String param) { Long len = null; param = FunctionUtils.getValueString(message, context, param); if (param == null) { return len; } - len = (long)param.getBytes().length; + len = (long) param.getBytes().length; return len; } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/LengthFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/LengthFunction.java index af300421..6574ff93 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/LengthFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/LengthFunction.java @@ -36,13 +36,13 @@ public class LengthFunction { */ @FunctionMethod(value = "length", alias = "len,CHAR_LENGTH", comment = "字符串长度") public Long lenght(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名或常量") String param) { + @FunctionParamter(value = "string", comment = "字段名或常量") String param) { Long len = null; param = FunctionUtils.getValueString(message, context, param); if (param == null) { return len; } - len = (long)param.length(); + len = (long) param.length(); return len; } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/Md5Function.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/Md5Function.java index 3619c27d..afd6ac05 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/Md5Function.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/Md5Function.java @@ -27,6 +27,10 @@ @Function public class Md5Function { + public static void main(String[] args) { + System.out.println(new Md5Function().MD5("password")); + } + /** * 字符串MD5加密 * @@ -37,7 +41,7 @@ public class Md5Function { */ @FunctionMethod(value = "md5", comment = "计算字符串的md5") public String concat(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名或常量") String param) { + @FunctionParamter(value = "string", comment = "字段名或常量") String param) { String result = ""; param = FunctionUtils.getValueString(message, context, param); if (param == null) { @@ -74,8 +78,4 @@ private String MD5(String s) { } } - public static void main(String[] args) { - System.out.println(new Md5Function().MD5("password")); - } - } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ParseUrlFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ParseUrlFunction.java index d87aefb7..18a2739d 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ParseUrlFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ParseUrlFunction.java @@ -32,9 +32,9 @@ public class ParseUrlFunction { @FunctionMethod(value = "parse_url", alias = "url", comment = "对url的解析,按key提取信息") public String parseurl(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "url") String urlStr, - @FunctionParamter(value = "string", comment = "支持HOST, PATH, QUERY, REF, PROTOCOL, AUTHORITY, FILE, 和 " - + "USERINFO,不区分大小写,支持字段,常量") String part) throws MalformedURLException { + @FunctionParamter(value = "string", comment = "url") String urlStr, + @FunctionParamter(value = "string", comment = "支持HOST, PATH, QUERY, REF, PROTOCOL, AUTHORITY, FILE, 和 " + + "USERINFO,不区分大小写,支持字段,常量") String part) throws MalformedURLException { return parseurl(message, context, urlStr, part, null); } @@ -50,11 +50,11 @@ public String parseurl(IMessage message, FunctionContext context, */ @FunctionMethod(value = "parse_url", alias = "url", comment = "对url的解析,按key提取信息") public String parseurl(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "url") String urlStr, - @FunctionParamter(value = "string", comment = "支持HOST, PATH, QUERY, REF, PROTOCOL, AUTHORITY, FILE, 和 " - + "USERINFO,不区分大小写,支持字段,常量") String part, - @FunctionParamter(value = "string", comment = "当part为QUERY时根据key的值取出在query string中的value值,否则忽略key参数,支持字段,常量") - String key) throws MalformedURLException { + @FunctionParamter(value = "string", comment = "url") String urlStr, + @FunctionParamter(value = "string", comment = "支持HOST, PATH, QUERY, REF, PROTOCOL, AUTHORITY, FILE, 和 " + + "USERINFO,不区分大小写,支持字段,常量") String part, + @FunctionParamter(value = "string", comment = "当part为QUERY时根据key的值取出在query string中的value值,否则忽略key参数,支持字段,常量") + String key) throws MalformedURLException { if (StringUtil.isEmpty(urlStr) || StringUtil.isEmpty(part)) { return null; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RTrimFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RTrimFunction.java index 32f9f4bd..d213b4a3 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RTrimFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RTrimFunction.java @@ -37,7 +37,7 @@ public class RTrimFunction { */ @FunctionMethod(value = "rtrim", comment = "去除右测空格") public String rtrim(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段名或常量") String fieldName) { + @FunctionParamter(value = "string", comment = "字段名或常量") String fieldName) { String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori)) { return null; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexCountFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexCountFunction.java index 4b29003a..58b4e44e 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexCountFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexCountFunction.java @@ -40,8 +40,8 @@ public class RegexCountFunction { */ @FunctionMethod(value = "regexcount", comment = "计算source中从start_position开始,匹配指定模式pattern的子串的次数") public Long regexcount(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表要匹配的列字段或常量值") String fieldName, - @FunctionParamter(value = "string", comment = "正则表达式代表类字段或常量值") String pattern) { + @FunctionParamter(value = "string", comment = "代表要匹配的列字段或常量值") String fieldName, + @FunctionParamter(value = "string", comment = "正则表达式代表类字段或常量值") String pattern) { Long index = null; String ori = FunctionUtils.getValueString(message, context, fieldName); pattern = FunctionUtils.getValueString(message, context, pattern); @@ -79,9 +79,9 @@ public Long regexcount(IMessage message, FunctionContext context, */ @FunctionMethod(value = "regexcount", comment = "计算source中从start_position开始,匹配指定模式pattern的子串的次数") public Long regexcount(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表要匹配的列字段或常量值") String fieldName, - @FunctionParamter(value = "string", comment = "正则表达式代表类字段或常量值") String pattern, - @FunctionParamter(value = "string", comment = "匹配开始的位置") Long position) { + @FunctionParamter(value = "string", comment = "代表要匹配的列字段或常量值") String fieldName, + @FunctionParamter(value = "string", comment = "正则表达式代表类字段或常量值") String pattern, + @FunctionParamter(value = "string", comment = "匹配开始的位置") Long position) { Long index = null; String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori) || StringUtil.isEmpty(pattern)) { @@ -104,7 +104,7 @@ public Long regexcount(IMessage message, FunctionContext context, break; } } - index = (long)startIndex; + index = (long) startIndex; return index; } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexFunction.java index dce7bffe..ff219d46 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexFunction.java @@ -58,44 +58,44 @@ public static boolean isNotRegexFunction(String functionName) { @FunctionMethod(value = "dipperregex", alias = "reg", comment = "通过正则表达式匹配字符串") public boolean match(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表要匹配的列字段或常量值") String fieldName, - @FunctionParamter(value = "string", comment = "正则表达式代表列字段或常量值") String pattern) { + @FunctionParamter(value = "string", comment = "代表要匹配的列字段或常量值") String fieldName, + @FunctionParamter(value = "string", comment = "正则表达式代表列字段或常量值") String pattern) { pattern = FunctionUtils.getConstant(pattern); String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori) || StringUtil.isEmpty(pattern)) { return false; } - boolean isMatch=StringUtil.matchRegex(ori, pattern); + boolean isMatch = StringUtil.matchRegex(ori, pattern); return isMatch; } @FunctionMethod(value = "regex", comment = "通过正则表达式匹配字符串") public boolean matchRegex(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表要匹配的列字段或常量值") String fieldName, - @FunctionParamter(value = "string", comment = "正则表达式代表列字段或常量值") String pattern) { + @FunctionParamter(value = "string", comment = "代表要匹配的列字段或常量值") String fieldName, + @FunctionParamter(value = "string", comment = "正则表达式代表列字段或常量值") String pattern) { return match(message, context, fieldName, pattern); } @FunctionMethod(value = "!regex", comment = "通过正则表达式匹配字符串") public boolean notMatchRegex(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表要匹配的列字段或常量值") String fieldName, - @FunctionParamter(value = "string", comment = "正则表达式代表列字段或常量值") String pattern) { + @FunctionParamter(value = "string", comment = "代表要匹配的列字段或常量值") String fieldName, + @FunctionParamter(value = "string", comment = "正则表达式代表列字段或常量值") String pattern) { return !match(message, context, fieldName, pattern); } @FunctionMethod(value = "regexGroup", alias = "regex_group", comment = "得到第一组匹配") public String regexGroup(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表要匹配的列字段或常量值") String fieldName, - @FunctionParamter(value = "string", comment = "正则表达式代表列字段或常量值") String regex) { + @FunctionParamter(value = "string", comment = "代表要匹配的列字段或常量值") String fieldName, + @FunctionParamter(value = "string", comment = "正则表达式代表列字段或常量值") String regex) { return regexGroupIndex(message, context, fieldName, regex, "1"); } @FunctionMethod(value = "regexGroupIndex", alias = "regex_group_index,REGEXP_EXTRACT", comment = "得到指定组匹配") public String regexGroupIndex(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表要匹配的列字段或常量值") String fieldName, - @FunctionParamter(value = "string", comment = "正则表达式代表列字段或常量值") String pattern, - @FunctionParamter(value = "string", comment = "指定分组数") String group) { + @FunctionParamter(value = "string", comment = "代表要匹配的列字段或常量值") String fieldName, + @FunctionParamter(value = "string", comment = "正则表达式代表列字段或常量值") String pattern, + @FunctionParamter(value = "string", comment = "指定分组数") String group) { if (StringUtil.isEmpty(fieldName) || StringUtil.isEmpty(pattern)) { return null; @@ -127,18 +127,18 @@ public String regexReplace(IMessage message, FunctionContext context, String fie @FunctionMethod(value = "extra", comment = "指定第一组匹作为配扩展字段值") public String extracteField(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "新的字段值") String newFieldName, - @FunctionParamter(value = "string", comment = "代表要匹配的列字段或常量值") String fieldName, - @FunctionParamter(value = "string", comment = "正则表达式代表列字段或常量值") String regex) { + @FunctionParamter(value = "string", comment = "新的字段值") String newFieldName, + @FunctionParamter(value = "string", comment = "代表要匹配的列字段或常量值") String fieldName, + @FunctionParamter(value = "string", comment = "正则表达式代表列字段或常量值") String regex) { return extraFieldByGroup(message, context, newFieldName, fieldName, regex, "1"); } @FunctionMethod(value = "extraByGroup", comment = "指定第几组匹作为配扩展字段值") public String extraFieldByGroup(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "新的字段值") String newFieldName, - @FunctionParamter(value = "string", comment = "代表要匹配的列字段或常量值") String fieldName, - @FunctionParamter(value = "string", comment = "正则表达式代表列字段或常量值") String regex, - @FunctionParamter(value = "string", comment = "指定分组数") String group) { + @FunctionParamter(value = "string", comment = "新的字段值") String newFieldName, + @FunctionParamter(value = "string", comment = "代表要匹配的列字段或常量值") String fieldName, + @FunctionParamter(value = "string", comment = "正则表达式代表列字段或常量值") String regex, + @FunctionParamter(value = "string", comment = "指定分组数") String group) { if (StringUtil.isEmpty(fieldName) || StringUtil.isEmpty(newFieldName) || StringUtil.isEmpty(regex)) { return null; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexInstrFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexInstrFunction.java index b70d3a6c..92d24cca 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexInstrFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexInstrFunction.java @@ -29,6 +29,18 @@ @Function public class RegexInstrFunction { + public static void main(String[] args) { + String ori = "i love www.taobao.com"; + String pattern = "o[a-zA-Z]{1}"; + String strTem = ""; + Pattern r = Pattern.compile(pattern); + Matcher m = r.matcher(ori); + if (m.find()) { + strTem = m.group(1); + } + System.out.println(strTem); + } + /** * 获取正则匹配的位置信息 * @@ -40,8 +52,8 @@ public class RegexInstrFunction { */ @FunctionMethod(value = "regexinstr", comment = "获取正则匹配的信息所在的位置") public Long regexinstr(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表要匹配的列名称或常量值") String fieldName, - @FunctionParamter(value = "string", comment = "正则表达式") String pattern) { + @FunctionParamter(value = "string", comment = "代表要匹配的列名称或常量值") String fieldName, + @FunctionParamter(value = "string", comment = "正则表达式") String pattern) { Long index = null; String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori) || StringUtil.isEmpty(pattern)) { @@ -53,7 +65,7 @@ public Long regexinstr(IMessage message, FunctionContext context, if (m.find()) { strTem = m.group(1); } - index = (long)ori.indexOf(strTem); + index = (long) ori.indexOf(strTem); return index; } @@ -68,9 +80,9 @@ public Long regexinstr(IMessage message, FunctionContext context, */ @FunctionMethod(value = "regexinstr", comment = "从指定位置获取正则匹配的信息所在的位置") public Long regexinstr(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表要匹配的列名称或常量值") String fieldName, - @FunctionParamter(value = "string", comment = "正则表达式") String pattern, - @FunctionParamter(value = "long", comment = "指定的位置") Long position) { + @FunctionParamter(value = "string", comment = "代表要匹配的列名称或常量值") String fieldName, + @FunctionParamter(value = "string", comment = "正则表达式") String pattern, + @FunctionParamter(value = "long", comment = "指定的位置") Long position) { Long index = null; String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori) || StringUtil.isEmpty(pattern)) { @@ -82,7 +94,7 @@ public Long regexinstr(IMessage message, FunctionContext context, if (m.find()) { strTem = m.group(1); } - index = (long)ori.indexOf(strTem, position.intValue()); + index = (long) ori.indexOf(strTem, position.intValue()); return index; } @@ -97,10 +109,10 @@ public Long regexinstr(IMessage message, FunctionContext context, */ @FunctionMethod(value = "regexinstr", comment = "从指定位置获取正则匹配的信息指定出现的次数所在的位置") public Long regexinstr(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表要匹配的列名称或常量值") String fieldName, - @FunctionParamter(value = "string", comment = "正则表达式") String pattern, - @FunctionParamter(value = "long", comment = "指定的位置") Long position, - @FunctionParamter(value = "long", comment = "指定正则出现的次数") Long occurrence) { + @FunctionParamter(value = "string", comment = "代表要匹配的列名称或常量值") String fieldName, + @FunctionParamter(value = "string", comment = "正则表达式") String pattern, + @FunctionParamter(value = "long", comment = "指定的位置") Long position, + @FunctionParamter(value = "long", comment = "指定正则出现的次数") Long occurrence) { Long index = null; String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori) || StringUtil.isEmpty(pattern)) { @@ -114,7 +126,7 @@ public Long regexinstr(IMessage message, FunctionContext context, } int i = 1; while (i <= occurrence) { - index = (long)ori.indexOf(strTem, position.intValue() + 1); + index = (long) ori.indexOf(strTem, position.intValue() + 1); position = index; i++; } @@ -132,11 +144,11 @@ public Long regexinstr(IMessage message, FunctionContext context, */ @FunctionMethod(value = "regexinstr", comment = "从指定位置获取正则匹配的信息指定出现的次数所在的位置") public Long regexinstr(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表要匹配的列名称或常量值") String fieldName, - @FunctionParamter(value = "string", comment = "正则表达式") String pattern, - @FunctionParamter(value = "long", comment = "指定的位置") Long position, - @FunctionParamter(value = "long", comment = "指定正则出现的次数") Long occurrence, - @FunctionParamter(value = "long", comment = "0表示返回匹配的开始位置,1表示返回匹配的结束位置") Long returnOption) { + @FunctionParamter(value = "string", comment = "代表要匹配的列名称或常量值") String fieldName, + @FunctionParamter(value = "string", comment = "正则表达式") String pattern, + @FunctionParamter(value = "long", comment = "指定的位置") Long position, + @FunctionParamter(value = "long", comment = "指定正则出现的次数") Long occurrence, + @FunctionParamter(value = "long", comment = "0表示返回匹配的开始位置,1表示返回匹配的结束位置") Long returnOption) { Long index = null; String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori) || StringUtil.isEmpty(pattern)) { @@ -150,7 +162,7 @@ public Long regexinstr(IMessage message, FunctionContext context, } int i = 1; while (i <= occurrence) { - index = (long)ori.indexOf(strTem, position.intValue() + 1); + index = (long) ori.indexOf(strTem, position.intValue() + 1); position = index; i++; } @@ -162,16 +174,4 @@ public Long regexinstr(IMessage message, FunctionContext context, } - public static void main(String[] args) { - String ori = "i love www.taobao.com"; - String pattern = "o[a-zA-Z]{1}"; - String strTem = ""; - Pattern r = Pattern.compile(pattern); - Matcher m = r.matcher(ori); - if (m.find()) { - strTem = m.group(1); - } - System.out.println(strTem); - } - } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexReplaceFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexReplaceFunction.java index 7fd1716c..4015fc08 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexReplaceFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexReplaceFunction.java @@ -40,9 +40,9 @@ public class RegexReplaceFunction { */ @FunctionMethod(value = "regexreplace", comment = "根据正则匹配替换字段信息") public String regexreplace(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "被替换的字段代表列字段或常量值") String fieldName, - @FunctionParamter(value = "string", comment = "要替换的字段代表列字段或常量值") String replaceStr, - @FunctionParamter(value = "string", comment = "正则表达式") String pattern) { + @FunctionParamter(value = "string", comment = "被替换的字段代表列字段或常量值") String fieldName, + @FunctionParamter(value = "string", comment = "要替换的字段代表列字段或常量值") String replaceStr, + @FunctionParamter(value = "string", comment = "正则表达式") String pattern) { String index = null; String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori) || StringUtil.isEmpty(pattern) || StringUtil.isEmpty(replaceStr)) { @@ -65,10 +65,10 @@ public String regexreplace(IMessage message, FunctionContext context, */ @FunctionMethod(value = "regexreplace", comment = "根据正则匹配替换字段信息") public String regexreplace(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "被替换的字段代表列字段或常量值") String fieldName, - @FunctionParamter(value = "string", comment = "要替换的字段代表列字段或常量值") String replaceStr, - @FunctionParamter(value = "string", comment = "正则表达式") String pattern, - @FunctionParamter(value = "long", comment = "正则匹配第几次数") Long occurrence) { + @FunctionParamter(value = "string", comment = "被替换的字段代表列字段或常量值") String fieldName, + @FunctionParamter(value = "string", comment = "要替换的字段代表列字段或常量值") String replaceStr, + @FunctionParamter(value = "string", comment = "正则表达式") String pattern, + @FunctionParamter(value = "long", comment = "正则匹配第几次数") Long occurrence) { String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori) || StringUtil.isEmpty(pattern)) { return NULL; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexSubStrFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexSubStrFunction.java index 5906a905..fcec8090 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexSubStrFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RegexSubStrFunction.java @@ -40,8 +40,8 @@ public class RegexSubStrFunction { */ @FunctionMethod(value = "regexsubstr", comment = "从start_position位置开始,source中第nth_occurrence次匹配指定模式pattern的子串") public String regexsubstr(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表要处理的列名称或常量值") String fieldName, - @FunctionParamter(value = "string", comment = "代表正则字段信息") String pattern) { + @FunctionParamter(value = "string", comment = "代表要处理的列名称或常量值") String fieldName, + @FunctionParamter(value = "string", comment = "代表正则字段信息") String pattern) { String index = null; String ori = FunctionUtils.getValueString(message, context, fieldName); pattern = FunctionUtils.getValueString(message, context, pattern); @@ -68,9 +68,9 @@ public String regexsubstr(IMessage message, FunctionContext context, */ @FunctionMethod(value = "regexsubstr", comment = "获取正则匹配的位置信息") public String regexsubstr(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表要处理的列名称或常量值") String fieldName, - @FunctionParamter(value = "string", comment = "代表正则字段信息") String pattern, - @FunctionParamter(value = "string", comment = "代表开始匹配的位置") Long position) { + @FunctionParamter(value = "string", comment = "代表要处理的列名称或常量值") String fieldName, + @FunctionParamter(value = "string", comment = "代表正则字段信息") String pattern, + @FunctionParamter(value = "string", comment = "代表开始匹配的位置") Long position) { String index = null; String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori) || StringUtil.isEmpty(pattern)) { @@ -96,10 +96,10 @@ public String regexsubstr(IMessage message, FunctionContext context, */ @FunctionMethod(value = "regexsubstr", comment = "获取正则匹配的位置信息") public String regexsubstr(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表要处理的列名称或常量值") String fieldName, - @FunctionParamter(value = "string", comment = "代表正则字段信息") String pattern, - @FunctionParamter(value = "string", comment = "代表开始匹配的位置") Long position, - @FunctionParamter(value = "long", comment = "代表指定第几次出现") Long occurrence) { + @FunctionParamter(value = "string", comment = "代表要处理的列名称或常量值") String fieldName, + @FunctionParamter(value = "string", comment = "代表正则字段信息") String pattern, + @FunctionParamter(value = "string", comment = "代表开始匹配的位置") Long position, + @FunctionParamter(value = "long", comment = "代表指定第几次出现") Long occurrence) { Long index = null; String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori) || StringUtil.isEmpty(pattern)) { @@ -114,7 +114,7 @@ public String regexsubstr(IMessage message, FunctionContext context, if (m.find()) { strTem = m.group(1); } - index = (long)ori.indexOf(strTem, position.intValue() + 1); + index = (long) ori.indexOf(strTem, position.intValue() + 1); position = index; i++; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RepeatFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RepeatFunction.java index fe3913c2..7388acd8 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RepeatFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/RepeatFunction.java @@ -38,8 +38,8 @@ public class RepeatFunction { */ @FunctionMethod(value = "repeat", comment = "重复输出指定次数的字符串") public String repeat(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表要重复输出代表列字段或常量值") String str, - @FunctionParamter(value = "long", comment = "指定输出的次数") String countStr) throws Exception { + @FunctionParamter(value = "string", comment = "代表要重复输出代表列字段或常量值") String str, + @FunctionParamter(value = "long", comment = "指定输出的次数") String countStr) throws Exception { StringBuilder sb = new StringBuilder(); str = FunctionUtils.getValueString(message, context, str); Integer count = FunctionUtils.getValueInteger(message, context, countStr); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ReplaceFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ReplaceFunction.java index 20dcb6e4..2c826360 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ReplaceFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ReplaceFunction.java @@ -39,9 +39,9 @@ public class ReplaceFunction { */ @FunctionMethod(value = "replace", comment = "使用新字符串替换原字符串中与指定的字符串完全匹配的字符串") public String trim(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "原字符串代表列字段或常量名") String a, - @FunctionParamter(value = "string", comment = "指定被替换的字符串") String OLD, - @FunctionParamter(value = "string", comment = "用于替换的字符串") String NEW) { + @FunctionParamter(value = "string", comment = "原字符串代表列字段或常量名") String a, + @FunctionParamter(value = "string", comment = "指定被替换的字符串") String OLD, + @FunctionParamter(value = "string", comment = "用于替换的字符串") String NEW) { a = FunctionUtils.getValueString(message, context, a); OLD = FunctionUtils.getValueString(message, context, OLD); NEW = FunctionUtils.getValueString(message, context, NEW); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ReverseFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ReverseFunction.java index 27b507c6..36527335 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ReverseFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ReverseFunction.java @@ -37,7 +37,7 @@ public class ReverseFunction { */ @FunctionMethod(value = "reverse", comment = "字符串倒序输出") public String reverse(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "要处理的字段代表列名称或常量值") String fieldName) { + @FunctionParamter(value = "string", comment = "要处理的字段代表列名称或常量值") String fieldName) { String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori)) { return null; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SoundxFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SoundxFunction.java index 0331c5ac..71ae8ce3 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SoundxFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SoundxFunction.java @@ -37,7 +37,7 @@ public class SoundxFunction { */ @FunctionMethod(value = "soundx", comment = "将普通字符串转换成soundex字符串") public String soundx(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "转化的字段代表列字段或常量名") String fieldName) { + @FunctionParamter(value = "string", comment = "转化的字段代表列字段或常量名") String fieldName) { String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori)) { return null; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SpaceFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SpaceFunction.java index 76e54703..40b55beb 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SpaceFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SpaceFunction.java @@ -37,7 +37,7 @@ public class SpaceFunction { */ @FunctionMethod(value = "space", comment = "空格字符串函数,返回长度为n的字符串") public String space(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "空格数量") String countStr) throws Exception { + @FunctionParamter(value = "string", comment = "空格数量") String countStr) throws Exception { StringBuilder sb = new StringBuilder(); if (StringUtil.isEmpty(countStr.toString())) { return null; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SplitPartFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SplitPartFunction.java index bede8fe6..5d67ee5a 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SplitPartFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SplitPartFunction.java @@ -39,9 +39,9 @@ public class SplitPartFunction { */ @FunctionMethod(value = "splitpart", comment = "依照分隔符separator拆分字符串str,返回从第start部分到第end部分的子串(闭区间)") public String splitpart(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "待拆分的源字段代表列字段或常量值") String str, - @FunctionParamter(value = "string", comment = "拆分字段使用的字符") String separator, - @FunctionParamter(value = "string", comment = "返回拆分结果的开始位置") String startStr) { + @FunctionParamter(value = "string", comment = "待拆分的源字段代表列字段或常量值") String str, + @FunctionParamter(value = "string", comment = "拆分字段使用的字符") String separator, + @FunctionParamter(value = "string", comment = "返回拆分结果的开始位置") String startStr) { StringBuilder sb = new StringBuilder(); String ori = FunctionUtils.getValueString(message, context, str); separator = FunctionUtils.getValueString(message, context, separator); @@ -72,10 +72,10 @@ public String splitpart(IMessage message, FunctionContext context, */ @FunctionMethod(value = "splitpart", comment = "依照分隔符separator拆分字符串str,返回从第start部分到第end部分的子串(闭区间)") public String splitpart(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "待拆分的源字段代表列字段或常量值") String str, - @FunctionParamter(value = "string", comment = "拆分字段使用的字符") String separator, - @FunctionParamter(value = "string", comment = "返回拆分结果的开始位置") String startStr, - @FunctionParamter(value = "string", comment = "返回拆分结果的结束位置") String endStr) { + @FunctionParamter(value = "string", comment = "待拆分的源字段代表列字段或常量值") String str, + @FunctionParamter(value = "string", comment = "拆分字段使用的字符") String separator, + @FunctionParamter(value = "string", comment = "返回拆分结果的开始位置") String startStr, + @FunctionParamter(value = "string", comment = "返回拆分结果的结束位置") String endStr) { StringBuilder sb = new StringBuilder(); String ori = FunctionUtils.getValueString(message, context, str); separator = FunctionUtils.getValueString(message, context, separator); @@ -108,9 +108,9 @@ public String splitpart(IMessage message, FunctionContext context, */ @FunctionMethod(value = "splitindex", comment = "以sep作为分隔符,将字符串str分隔成若干段,取其中的第index段") public String splitindex(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "原字符串代表列名称或常量值") String str, - @FunctionParamter(value = "string", comment = "处理源字段使用的字符") String separator, - @FunctionParamter(value = "long", comment = "要返回值的索引") String indexStr) { + @FunctionParamter(value = "string", comment = "原字符串代表列名称或常量值") String str, + @FunctionParamter(value = "string", comment = "处理源字段使用的字符") String separator, + @FunctionParamter(value = "long", comment = "要返回值的索引") String indexStr) { String sb = null; String ori = FunctionUtils.getValueString(message, context, str); separator = FunctionUtils.getValueString(message, context, separator); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/String2Map.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/String2Map.java index 29c0ad32..582e2790 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/String2Map.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/String2Map.java @@ -28,21 +28,21 @@ @Function public class String2Map { - @FunctionMethod(value = "STR_TO_MAP",alias = "str_to_map", comment = "字符串解析成map") - public Map string2Map(IMessage message, FunctionContext context, + @FunctionMethod(value = "STR_TO_MAP", alias = "str_to_map", comment = "字符串解析成map") + public Map string2Map(IMessage message, FunctionContext context, @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String str) { String ori = FunctionUtils.getValueString(message, context, str); if (StringUtil.isEmpty(ori)) { return null; } - String[] values= ori.split(","); - Map result=new HashMap<>(); - for(String value:values){ - String[] kv=value.split("="); - if(kv.length!=2){ - throw new RuntimeException("can not parse, "+ori); + String[] values = ori.split(","); + Map result = new HashMap<>(); + for (String value : values) { + String[] kv = value.split("="); + if (kv.length != 2) { + throw new RuntimeException("can not parse, " + ori); } - result.put(kv[0],kv[1]); + result.put(kv[0], kv[1]); } return result; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SubStringIndexFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SubStringIndexFunction.java index 23c2ac17..5ce16711 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SubStringIndexFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SubStringIndexFunction.java @@ -16,8 +16,6 @@ */ package org.apache.rocketmq.streams.script.function.impl.string; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.script.annotation.Function; @@ -28,7 +26,6 @@ @Function public class SubStringIndexFunction { - private static final Log LOG = LogFactory.getLog(SubStringIndexFunction.class); /** * 截取字符串a第count分隔符之前的字符串,如count为正则从左边开始截取,如果为负则从右边开始截取 @@ -42,9 +39,9 @@ public class SubStringIndexFunction { */ @FunctionMethod(value = "substringindex", alias = "substr_index", comment = "截取字符串a第count分隔符之前的字符串,如count为正则从左边开始截取,如果为负则从右边开始截取") public String substringindex(IMessage message, FunctionContext context, - @FunctionParamter(comment = "带拆分的字符串代表字列名称或常量值", value = "string") String a, - @FunctionParamter(comment = "指定用于拆分原始字段的字符代表列名称或常量值", value = "string") String SEP, - @FunctionParamter(comment = "用于表示开始拆分的位置,正数表示从左边拆分,负数表示从右边拆分", value = "string") String countStr) { + @FunctionParamter(comment = "带拆分的字符串代表字列名称或常量值", value = "string") String a, + @FunctionParamter(comment = "指定用于拆分原始字段的字符代表列名称或常量值", value = "string") String SEP, + @FunctionParamter(comment = "用于表示开始拆分的位置,正数表示从左边拆分,负数表示从右边拆分", value = "string") String countStr) { String result = null; a = FunctionUtils.getValueString(message, context, a); SEP = FunctionUtils.getValueString(message, context, SEP); @@ -79,12 +76,11 @@ public String substringindex(IMessage message, FunctionContext context, return result; } - @FunctionMethod(value = "substr", alias = "substring", comment = "截取从指定的索引startIndex开始扩展到索引endIndex处的字符") public String substringindex(IMessage message, FunctionContext context, - @FunctionParamter(comment = "带拆分的字符串代表字列名称或常量值", value = "string") String oriMsg, - @FunctionParamter(comment = "指定用于拆分原始字段的字符代表列名称或常量值", value = "string") Integer startIndex, - @FunctionParamter(comment = "指定用于拆分原始字段的字符代表列名称或常量值", value = "string") Integer endIndex) { + @FunctionParamter(comment = "带拆分的字符串代表字列名称或常量值", value = "string") String oriMsg, + @FunctionParamter(comment = "指定用于拆分原始字段的字符代表列名称或常量值", value = "string") Integer startIndex, + @FunctionParamter(comment = "指定用于拆分原始字段的字符代表列名称或常量值", value = "string") Integer endIndex) { oriMsg = FunctionUtils.getValueString(message, context, oriMsg); int msgLength = oriMsg.length(); if (startIndex >= msgLength) { @@ -97,17 +93,17 @@ public String substringindex(IMessage message, FunctionContext context, @FunctionMethod(value = "substr", alias = "blink_substring", comment = "截取从指定的索引startIndex处开始扩展到此字符串的结尾") public String substringindex(IMessage message, FunctionContext context, - @FunctionParamter(comment = "带拆分的字符串代表字列名称或常量值", value = "string") String oriMsg, - @FunctionParamter(comment = "指定用于拆分原始字段的字符代表列名称或常量值", value = "string") Integer startIndex) { + @FunctionParamter(comment = "带拆分的字符串代表字列名称或常量值", value = "string") String oriMsg, + @FunctionParamter(comment = "指定用于拆分原始字段的字符代表列名称或常量值", value = "string") Integer startIndex) { oriMsg = FunctionUtils.getValueString(message, context, oriMsg); return oriMsg == null ? null : startIndex >= oriMsg.length() ? "" : oriMsg.substring(startIndex); } @FunctionMethod(value = "blink_substr", alias = "blink_substring", comment = "截取从指定的索引startIndex开始,长度为len的字符,index从1开始,需要做下处理") public String substringindexForBlink(IMessage message, FunctionContext context, - @FunctionParamter(comment = "带拆分的字符串代表字列名称或常量值", value = "string") String oriMsgField, - @FunctionParamter(comment = "指定用于拆分原始字段的字符代表列名称或常量值", value = "string") String startIndex, - @FunctionParamter(comment = "指定用于拆分原始字段的字符代表列名称或常量值", value = "string") String len) { + @FunctionParamter(comment = "带拆分的字符串代表字列名称或常量值", value = "string") String oriMsgField, + @FunctionParamter(comment = "指定用于拆分原始字段的字符代表列名称或常量值", value = "string") String startIndex, + @FunctionParamter(comment = "指定用于拆分原始字段的字符代表列名称或常量值", value = "string") String len) { String oriMsg = FunctionUtils.getValueString(message, context, oriMsgField); String index = FunctionUtils.getValueString(message, context, startIndex); String lengthStr = FunctionUtils.getValueString(message, context, len); @@ -124,10 +120,10 @@ public String substringindexForBlink(IMessage message, FunctionContext context, fromIndex = oriMsg.length() + fromIndex - length + 1; endIndex = fromIndex + length; } - if (oriMsg!=null&&endIndex >= oriMsg.length()) { + if (oriMsg != null && endIndex >= oriMsg.length()) { endIndex = oriMsg.length(); } - if(oriMsg==null){ + if (oriMsg == null) { return null; } return oriMsg.substring(fromIndex, endIndex); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ToCharFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ToCharFunction.java index 9a6d4e8f..2b32d765 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ToCharFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ToCharFunction.java @@ -38,7 +38,7 @@ public class ToCharFunction { */ @FunctionMethod(value = "tochar", alias = "tostring", comment = "转化为字符串") public String tochar(IMessage message, FunctionContext context, - @FunctionParamter(comment = "带转换的字符串代表列名称或常量值", value = "string") String fieldName) { + @FunctionParamter(comment = "带转换的字符串代表列名称或常量值", value = "string") String fieldName) { if (StringUtil.isEmpty(fieldName.toString())) { return null; } @@ -56,7 +56,7 @@ public String tochar(IMessage message, FunctionContext context, */ @FunctionMethod(value = "tochar", alias = "tostring", comment = "转化为字符串") public String tochar(IMessage message, FunctionContext context, - @FunctionParamter(comment = "带转换的字符串代表Boolean类型的常量值", value = "boolean") Boolean fieldName) { + @FunctionParamter(comment = "带转换的字符串代表Boolean类型的常量值", value = "boolean") Boolean fieldName) { if (StringUtil.isEmpty(fieldName.toString())) { return null; } @@ -73,7 +73,7 @@ public String tochar(IMessage message, FunctionContext context, */ @FunctionMethod(value = "tochar", alias = "tostring", comment = "转化为字符串") public String tochar(IMessage message, FunctionContext context, - @FunctionParamter(comment = "带转换的字符串代表Long类型的常量值", value = "long") Long fieldName) { + @FunctionParamter(comment = "带转换的字符串代表Long类型的常量值", value = "long") Long fieldName) { if (StringUtil.isEmpty(fieldName.toString())) { return null; } @@ -90,7 +90,7 @@ public String tochar(IMessage message, FunctionContext context, */ @FunctionMethod(value = "tochar", alias = "tostring", comment = "转化为字符串") public String tochar(IMessage message, FunctionContext context, - @FunctionParamter(comment = "带转换的字符串代表BigDecimal类型的常量值", value = "bigdecimal") BigDecimal fieldName) { + @FunctionParamter(comment = "带转换的字符串代表BigDecimal类型的常量值", value = "bigdecimal") BigDecimal fieldName) { if (StringUtil.isEmpty(fieldName.toString())) { return null; } @@ -107,7 +107,7 @@ public String tochar(IMessage message, FunctionContext context, */ @FunctionMethod(value = "tochar", alias = "tostring", comment = "转化为字符串") public String tochar(IMessage message, FunctionContext context, - @FunctionParamter(comment = "带转换的字符串代表double类型的常量值", value = "double") Double fieldName) { + @FunctionParamter(comment = "带转换的字符串代表double类型的常量值", value = "double") Double fieldName) { if (StringUtil.isEmpty(fieldName.toString())) { return null; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ToLowerFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ToLowerFunction.java index 3581f5ff..928c4dc9 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ToLowerFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ToLowerFunction.java @@ -27,11 +27,11 @@ @Function public class ToLowerFunction { - public static boolean isLowFunction(String functionName){ - if(functionName==null){ + public static boolean isLowFunction(String functionName) { + if (functionName == null) { return false; } - if("tolower".equals(functionName.toLowerCase())||"lower".equals(functionName.toLowerCase())){ + if ("tolower".equals(functionName.toLowerCase()) || "lower".equals(functionName.toLowerCase())) { return true; } return false; @@ -47,7 +47,7 @@ public static boolean isLowFunction(String functionName){ */ @FunctionMethod(value = "tolower", alias = "lower", comment = "将值转换为小写") public String tolower(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "待转换的字符串代表列名称或常量值") String fieldName) { + @FunctionParamter(value = "string", comment = "待转换的字符串代表列名称或常量值") String fieldName) { String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori)) { return null; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ToUpperFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ToUpperFunction.java index cd0d60cf..49a17a1a 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ToUpperFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/ToUpperFunction.java @@ -37,7 +37,7 @@ public class ToUpperFunction { */ @FunctionMethod(value = "toupper", alias = "upper", comment = "转换为大写") public String toupper(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "待转换的字符串代表列名称或常量值") String fieldName) { + @FunctionParamter(value = "string", comment = "待转换的字符串代表列名称或常量值") String fieldName) { String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori)) { return null; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/TransLateFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/TransLateFunction.java index 6c953738..3b639c23 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/TransLateFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/TransLateFunction.java @@ -40,9 +40,9 @@ public class TransLateFunction { @FunctionMethod(value = "translate", comment = "将原字符串中指定的字符串替换为新的字符串") public String translate(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "原字符串代表列名称或常量值") String str1, - @FunctionParamter(value = "string", comment = "指定的要替换的字符串代表列名称或常量值") String str2, - @FunctionParamter(value = "string", comment = "新的字符串代表列名称或常量值") String str3) { + @FunctionParamter(value = "string", comment = "原字符串代表列名称或常量值") String str1, + @FunctionParamter(value = "string", comment = "指定的要替换的字符串代表列名称或常量值") String str2, + @FunctionParamter(value = "string", comment = "新的字符串代表列名称或常量值") String str3) { str1 = FunctionUtils.getValueString(message, context, str1); str2 = FunctionUtils.getValueString(message, context, str2); str3 = FunctionUtils.getValueString(message, context, str3); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/TrimFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/TrimFunction.java index 9f29f5f7..5732f6f2 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/TrimFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/TrimFunction.java @@ -27,6 +27,16 @@ @Function public class TrimFunction { + public static boolean isTrimFunction(String functionName) { + if (functionName == null) { + return false; + } + if ("trim".equals(functionName.toLowerCase())) { + return true; + } + return false; + } + /** * 将输入字符串str去除左右空格 * @@ -37,7 +47,7 @@ public class TrimFunction { */ @FunctionMethod(value = "trim", comment = "将输入字符串str去除左右空格") public String trim(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "要去除空格的字符串") String fieldName) { + @FunctionParamter(value = "string", comment = "要去除空格的字符串") String fieldName) { String ori = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(ori)) { return null; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/UrlDecodeFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/UrlDecodeFunction.java index 6fa13dc9..e5e8a379 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/UrlDecodeFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/UrlDecodeFunction.java @@ -39,7 +39,7 @@ public class UrlDecodeFunction { */ @FunctionMethod(value = "urldecode", comment = "将输入字符串从application/x-www-form-urlencoded MIME格式转为正常字符串") public String urldecode(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "带编码的字符串代表列名称或常量值") String str) { + @FunctionParamter(value = "string", comment = "带编码的字符串代表列名称或常量值") String str) { String result = null; String ori = FunctionUtils.getValueString(message, context, str); if (StringUtil.isEmpty(ori)) { @@ -63,8 +63,8 @@ public String urldecode(IMessage message, FunctionContext context, */ @FunctionMethod(value = "urldecode", comment = "将输入字符串从application/x-www-form-urlencoded MIME格式根据指定的编码转为正常字符串") public String urldecode(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "带编码的字符串代表列名称或常量值") String fileName, - @FunctionParamter(value = "string", comment = "指定的编码格式") String encodeing) { + @FunctionParamter(value = "string", comment = "带编码的字符串代表列名称或常量值") String fileName, + @FunctionParamter(value = "string", comment = "指定的编码格式") String encodeing) { String result = null; fileName = FunctionUtils.getValueString(message, context, fileName); encodeing = FunctionUtils.getValueString(message, context, encodeing); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/UrlEncodeFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/UrlEncodeFunction.java index 7a063ff3..cac6e64a 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/UrlEncodeFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/UrlEncodeFunction.java @@ -29,6 +29,25 @@ @Function public class UrlEncodeFunction { + public static void main(String[] args) throws UnsupportedEncodingException { + StringBuilder result = new StringBuilder(); + String ss = "示例for url_encode:// (fdsf)"; + char temp[] = ss.toCharArray(); + for (char cha : temp) { + if (cha > 64 && cha < 91) { + result.append(cha); + } else if (cha > 96 && cha < 123) { + result.append(cha); + } else if (cha == 46 || cha == 45 || cha == 42 || cha == 95) { + result.append(cha); + } else { + result.append(URLEncoder.encode(cha + "", "UTF-8")); + } + } + System.out.println(result.toString()); + + } + /** * 将输入字符串编码为application/x-www-form-urlencoded MIME格式 * @@ -39,7 +58,7 @@ public class UrlEncodeFunction { */ @FunctionMethod(value = "urlencode", comment = "将输入字符串编码为application/x-www-form-urlencoded MIME格式") public String urlencode(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "带编码的字符串代表列名称或常量值") String str) { + @FunctionParamter(value = "string", comment = "带编码的字符串代表列名称或常量值") String str) { String ori = FunctionUtils.getValueString(message, context, str); StringBuilder result = new StringBuilder(); if (StringUtil.isEmpty(ori)) { @@ -74,8 +93,8 @@ public String urlencode(IMessage message, FunctionContext context, */ @FunctionMethod(value = "urlencode", comment = "将输入字符串根据指定的编码编码为application/x-www-form-urlencoded MIME格式") public String urlencode(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "带编码的字符串代表列名称或常量值") String str, - @FunctionParamter(value = "string", comment = "指定的编码") String encodeing) { + @FunctionParamter(value = "string", comment = "带编码的字符串代表列名称或常量值") String str, + @FunctionParamter(value = "string", comment = "指定的编码") String encodeing) { String ori = FunctionUtils.getValueString(message, context, str); StringBuilder result = new StringBuilder(); if (StringUtil.isEmpty(ori)) { @@ -100,23 +119,4 @@ public String urlencode(IMessage message, FunctionContext context, return result.toString(); } - public static void main(String[] args) throws UnsupportedEncodingException { - StringBuilder result = new StringBuilder(); - String ss = "示例for url_encode:// (fdsf)"; - char temp[] = ss.toCharArray(); - for (char cha : temp) { - if (cha > 64 && cha < 91) { - result.append(cha); - } else if (cha > 96 && cha < 123) { - result.append(cha); - } else if (cha == 46 || cha == 45 || cha == 42 || cha == 95) { - result.append(cha); - } else { - result.append(URLEncoder.encode(cha + "", "UTF-8")); - } - } - System.out.println(result.toString()); - - } - } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/type/CastFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/type/CastFunction.java index a8100982..28ced289 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/type/CastFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/type/CastFunction.java @@ -32,7 +32,7 @@ public class CastFunction { @Deprecated @FunctionMethod(value = "cast", comment = "转换字符串为某种类型") public Object cast(IMessage message, FunctionContext context, @FunctionParamter("代表带转换数据的字段名或常量") String fieldName, - @FunctionParamter(value = "string", comment = "转换后的类型,支持字读名或常量,小写的类型如:string,int,long") String dataTypeName) { + @FunctionParamter(value = "string", comment = "转换后的类型,支持字读名或常量,小写的类型如:string,int,long") String dataTypeName) { String value = FunctionUtils.getValueString(message, context, fieldName); if (value == null) { return null; @@ -48,8 +48,8 @@ public Object cast(IMessage message, FunctionContext context, @FunctionParamter( @Deprecated @FunctionMethod(value = "byte2String", comment = "转换字符串为某种类型") public String byteEncode(IMessage message, FunctionContext context, - @FunctionParamter("代表带转换数据的字段名或常量") String fieldName, - @FunctionParamter(value = "string", comment = "转换后的类型,支持字读名或常量,小写的类型如:string,int,long") String chartcode) { + @FunctionParamter("代表带转换数据的字段名或常量") String fieldName, + @FunctionParamter(value = "string", comment = "转换后的类型,支持字读名或常量,小写的类型如:string,int,long") String chartcode) { Object value = FunctionUtils.getValue(message, context, fieldName); if (FunctionUtils.isConstant(chartcode)) { chartcode = FunctionUtils.getConstant(chartcode); @@ -58,7 +58,7 @@ public String byteEncode(IMessage message, FunctionContext context, if (value == null || !byte[].class.isInstance(value)) { return null; } - byte[] data = (byte[])value; + byte[] data = (byte[]) value; String result = null; try { result = new String(data, chartcode); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/udtf/UDTFFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/udtf/UDTFFunction.java index 606f9542..6e9b0e06 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/udtf/UDTFFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/udtf/UDTFFunction.java @@ -37,7 +37,7 @@ public class UDTFFunction { */ @FunctionMethod(value = "renameudtf", alias = "udtfrename", comment = "把udtf拆分的列加上列名,生成新的message") public void udtf(IMessage message, FunctionContext context, - @FunctionParamter(value = "array", comment = "对应的字段名") String... fieldNames) { + @FunctionParamter(value = "array", comment = "对应的字段名") String... fieldNames) { JSONObject jsonObject = message.getMessageBody(); final JSONObject temp = new JSONObject(); temp.putAll(jsonObject); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/model/FunctionConfigure.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/model/FunctionConfigure.java index 0af46b5b..3269d106 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/model/FunctionConfigure.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/model/FunctionConfigure.java @@ -25,8 +25,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -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.IMessage; import org.apache.rocketmq.streams.common.datatype.DataType; @@ -35,24 +33,23 @@ import org.apache.rocketmq.streams.common.utils.CollectionUtil; import org.apache.rocketmq.streams.common.utils.DataTypeUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * 每一个注册的function会有一个engine来代表。引擎主要用于执行一个service的方法 */ public class FunctionConfigure { - private static final Log LOG = LogFactory.getLog(FunctionConfigure.class); - /** * 函数名称,可以把任务springbean的方法注册成function,并取一个functionname */ public static final String FUNCTION_NAME = "functionName"; - /** * 通过这个值从data json中获取需要执行方法的实际参数数据 */ public static final String FUNCTION_PARAMETERS = "paramters"; - + private static final Logger LOGGER = LoggerFactory.getLogger(FunctionConfigure.class); /** * 要执行的方法 */ @@ -149,6 +146,17 @@ public FunctionConfigure(String functionName, Method method, Object bean, Functi } } + /** + * 获得方法名 + * + * @param jsonConfigure + * @return + */ + public static String parseFunctionName(String jsonConfigure) { + JSONObject jsonObject = JSONObject.parseObject(jsonConfigure); + return jsonObject.getString(FUNCTION_NAME); + } + public FunctionType getFunctionType() { return functionType; } @@ -402,7 +410,7 @@ private Object[] parseParameters(String parameterConfigure, Object... dataParamt return parameters; } catch (Exception e) { - LOG.error("parseParameters error :" + parameterConfigure + ";detail info is " + e.getMessage(), e); + LOGGER.error("parseParameters error :" + parameterConfigure + ";detail info is " + e.getMessage(), e); return null; } @@ -433,7 +441,7 @@ public Object[] parseParameters(String parameterConfigure, int configureParamete } return parameters; } catch (Exception e) { - LOG.error("parseParameters error :" + parameterConfigure + ";detail info is " + e.getMessage(), e); + LOGGER.error("parseParameters error :" + parameterConfigure + ";detail info is " + e.getMessage(), e); return null; } @@ -467,17 +475,6 @@ public Object getBean() { return this.bean; } - /** - * 获得方法名 - * - * @param jsonConfigure - * @return - */ - public static String parseFunctionName(String jsonConfigure) { - JSONObject jsonObject = JSONObject.parseObject(jsonConfigure); - return jsonObject.getString(FUNCTION_NAME); - } - /** * 初始化参数列表,先放入数据参数,数据参数在列表的最前面 * @@ -535,7 +532,7 @@ public boolean startWith(Class[] classes) { } } } catch (Exception e) { - LOG.error("startWith执行异常,将返回true", e); + LOGGER.error("startWith执行异常,将返回true", e); } return true; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/model/FunctionInfo.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/model/FunctionInfo.java index f9a9142c..025af05a 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/model/FunctionInfo.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/model/FunctionInfo.java @@ -17,14 +17,11 @@ package org.apache.rocketmq.streams.script.function.model; import java.util.Objects; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; /** * 保存每个函数的参数和描述信息 主要用于展示 */ public class FunctionInfo { - private static final Log LOG = LogFactory.getLog(FunctionInfo.class); private String params;//参数列表和参数类型 private String comment;//函数的说明信息 private String returnType;//返回的类型 @@ -87,7 +84,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - FunctionInfo that = (FunctionInfo)o; + FunctionInfo that = (FunctionInfo) o; return Objects.equals(params, that.params) && Objects.equals(comment, that.comment) && Objects.equals( returnType, that.returnType) && Objects.equals(functionName, that.functionName) && Objects.equals(group, that.group); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/model/FunctionType.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/model/FunctionType.java index 189ca326..f76f2344 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/model/FunctionType.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/model/FunctionType.java @@ -18,7 +18,6 @@ public enum FunctionType { - /** * 1:1 */ @@ -44,13 +43,12 @@ public enum FunctionType { */ UNKNOWN("unknow"); private String name; - private FunctionType(String name){ - this.name=name; - } - + private FunctionType(String name) { + this.name = name; + } - public String getName(){ + public String getName() { return name; } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/service/IFunctionService.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/service/IFunctionService.java index 9db1ea3a..4493c1e6 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/service/IFunctionService.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/service/IFunctionService.java @@ -45,8 +45,8 @@ public interface IFunctionService { */ void registeFunction(String functionName, Object bean, Method method, FunctionType functionType); - void registeUserDefinedUDTFFunction(String functionName, Object bean, Method method); + /** * 给一个接口取名字,并注册。 可以通过名字获取接口,单不支持通过函数名调用 * @@ -91,8 +91,8 @@ FunctionConfigure getFunctionConfigure(IMessage message, FunctionContext context */ FunctionConfigure getFunctionConfigure(String functionName, Object... parameters); - DataType getReturnDataType(String functionName); + /** * 执行函数,或先找到合适的FunctionConfigure,然后执行反射。函数带message,context前缀 * diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/service/impl/DefaultFunctionServiceImpl.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/service/impl/DefaultFunctionServiceImpl.java index eb70bd1b..94f602f4 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/service/impl/DefaultFunctionServiceImpl.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/service/impl/DefaultFunctionServiceImpl.java @@ -23,8 +23,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.datatype.DataType; import org.apache.rocketmq.streams.common.utils.StringUtil; @@ -39,10 +37,12 @@ import org.apache.rocketmq.streams.script.function.service.IDipperInterfaceAdpater; import org.apache.rocketmq.streams.script.function.service.IFunctionService; import org.apache.rocketmq.streams.script.utils.FunctionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class DefaultFunctionServiceImpl implements IFunctionService { - private static final Log LOG = LogFactory.getLog(DefaultFunctionServiceImpl.class); + private static final Logger LOGGER = LoggerFactory.getLogger(DefaultFunctionServiceImpl.class); /** * 重要,函数注册中心。FunctionConfigureMap 保存多个同名的FunctionConfigure @@ -66,46 +66,26 @@ public class DefaultFunctionServiceImpl implements IFunctionService { @Deprecated protected Map functionParam2Engies = new IngoreInsensitiveMap(); - protected class IngoreInsensitiveMap extends HashMap { - - @Override - public V get(Object key) { - if (key == null) { - return null; - } - String keyString = (String)key; - keyString = keyString.toLowerCase(); - return super.get(keyString); - } - - @Override - public V put(String key, V value) { - if (key == null) { - return value; - } - String keyString = key.toLowerCase(); - return super.put(keyString, value); - } - } - public void registeFunction(String functionName, IDipperInterfaceAdpater dipperInterfaceAdpater) { this.functionName2Adapter.put(functionName, dipperInterfaceAdpater); } + @Override public void registeUserDefinedUDTFFunction(String functionName, Object bean, Method method) { - FunctionConfigure functionConfigure= registeFunctionInner(functionName,bean,method,FunctionType.UDTF); + FunctionConfigure functionConfigure = registeFunctionInner(functionName, bean, method, FunctionType.UDTF); functionConfigure.setUserDefinedUDTF(true); } + @Override public void registeFunction(String functionName, Object bean, Method method, FunctionType functionType) { - registeFunctionInner(functionName,bean,method,functionType); + registeFunctionInner(functionName, bean, method, functionType); } private FunctionConfigure registeFunctionInner(String functionName, Object bean, Method method, FunctionType functionType) { - // LOG.debug("FunctionServiceImpl registeFunction,functionName:" + functionName + " ,method:" + method); + // LOG.debug("FunctionServiceImpl registeFunction,functionName:" + functionName + " ,method:" + method); try { if (IDipperInterfaceAdpater.class.isInstance(bean)) { - registeFunction(functionName, (IDipperInterfaceAdpater)bean); + registeFunction(functionName, (IDipperInterfaceAdpater) bean); return null; } FunctionConfigureMap functionConfigureMap = functionName2Engies.get(functionName); @@ -117,8 +97,8 @@ private FunctionConfigure registeFunctionInner(String functionName, Object bean, functionConfigureMap.registFunction(engine); return engine; } catch (Exception e) { - LOG.error("DefaultFunctionServiceImpl registeFunction error", e); - throw new RuntimeException("can not regeiste this method "+functionName); + LOGGER.error("DefaultFunctionServiceImpl registeFunction error", e); + throw new RuntimeException("can not regeiste this method " + functionName); } } @@ -161,7 +141,7 @@ public void registeFunction(Object bean) { FunctionType functionType = FunctionType.UDF; if (udaf != null) { functionType = FunctionType.UDAF; - String name = ((UDAFFunction)udaf).value(); + String name = ((UDAFFunction) udaf).value(); this.className2InnerInterface.put(name, bean); this.className2InnerInterface.put(name.toUpperCase(), bean); return; @@ -217,7 +197,7 @@ private String getParamsComment(Method method) { for (Annotation[] parameterAnnotation : parameterAnnotations) { for (Annotation annotation : parameterAnnotation) { if (annotation instanceof FunctionParamter) { - FunctionParamter param = (FunctionParamter)annotation; + FunctionParamter param = (FunctionParamter) annotation; result.append(param.comment() + "@" + param.value() + ","); } } @@ -264,7 +244,7 @@ public T executeFunction(IMessage message, FunctionContext context, String f Object[] allParameters = createAllParameter(message, context, parameters); FunctionConfigure functionConfigure = getFunctionConfigure(functionName, allParameters); if (functionConfigure == null) { - LOG.warn("not found engine for " + functionName); + LOGGER.warn("not found engine for " + functionName); return null; } return directExecuteFunction(functionConfigure, allParameters); @@ -272,7 +252,7 @@ public T executeFunction(IMessage message, FunctionContext context, String f @Override public FunctionConfigure getFunctionConfigure(IMessage message, FunctionContext context, String functionName, - Object... parameters) { + Object... parameters) { Object[] allParameters = createAllParameter(message, context, parameters); return getFunctionConfigure(functionName, allParameters); } @@ -313,25 +293,26 @@ public FunctionConfigure getFunctionConfigure(String functionName, Object... all @Override public DataType getReturnDataType(String functionName) { FunctionConfigureMap functionConfigureMap = functionName2Engies.get(functionName); if (functionConfigureMap == null) { - LOG.warn("get function may be not registe engine for " + functionName); + LOGGER.warn("get function may be not registe engine for " + functionName); return null; } List functionConfigureList = functionConfigureMap.getFunctionConfigureList(); - if(functionConfigureList==null||functionConfigureList.size()==0){ + if (functionConfigureList == null || functionConfigureList.size() == 0) { return null; } - DataType returnDataType=null; - for(FunctionConfigure functionConfigure:functionConfigureList){ - if(returnDataType==null){ - returnDataType= functionConfigure.getReturnDataType(); + DataType returnDataType = null; + for (FunctionConfigure functionConfigure : functionConfigureList) { + if (returnDataType == null) { + returnDataType = functionConfigure.getReturnDataType(); continue; } - if(!returnDataType.getDataTypeName().equals(functionConfigure.getReturnDataType().getDataTypeName())){ - throw new RuntimeException("can not return returnDataType , the FunctionConfigureMap has different returnDataType, the funtcion is "+functionName); + if (!returnDataType.getDataTypeName().equals(functionConfigure.getReturnDataType().getDataTypeName())) { + throw new RuntimeException("can not return returnDataType , the FunctionConfigureMap has different returnDataType, the funtcion is " + functionName); } } return null; } + @Override public T directExecuteFunction(String functionName, Object... allParameters) { FunctionConfigure engine = getFunctionConfigure(functionName, allParameters); @@ -343,21 +324,21 @@ public T directExecuteFunction(String functionName, Object... allParameters) } protected T directExecuteFunction(FunctionConfigure engine, Object... allParameters) { - return (T)engine.execute(allParameters); + return (T) engine.execute(allParameters); } @Override public boolean startWith(String functionName, Class[] classes) { FunctionConfigureMap functionConfigureMap = functionName2Engies.get(functionName); if (functionConfigureMap == null) { - LOG.warn("startWith may be not registe engine for " + functionName); + LOGGER.warn("startWith may be not registe engine for " + functionName); return false; } return functionConfigureMap.startWith(classes); } private T doDipperInterface(IMessage message, FunctionContext context, IDipperInterfaceAdpater adpater, - Object[] parameters) { + Object[] parameters) { JSONObject jsonObject = new JSONObject(); for (int i = 0; i < adpater.count(); i++) { @@ -370,7 +351,7 @@ private T doDipperInterface(IMessage message, FunctionContext context, IDipp } jsonObject.put(name, FunctionUtils.getValue(message, context, parameters[i].toString())); } - T t = (T)adpater.doScript(jsonObject.toJSONString()); + T t = (T) adpater.doScript(jsonObject.toJSONString()); dealBreakExecute(context, t); return t; } @@ -378,7 +359,7 @@ private T doDipperInterface(IMessage message, FunctionContext context, IDipp private void dealBreakExecute(FunctionContext context, T t) { try { if (t instanceof Map) { - if (StringUtil.equals((((Map)t).get("breakExecute") + ""), "1")) { + if (StringUtil.equals((((Map) t).get("breakExecute") + ""), "1")) { context.breakExecute(); } } @@ -409,7 +390,7 @@ public T getInnerInteface(String interfacName) { if (!className2InnerInterface.containsKey(interfacName)) { return null; } - return (T)className2InnerInterface.get(interfacName); + return (T) className2InnerInterface.get(interfacName); } public Map getFunctionName2Engies() { @@ -424,4 +405,26 @@ public Map getFunctionParam2Engies() { return functionParam2Engies; } + protected class IngoreInsensitiveMap extends HashMap { + + @Override + public V get(Object key) { + if (key == null) { + return null; + } + String keyString = (String) key; + keyString = keyString.toLowerCase(); + return super.get(keyString); + } + + @Override + public V put(String key, V value) { + if (key == null) { + return value; + } + String keyString = key.toLowerCase(); + return super.put(keyString, value); + } + } + } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/service/impl/ScanFunctionService.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/service/impl/ScanFunctionService.java index a92b279a..30b58001 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/service/impl/ScanFunctionService.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/service/impl/ScanFunctionService.java @@ -18,18 +18,18 @@ import java.io.File; import java.lang.reflect.Modifier; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.calssscaner.AbstractScan; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.function.service.IFunctionService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * 提供扫描function的能力 */ public class ScanFunctionService extends DefaultFunctionServiceImpl implements IFunctionService { - private static final Log LOG = LogFactory.getLog(ScanFunctionService.class); + private static final Logger LOGGER = LoggerFactory.getLogger(ScanFunctionService.class); private static final ScanFunctionService functionService = new ScanFunctionService(); @@ -41,7 +41,7 @@ protected void doProcessor(Class clazz, String functionName) { clazz.getModifiers()) || Modifier.isPrivate(clazz.getModifiers())) { return; } - if (clazz.getName().indexOf("$") != -1) { + if (clazz.getName().contains("$")) { return; } Object o = clazz.newInstance(); @@ -53,7 +53,7 @@ protected void doProcessor(Class clazz, String functionName) { } catch (Exception e) { e.printStackTrace(); - LOG.error("初始化类错误" + e.getMessage(), e); + LOGGER.error("初始化类错误" + e.getMessage(), e); } } }; @@ -65,6 +65,7 @@ public ScanFunctionService() { public ScanFunctionService(boolean scanDipper) { super(); if (scanDipper) { + scan.scanPackage("org.apache.rocketmq.streams.dim.function.script"); scan.scanPackage("org.apache.rocketmq.streams.script.function.impl"); scan.scanPackage("org.apache.rocketmq.streams.filter.function"); scan.scanPackage("org.apache.rocketmq.streams.dim.function"); @@ -83,11 +84,11 @@ public void scanClassDir(String dir, String packageName, ClassLoader classLoader scan.scanClassDir(dir, packageName, classLoader); } - public void scanePackages(String... packageNames) { + public void scanPackages(String... packageNames) { scan.scanPackages(packageNames); } - public void scanePackage(String packageName) { + public void scanPackage(String packageName) { scan.scanPackages(packageName); } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/GroupScriptExpression.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/GroupScriptExpression.java index ad48b863..c3b3a821 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/GroupScriptExpression.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/GroupScriptExpression.java @@ -26,6 +26,7 @@ import org.apache.rocketmq.streams.common.topology.model.AbstractRule; import org.apache.rocketmq.streams.common.utils.CollectionUtil; import org.apache.rocketmq.streams.common.utils.PrintUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.script.context.FunctionContext; import org.apache.rocketmq.streams.script.parser.imp.FunctionParser; import org.apache.rocketmq.streams.script.service.IScriptExpression; @@ -37,23 +38,18 @@ */ public class GroupScriptExpression implements IScriptExpression { - private IScriptExpression ifExpresssion; - - private List thenExpresssions; - - private List elseExpressions; - - private String scriptParameterStr; + private static final String TAB = FunctionParser.TAB; protected List rules; - protected List elseIfExpressions = new ArrayList<>(); - - private static final String TAB = FunctionParser.TAB; - protected List beforeExpressions; protected List afterExpressions; - - private transient ICaseDependentParser caseDependentParser; + private IScriptExpression ifExpresssion; + private transient String boolVar; + private transient AbstractRule rule; + private List thenExpresssions; + private List elseExpressions; + private String scriptParameterStr; + private transient IGroupScriptOptimization caseDependentParser; @Override public Object getScriptParamter(IMessage message, FunctionContext context) { @@ -65,15 +61,15 @@ public String getScriptParameterStr() { return scriptParameterStr; } + public void setScriptParameterStr(String scriptParameterStr) { + this.scriptParameterStr = scriptParameterStr; + } + @Override public Object executeExpression(IMessage channelMessage, FunctionContext context) { - if (beforeExpressions != null) { - for (IScriptExpression scriptExpression : beforeExpressions) { - scriptExpression.executeExpression(channelMessage, context); - } - } + executeBeforeExpression(channelMessage, context); + Boolean result = executeIf(this, channelMessage, context); - Boolean result = (Boolean) ifExpresssion.executeExpression(channelMessage, context); Object value = null; if (result) { if (thenExpresssions != null) { @@ -86,7 +82,7 @@ public Object executeExpression(IMessage channelMessage, FunctionContext context if (elseIfExpressions != null && elseIfExpressions.size() > 0) { for (int i = elseIfExpressions.size() - 1; i >= 0; i--) { GroupScriptExpression expression = elseIfExpressions.get(i); - boolean success = (Boolean) expression.ifExpresssion.executeExpression(channelMessage, context); + boolean success = executeIf(expression, channelMessage, context); if (success) { if (expression.thenExpresssions != null) { for (IScriptExpression scriptExpression : expression.thenExpresssions) { @@ -107,6 +103,26 @@ public Object executeExpression(IMessage channelMessage, FunctionContext context return null; } + protected Boolean executeIf(GroupScriptExpression groupScriptExpression, IMessage message, FunctionContext context) { + if (StringUtil.isNotEmpty(groupScriptExpression.boolVar)) { + return message.getMessageBody().getBooleanValue(groupScriptExpression.boolVar); + } else if (groupScriptExpression.rule != null) { + return groupScriptExpression.rule.doMessage(message, context); + } else { + Boolean result = (Boolean) groupScriptExpression.ifExpresssion.executeExpression(message, context); + return result; + } + } + + public void executeBeforeExpression(IMessage channelMessage, FunctionContext context) { + if (beforeExpressions != null) { + for (IScriptExpression scriptExpression : beforeExpressions) { + scriptExpression.executeExpression(channelMessage, context); + } + } + + } + protected Object executeAfterExpression(IMessage channelMessage, FunctionContext context) { Object object = null; if (afterExpressions != null) { @@ -205,7 +221,7 @@ protected List getDependentFieldsInner() { if (elseIfExpressions != null) { parameters.addAll(elseIfExpressions); } - ICaseDependentParser caseDependentParser = loadCaseDependentParser(); + IGroupScriptOptimization caseDependentParser = loadCaseDependentParser(); if (parameters != null && parameters.size() > 0) { for (IScriptExpression scriptExpression : parameters) { List names = null; @@ -222,6 +238,31 @@ protected List getDependentFieldsInner() { return fieldNames; } + public Set getIFDependentFields() { + Set fieldNames = new HashSet<>(); + List parameters = new ArrayList<>(); + if (ifExpresssion != null) { + parameters.add(ifExpresssion); + } + + IGroupScriptOptimization caseDependentParser = loadCaseDependentParser(); + if (ifExpresssion != null && parameters.size() > 0) { + for (IScriptExpression scriptExpression : parameters) { + List names = null; + if (caseDependentParser != null && caseDependentParser.isCaseFunction(scriptExpression)) { + names = new ArrayList<>(caseDependentParser.getDependentFields(scriptExpression)); + } else { + names = scriptExpression.getDependentFields(); + } + if (names != null) { + fieldNames.addAll(names); + } + } + } + return fieldNames; + + } + protected Set getNewFieldNamesInner() { Set set = new HashSet<>(); List parameters = new ArrayList<>(); @@ -242,7 +283,7 @@ protected Set getNewFieldNamesInner() { return set; } - protected ICaseDependentParser loadCaseDependentParser() { + protected IGroupScriptOptimization loadCaseDependentParser() { if (this.caseDependentParser != null) { return this.caseDependentParser; } @@ -250,34 +291,19 @@ protected ICaseDependentParser loadCaseDependentParser() { if (caseDependentParser != null) { return this.caseDependentParser; } - ServiceLoaderComponent caseServiceLoader = ServiceLoaderComponent.getInstance(ICaseDependentParser.class); + ServiceLoaderComponent caseServiceLoader = ServiceLoaderComponent.getInstance(IGroupScriptOptimization.class); - List caseDependentParsers = caseServiceLoader.loadService(); - ICaseDependentParser caseDependentParser = null; + List caseDependentParsers = caseServiceLoader.loadService(); + IGroupScriptOptimization caseDependentParser = null; if (caseDependentParsers != null && caseDependentParsers.size() > 0) { caseDependentParser = caseDependentParsers.get(0); + caseDependentParser.compile(this); } return caseDependentParser; } } - public void setIfExpresssion(IScriptExpression ifExpresssion) { - this.ifExpresssion = ifExpresssion; - } - - public void setThenExpresssions(List thenExpresssions) { - this.thenExpresssions = thenExpresssions; - } - - public void setElseExpressions(List elseExpressions) { - this.elseExpressions = elseExpressions; - } - - public void setScriptParameterStr(String scriptParameterStr) { - this.scriptParameterStr = scriptParameterStr; - } - public List getElseIfExpressions() { return elseIfExpressions; } @@ -290,6 +316,10 @@ public IScriptExpression getIfExpresssion() { return ifExpresssion; } + public void setIfExpresssion(IScriptExpression ifExpresssion) { + this.ifExpresssion = ifExpresssion; + } + public List getBeforeExpressions() { return beforeExpressions; } @@ -312,6 +342,10 @@ public List getElseExpressions() { return elseExpressions; } + public void setElseExpressions(List elseExpressions) { + this.elseExpressions = elseExpressions; + } + public List getRules() { return rules; } @@ -324,6 +358,10 @@ public List getThenExpresssions() { return thenExpresssions; } + public void setThenExpresssions(List thenExpresssions) { + this.thenExpresssions = thenExpresssions; + } + public Map> getBeforeDependents() { Map> map = new HashMap<>(); if (CollectionUtil.isEmpty(beforeExpressions)) { @@ -337,4 +375,20 @@ public List getThenExpresssions() { } return map; } + + public String getBoolVar() { + return boolVar; + } + + public void setBoolVar(String boolVar) { + this.boolVar = boolVar; + } + + public AbstractRule getRule() { + return rule; + } + + public void setRule(AbstractRule rule) { + this.rule = rule; + } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/ICaseDependentParser.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/IGroupScriptOptimization.java similarity index 91% rename from rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/ICaseDependentParser.java rename to rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/IGroupScriptOptimization.java index b4aacec1..7c4bc215 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/ICaseDependentParser.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/IGroupScriptOptimization.java @@ -19,7 +19,7 @@ import java.util.Set; import org.apache.rocketmq.streams.script.service.IScriptExpression; -public interface ICaseDependentParser { +public interface IGroupScriptOptimization { /** * 获取依赖的字段名 * @@ -28,4 +28,7 @@ public interface ICaseDependentParser { Set getDependentFields(IScriptExpression scriptExpression); boolean isCaseFunction(IScriptExpression scriptExpression); + + void compile(GroupScriptExpression groupScriptExpression); + } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/ScriptExpression.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/ScriptExpression.java index e7e39aeb..034244e0 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/ScriptExpression.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/ScriptExpression.java @@ -22,16 +22,15 @@ import java.util.List; import java.util.Map; import java.util.Set; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.cache.softreference.ICache; import org.apache.rocketmq.streams.common.cache.softreference.impl.SoftReferenceCache; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.configuration.SystemContext; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.optimization.HomologousVar; -import org.apache.rocketmq.streams.common.utils.PrintUtil; +import org.apache.rocketmq.streams.common.utils.IdUtil; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.common.utils.ReflectUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.script.ScriptComponent; @@ -43,6 +42,8 @@ import org.apache.rocketmq.streams.script.service.IScriptExpression; import org.apache.rocketmq.streams.script.service.IScriptParamter; import org.apache.rocketmq.streams.script.utils.FunctionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * 一个函数,如a=now();就是一个表达式 这里是函数真正执行的地方 @@ -50,32 +51,26 @@ @SuppressWarnings("rawtypes") public class ScriptExpression implements IScriptExpression { - private static Log LOG = LogFactory.getLog(ScriptExpression.class); - + private static final Logger LOGGER = LoggerFactory.getLogger(ScriptExpression.class); + private final static ICache cache = new SoftReferenceCache<>(); + protected transient Boolean isMultiField; + protected transient HomologousVar homologousVar; + protected transient volatile CompileScriptExpression compileScriptExpression; + protected transient volatile CompileParameter compileParameter; + protected transient Boolean hasSubField = null; private String newFieldName; - - protected transient Boolean ismutilField;//mutil fields eg :a.b.c - private String expressionStr; - private String functionName; - private List parameters; - private Long groupId; - protected transient HomologousVar homologousVar; - - protected transient volatile CompileScriptExpression compileScriptExpression; - - protected transient volatile CompileParameter compileParameter; - private transient static ICache cache = new SoftReferenceCache<>(); + private ScriptComponent scriptComponent = ScriptComponent.getInstance(); @Override public Object executeExpression(IMessage message, FunctionContext context) { - + String configName = message.getHeader().getPipelineName(); try { - if (ismutilField == null && newFieldName != null) { - ismutilField = newFieldName.indexOf(".") != -1; + if (isMultiField == null && newFieldName != null) { + isMultiField = newFieldName.contains("."); } Boolean isMatch = null; if (this.homologousVar != null) { @@ -107,34 +102,22 @@ public Object executeExpression(IMessage message, FunctionContext context) { value = execute(message, context); } long cost = System.currentTimeMillis() - startTime; - long timeout = 10; - if (ComponentCreator.getProperties().getProperty(ConfigureFileKey.MONITOR_SLOW_TIMEOUT) != null) { - timeout = Long.valueOf(ComponentCreator.getProperties().getProperty(ConfigureFileKey.MONITOR_SLOW_TIMEOUT)); + long timeout = 1000; + if (SystemContext.getProperty(ConfigurationKey.MONITOR_SLOW_TIMEOUT) != null) { + timeout = Long.parseLong(SystemContext.getProperty(ConfigurationKey.MONITOR_SLOW_TIMEOUT)); } if (cost > timeout) { - String varValue = ""; - if (this.getScriptParamters() != null && this.getScriptParamters().size() > 0) { - varValue = IScriptOptimization.getParameterValue(this.getParameters().get(0)); - varValue = message.getMessageBody().getString(varValue); - } - LOG.warn("SLOW-" + cost + "----" + this.toString() + PrintUtil.LINE + "the var value is " + varValue); + LOGGER.warn("[{}][{}] Expression_Exec_Slow_({})_Cost({})", IdUtil.instanceId(), configName, this.expressionStr, cost); + } return value; } catch (Exception e) { - e.printStackTrace(); - String varValue = ""; - if (this.getScriptParamters() != null && this.getScriptParamters().size() > 0) { - varValue = IScriptOptimization.getParameterValue(this.getParameters().get(0)); - varValue = message.getMessageBody().getString(varValue); - } - LOG.error("ERROR-" + this.toString() + PrintUtil.LINE + "the var value is " + varValue, e); - throw new RuntimeException(e); + Object[] ps = createParameters(message, context, false, null); + throw new RuntimeException("[" + this.expressionStr + "]--[" + MapKeyUtil.createKeyFromObject(ps) + "]", e); } } - private ScriptComponent scriptComponent = ScriptComponent.getInstance(); - public Object execute(IMessage message, FunctionContext context) { Object[] ps = null; FunctionConfigure functionConfigure = null; @@ -161,7 +144,8 @@ public Object execute(IMessage message, FunctionContext context) { varValue = IScriptOptimization.getParameterValue(this.getParameters().get(0)); varValue = message.getMessageBody().getString(varValue); } - throw new RuntimeException("can not find function " + functionName + "ERROR-" + this.toString() + PrintUtil.LINE + "the var value is " + varValue + PrintUtil.LINE + this.toString() + "the var value is " + varValue); + String configName = message.getHeader().getPipelineName(); + throw new RuntimeException("[" + this.expressionStr + "]--[" + MapKeyUtil.createKeyFromObject(ps) + "]"); } Object value = executeFunctionConfigue(message, context, functionConfigure, ps); compileScriptExpression = new CompileScriptExpression(this, functionConfigure); @@ -199,13 +183,11 @@ public Object executeFunctionConfigue(IMessage message, FunctionContext context, return value; } - protected transient Boolean hasSubField = null; - public void setValue2Var(IMessage message, AbstractContext context, String newFieldName, Object value) { - if (newFieldName == null || value == null) { + if (newFieldName == null) { return; } - if (!ismutilField) { + if (!isMultiField) { message.getMessageBody().put(newFieldName, value); return; } @@ -261,7 +243,7 @@ private Object[] createParameters(IMessage message, FunctionContext context, boo if (value == null) { paras[i] = null; } - if (String.class.isInstance(value)) { + if (value instanceof String) { String str = (String) value; Object object = FunctionUtils.getValue(message, context, str); paras[i] = object; @@ -310,15 +292,15 @@ public String getFunctionName() { return functionName; } + public void setFunctionName(String functionName) { + this.functionName = functionName; + } + @Override public String getExpressionDescription() { return expressionStr; } - public void setFunctionName(String functionName) { - this.functionName = functionName; - } - public List getParameters() { return parameters; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/ScriptParameter.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/ScriptParameter.java index e5d19f30..32262297 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/ScriptParameter.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/ScriptParameter.java @@ -41,7 +41,7 @@ public class ScriptParameter implements IScriptParamter { public ScriptParameter(String simpleParamter) { this.scriptParameterStr = simpleParamter; - if (ContantsUtil.isContant(simpleParamter)) { + if (ContantsUtil.isContant(simpleParamter) || simpleParamter.startsWith("(") && simpleParamter.endsWith(")")) { this.leftVarName = simpleParamter; } else if (simpleParamter.indexOf("==") != -1) { doConditonValue(simpleParamter, "=="); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/AggregationScript.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/AggregationScript.java index d8d8cde3..1bbef65b 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/AggregationScript.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/AggregationScript.java @@ -22,9 +22,8 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -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.Context; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; import org.apache.rocketmq.streams.common.utils.StringUtil; @@ -40,19 +39,33 @@ import org.apache.rocketmq.streams.script.function.aggregation.MinAccumulator; import org.apache.rocketmq.streams.script.function.aggregation.SumAccumulator; import org.apache.rocketmq.streams.script.service.IAccumulator; +import org.apache.rocketmq.streams.script.utils.FunctionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * 主要在window中使用,做统计计算使用 */ public class AggregationScript implements IStreamOperator> { - private static final Log LOG = LogFactory.getLog(AggregationScript.class); - private static Set supportQuickStoreModelFunctions=new HashSet(){{ - add("max"); - add("min"); - add("count"); - add("sum"); - add("avg"); + /** + * the way to accumulate: single or multi + */ + public static final String INNER_AGGREGATION_COMPUTE_KEY = "_inner_aggregation_single_"; + public static final String INNER_AGGREGATION_COMPUTE_SINGLE = "single"; + public static final String INNER_AGGREGATION_COMPUTE_MULTI = "multi"; + public static final String INNER_AGGREGATION_VALUE_KEY = "_inner_aggregation_result_"; + /** + * inner field in message + */ + public static final String _INNER_AGGREGATION_FUNCTION_VALUE_ = "_INNER_AGGREGATION_FUNCTION_VALUE_"; + private static final Logger LOGGER = LoggerFactory.getLogger(AggregationScript.class); + private static Set supportQuickStoreModelFunctions = new HashSet() {{ + add("max"); + add("min"); + add("count"); + add("sum"); + add("avg"); }}; private static Map aggregationEngineMap = new ConcurrentHashMap() {{ put("max", MaxAccumulator.class); @@ -66,44 +79,23 @@ public class AggregationScript implements IStreamOperator doMessage(IMessage message, AbstractContext context) { @@ -155,7 +168,7 @@ public List doMessage(IMessage message, AbstractContext context) { director = getAggregationFunction(functionName); } if (director != null) { - String introduction = (String)message.getMessageBody().getOrDefault(INNER_AGGREGATION_COMPUTE_KEY, ""); + String introduction = (String) message.getMessageBody().getOrDefault(INNER_AGGREGATION_COMPUTE_KEY, ""); boolean isSingleAccumulate = INNER_AGGREGATION_COMPUTE_SINGLE.equals(introduction); boolean isMultiAccumulate = INNER_AGGREGATION_COMPUTE_MULTI.equals(introduction); if (isSingleAccumulate && accumulator != null && parameterNames != null) { @@ -177,12 +190,12 @@ public List doMessage(IMessage message, AbstractContext context) { private Object[] getValueFromMessage(String[] parameterNames, IMessage message) { Object[] parameterValues = new Object[parameterNames.length]; for (int index = 0; index < parameterNames.length; index++) { - if (isConstValue(parameterNames[index])) { - parameterValues[index] = parameterNames[index]; + if ("*".equals(parameterNames[index])) { + parameterValues[index] = message.getMessageBody(); } else { - parameterValues[index] = message.getMessageBody().getOrDefault(parameterNames[index], - parameterNames[index]); + parameterValues[index] = FunctionUtils.getValue(message, new Context(message), parameterNames[index]); } + } return parameterValues; } @@ -191,28 +204,6 @@ private boolean isConstValue(String parameter) { return parameter.startsWith("\"") || parameter.startsWith("'"); } - public static IAccumulator getAggregationFunction(String functionName) { - if (StringUtil.isEmpty(functionName)) { - return null; - } - try { - IAccumulator accumulator = scriptComponent.getFunctionService().getInnerInteface(functionName); - if (accumulator != null) { - return accumulator; - } - return aggregationEngineMap.containsKey(functionName) ? (IAccumulator)aggregationEngineMap.get(functionName) - .newInstance() : null; - - } catch (Exception e) { - LOG.error("failed in getting aggregation function, " + functionName, e); - } - return null; - } - - - public static void registUDAF(String functionName,Class accumulator){ - aggregationEngineMap.put(functionName,accumulator); - } public Object getAccumulator() { return accumulator; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/FunctionScript.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/FunctionScript.java index 34500c90..e2880034 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/FunctionScript.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/FunctionScript.java @@ -19,22 +19,18 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.interfaces.IBaseStreamOperator; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; -import org.apache.rocketmq.streams.common.topology.ChainStage; -import org.apache.rocketmq.streams.common.topology.builder.IStageBuilder; +import org.apache.rocketmq.streams.common.model.ThreadContext; +import org.apache.rocketmq.streams.common.topology.IStageBuilder; import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; +import org.apache.rocketmq.streams.common.topology.model.AbstractChainStage; import org.apache.rocketmq.streams.common.topology.model.AbstractScript; import org.apache.rocketmq.streams.common.topology.stages.ScriptChainStage; import org.apache.rocketmq.streams.common.utils.CollectionUtil; @@ -45,25 +41,27 @@ import org.apache.rocketmq.streams.script.service.IScriptExpression; import org.apache.rocketmq.streams.script.service.IScriptParamter; import org.apache.rocketmq.streams.serviceloader.ServiceLoaderComponent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * * 对外提供的脚本算子,通过输入脚本,来实现业务逻辑 * 脚本存储的成员变量是value字段 */ -public class FunctionScript extends AbstractScript, FunctionContext> implements IStreamOperator>, IStageBuilder, IAfterConfigurableRefreshListener { - - private static final Log LOG = LogFactory.getLog(FunctionScript.class); +public class FunctionScript extends AbstractScript, FunctionContext> implements IStreamOperator>, IStageBuilder> { + private static final Logger LOGGER = LoggerFactory.getLogger(FunctionScript.class); + protected transient IScriptOptimization.IOptimizationCompiler optimizationCompiler; + //protected transient ScriptExpressionGroupsProxy scriptExpressionGroupsProxy; + protected transient Map> name2ScriptExpressions = null; + protected transient AtomicBoolean hasStart = new AtomicBoolean(false); /** * 脚本解析的表达式列表 */ - private transient List scriptExpressions = new ArrayList(); - //protected transient ScriptExpressionGroupsProxy scriptExpressionGroupsProxy; + private transient List scriptExpressions = new ArrayList<>(); /** * 表达式,转化成streamoperator接口列表,可以在上层中使用 */ - private transient List> receivers = new ArrayList<>(); - - protected transient IScriptOptimization.IOptimizationCompiler optimizationCompiler; + private transient List>> receivers = new ArrayList<>(); public FunctionScript() { setType(AbstractScript.TYPE); @@ -80,20 +78,22 @@ public FunctionScript(String value) { @Override protected boolean initConfigurable() { String value = this.value; - /** - * 健壮性检查,对于不符合规范的字符做转型 + /* + 健壮性检查,对于不符合规范的字符做转型 */ value = value.replace("’", "'"); value = value.replace("‘", "'"); value = value.replace("’", "'"); this.scriptExpressions = FunctionParser.getInstance().parse(value); + //转化成istreamoperator 接口 - for (IScriptExpression scriptExpression : this.scriptExpressions) { + for (IScriptExpression scriptExpression : this.scriptExpressions) { receivers.add((message, context) -> { scriptExpression.executeExpression(message, context); return message; }); } + doScriptOptimization(); return true; } @@ -105,15 +105,32 @@ public List doMessage(IMessage message, AbstractContext context) { context.syncSubContext(functionContext); } - List result = AbstractContext.executeScript(message, functionContext, this.receivers); + List result = executeScript(message, functionContext, this.receivers); + if (context != null) { context.syncContext(functionContext); } return result; } + public Object executeScriptAsFunction(IMessage message, AbstractContext context) { + + FunctionContext functionContext = new FunctionContext(message); + if (context != null) { + context.syncSubContext(functionContext); + } + + executeScript(message, functionContext, this.receivers); + + Object returnValue = functionContext.getReturnValue(); + if (context != null) { + context.syncContext(functionContext); + } + return returnValue; + } + @Override - public ChainStage createStageChain(PipelineBuilder pipelineBuilder) { + public AbstractChainStage createStageChain(PipelineBuilder pipelineBuilder) { ScriptChainStage scriptChainStage = new ScriptChainStage(); pipelineBuilder.addConfigurables(this); scriptChainStage.setScript(this); @@ -135,14 +152,8 @@ public Map> getDependentFields() { Set newFieldNames = scriptExpression.getNewFieldNames(); if (newFieldNames != null && newFieldNames.size() > 0) { List fieldNames = scriptExpression.getDependentFields(); - Iterator it = newFieldNames.iterator(); - while (it.hasNext()) { - String newFieldName = it.next(); - List list = newFieldName2DependentFields.get(newFieldName); - if (list == null) { - list = new ArrayList<>(); - newFieldName2DependentFields.put(newFieldName, list); - } + for (String newFieldName : newFieldNames) { + List list = newFieldName2DependentFields.computeIfAbsent(newFieldName, k -> new ArrayList<>()); if (fieldNames != null) { list.addAll(fieldNames); } @@ -174,8 +185,6 @@ public String[] getDependentParameters() { return parameterSet.toArray(new String[0]); } - protected transient Map name2ScriptExpressions = null; - /** * 跟定字段,查找对应的脚本 * @@ -187,9 +196,9 @@ public List getScriptsByDependentField(String fieldName) { if (name2ScriptExpressions == null) { synchronized (this) { if (name2ScriptExpressions == null) { - Map map = new HashMap<>(); - for (IScriptExpression expression : this.scriptExpressions) { - if (ScriptExpression.class.isInstance(expression)) { + Map> map = new HashMap<>(); + for (IScriptExpression expression : this.scriptExpressions) { + if (expression instanceof ScriptExpression) { ScriptExpression scriptExpression = (ScriptExpression) expression; if (scriptExpression.getNewFieldName() != null) { map.put(scriptExpression.getNewFieldName(), scriptExpression); @@ -212,6 +221,103 @@ public List getScriptsByDependentField(String fieldName) { return converse; } + protected List executeScript(IMessage channelMessage, C context, + List> scriptExpressions) { + List messages = new ArrayList<>(); + if (scriptExpressions == null) { + return messages; + } + boolean isSplitMode = context.isSplitModel(); + context.closeSplitMode(channelMessage); + int nextIndex = 1; + //long start=System.currentTimeMillis(); + executeScript(scriptExpressions.get(0), channelMessage, context, nextIndex, scriptExpressions); + + if (!context.isContinue()) { + context.setSplitModel(isSplitMode || context.isSplitModel()); + return null; + } + + if (context.isSplitModel()) { + messages = context.getSplitMessages(); + } else { + messages.add(context.getMessage()); + } + context.setSplitModel(isSplitMode || context.isSplitModel()); + return messages; + } + + /** + * 执行当前规则,如果规则符合拆分逻辑调拆分逻辑。为了是减少循环次数,一次循环多条规则 + * + * @param currentExpression + * @param channelMessage + * @param context + * @param nextIndex + * @param scriptExpressions + */ + protected void executeScript( + IBaseStreamOperator currentExpression, + IMessage channelMessage, C context, int nextIndex, + List> scriptExpressions) { + //long start=System.currentTimeMillis(); + + /** + * 为了兼容blink udtf,通过localthread把context传给udtf的collector + */ + ThreadContext threadContext = ThreadContext.getInstance(); + threadContext.set(context); + currentExpression.doMessage(channelMessage, context); + + //System.out.println(currentExpression.toString()+" cost time is "+(System.currentTimeMillis()-start)); + if (context.isContinue() == false) { + return; + } + if (nextIndex >= scriptExpressions.size()) { + return; + } + IBaseStreamOperator nextScriptExpression = scriptExpressions.get(nextIndex); + nextIndex++; + if (context.isSplitModel()) { + // start=System.currentTimeMillis(); + executeSplitScript(nextScriptExpression, channelMessage, context, nextIndex, scriptExpressions); + + //System.out.println(currentExpression.toString()+" cost time is "+(System.currentTimeMillis()-start)); + } else { + executeScript(nextScriptExpression, channelMessage, context, nextIndex, scriptExpressions); + } + } + + protected void executeSplitScript( + IBaseStreamOperator currentExpression, IMessage channelMessage, C context, int nextIndex, + List> scriptExpressions) { + if (context.getSplitMessages() == null || context.getSplitMessages().size() == 0) { + return; + } + List result = new ArrayList<>(); + List splitMessages = new ArrayList(); + splitMessages.addAll(context.getSplitMessages()); + int splitMessageOffset = 0; + for (IMessage message : splitMessages) { + context.closeSplitMode(message); + message.getHeader().addLayerOffset(splitMessageOffset); + splitMessageOffset++; + executeScript(currentExpression, message, context, nextIndex, scriptExpressions); + if (!context.isContinue()) { + context.cancelBreak(); + continue; + } + if (context.isSplitModel()) { + result.addAll(context.getSplitMessages()); + } else { + result.add(context.getMessage()); + } + + } + context.openSplitModel(); + context.setSplitMessages(result); + } + protected void findAllScript(IScriptExpression expression, List result) { result.add(expression.toString()); List dependentFieldNames = expression.getDependentFields(); @@ -240,10 +346,7 @@ public void setScript(String script) { this.value = script; } - protected transient AtomicBoolean hasStart = new AtomicBoolean(false); - - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { + protected void doScriptOptimization() { if (hasStart.compareAndSet(false, true)) { IScriptOptimization scriptOptimization = null; @@ -255,17 +358,17 @@ public void doProcessAfterRefreshConfigurable(IConfigurableService configurableS } if (this.scriptExpressions == null) { - LOG.debug("empty function"); + LOGGER.debug("empty function"); } else { List expressions = this.scriptExpressions; if (scriptOptimization != null) { this.optimizationCompiler = scriptOptimization.compile(this.scriptExpressions, this); expressions = this.optimizationCompiler.getOptimizationExpressionList(); } - this.scriptExpressions = expressions; - List> newReceiver = new ArrayList<>(); + // this.scriptExpressions = expressions; + List>> newReceiver = new ArrayList<>(); //转化成istreamoperator 接口 - for (IScriptExpression scriptExpression : expressions) { + for (IScriptExpression scriptExpression : expressions) { newReceiver.add((message, context) -> { scriptExpression.executeExpression(message, context); return message; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/GroovyScriptOperator.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/GroovyScriptOperator.java index a7562422..07aa3a5b 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/GroovyScriptOperator.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/GroovyScriptOperator.java @@ -26,34 +26,48 @@ import javax.script.ScriptEngine; import javax.script.ScriptEngineManager; import javax.script.ScriptException; -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.IMessage; import org.apache.rocketmq.streams.common.context.Message; import org.apache.rocketmq.streams.script.context.FunctionContext; import org.apache.rocketmq.streams.script.operator.AbstractScriptOperator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * 实现思路,通过INNER_MESSAG 把message的jsonobject传给groovy,groovy中直接操作jsonobject */ public class GroovyScriptOperator extends AbstractScriptOperator { - protected static final Log LOG = LogFactory.getLog(GroovyScriptOperator.class); protected static final String GROOVY_NAME = "groovy"; + private static final Logger LOGGER = LoggerFactory.getLogger(GroovyScriptOperator.class); protected transient Invocable inv; protected transient ScriptEngine engine; + public static void main(String[] args) throws ScriptException { + //javax.operator.Bindings + JSONObject jsonObject = new JSONObject(); + jsonObject.put("age", 18); + jsonObject.put("date", new Date()); + GroovyScriptOperator groovyScript = new GroovyScriptOperator(); + groovyScript.setValue("_msg.put(\"name\",'chris');"); + groovyScript.init(); + Message message = new Message(jsonObject); + + groovyScript.doMessage(message, new FunctionContext(message)); + System.out.println(jsonObject); + } + @Override protected boolean initConfigurable() { try { super.initConfigurable(); ScriptEngineManager factory = new ScriptEngineManager(); ScriptEngine engine = factory.getEngineByName(GROOVY_NAME); - inv = (Invocable)engine; + inv = (Invocable) engine; this.engine = engine; registFunction(); } catch (Exception e) { - LOG.error("groovy init error " + getValue(), e); + LOGGER.error("groovy init error " + getValue(), e); return false; } return true; @@ -79,20 +93,6 @@ protected String executeGroovy(String script, Bindings binding) { return null; } - public static void main(String[] args) throws ScriptException { - //javax.operator.Bindings - JSONObject jsonObject = new JSONObject(); - jsonObject.put("age", 18); - jsonObject.put("date", new Date()); - GroovyScriptOperator groovyScript = new GroovyScriptOperator(); - groovyScript.setValue("_msg.put(\"name\",'chris');"); - groovyScript.init(); - Message message = new Message(jsonObject); - - groovyScript.doMessage(message, new FunctionContext(message)); - System.out.println(jsonObject); - } - @Override public List getScriptsByDependentField(String fieldName) { throw new RuntimeException("can not support this method:getScriptsByDependentField"); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/JPythonScriptOperator.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/JPythonScriptOperator.java index 5cc929aa..4ce8b0d1 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/JPythonScriptOperator.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/JPythonScriptOperator.java @@ -21,22 +21,34 @@ import java.util.List; import java.util.Map; import java.util.Properties; -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.IMessage; import org.apache.rocketmq.streams.common.context.Message; import org.apache.rocketmq.streams.script.context.FunctionContext; import org.apache.rocketmq.streams.script.operator.AbstractScriptOperator; import org.python.util.PythonInterpreter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * 实现思路,通过INNER_MESSAG 把message的jsonobject传给python,python中直接操作jsonobject */ public class JPythonScriptOperator extends AbstractScriptOperator { - protected static final Log LOG = LogFactory.getLog(JPythonScriptOperator.class); + private static final Logger LOGGER = LoggerFactory.getLogger(JPythonScriptOperator.class); protected transient PythonInterpreter interpreter; + public static void main(String[] args) { + JPythonScriptOperator pythonScript = new JPythonScriptOperator(); + pythonScript.setValue("_msg.put('age',18);"); + pythonScript.init(); + JSONObject jsonObject = new JSONObject(); + jsonObject.put("name", "chris"); + Message message = new Message(jsonObject); + + pythonScript.doMessage(message, new FunctionContext(message)); + System.out.println(jsonObject); + } + @Override protected boolean initConfigurable() { try { @@ -54,7 +66,7 @@ protected boolean initConfigurable() { interpreter.exec("import sys"); registFunction(); } catch (Exception e) { - LOG.error("jython init error " + getValue(), e); + LOGGER.error("jython init error " + getValue(), e); return false; } return true; @@ -69,18 +81,6 @@ public List doMessage(IMessage message, AbstractContext context) { return messages; } - public static void main(String[] args) { - JPythonScriptOperator pythonScript = new JPythonScriptOperator(); - pythonScript.setValue("_msg.put('age',18);"); - pythonScript.init(); - JSONObject jsonObject = new JSONObject(); - jsonObject.put("name", "chris"); - Message message = new Message(jsonObject); - - pythonScript.doMessage(message, new FunctionContext(message)); - System.out.println(jsonObject); - } - @Override public List getScriptsByDependentField(String fieldName) { throw new RuntimeException("can not support this method:getScriptsByDependentField"); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/ScriptOperator.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/ScriptOperator.java index ddbc8d05..cb671a53 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/ScriptOperator.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/ScriptOperator.java @@ -21,7 +21,8 @@ */ public class ScriptOperator extends FunctionScript { - public ScriptOperator() {} + public ScriptOperator() { + } public ScriptOperator(String value) { super(value); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/compile/CompileParameter.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/compile/CompileParameter.java index f5b81343..8269e54f 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/compile/CompileParameter.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/compile/CompileParameter.java @@ -48,7 +48,7 @@ public CompileParameter(IScriptParamter iscriptParamter, boolean needContext) { isSimpleParameter = false; return; } - ScriptParameter scriptParameter = (ScriptParameter)iscriptParamter; + ScriptParameter scriptParameter = (ScriptParameter) iscriptParamter; if (scriptParameter.getRigthVarName() != null || scriptParameter.getFunctionName() != null) { isSimpleParameter = false; return; @@ -60,7 +60,7 @@ public CompileParameter(IScriptParamter iscriptParamter, boolean needContext) { if (!String.class.isInstance(this.leftValue)) { return; } - String varName = (String)this.leftValue; + String varName = (String) this.leftValue; if (FunctionUtils.isConstant(varName)) { this.leftValue = FunctionUtils.getConstant(varName); } else if (FunctionUtils.isLong(varName)) { @@ -74,6 +74,11 @@ public CompileParameter(IScriptParamter iscriptParamter, boolean needContext) { } } + public static void main(String[] args) { + String var = "abdf"; + System.out.println(var.indexOf(".")); + } + /** * 获取参数的值 * @@ -88,7 +93,7 @@ public Object getValue(IMessage message, FunctionContext context) { if (needContext || !String.class.isInstance(value)) { return value; } - String str = (String)value; + String str = (String) value; Object object = FunctionUtils.getValue(message, context, str); return object; } @@ -96,7 +101,7 @@ public Object getValue(IMessage message, FunctionContext context) { * 如果是无前缀,且是字段名,则获取具体字段 */ if (needContext == false && isField) { - return FunctionUtils.getFiledValue(message, context, (String)this.leftValue); + return FunctionUtils.getFiledValue(message, (String) this.leftValue); } return this.leftValue; } @@ -124,9 +129,4 @@ public void setField(boolean field) { isField = field; } - public static void main(String[] args) { - String var = "abdf"; - System.out.println(var.indexOf(".")); - } - } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/compile/CompileScriptExpression.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/compile/CompileScriptExpression.java index ae694a84..c891f614 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/compile/CompileScriptExpression.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/compile/CompileScriptExpression.java @@ -62,7 +62,7 @@ public CompileScriptExpression(ScriptExpression scriptExpression, FunctionConfig int startIndex = 0; if (this.containsContext) { parameterTemplete[0] = new Message(new JSONObject()); - parameterTemplete[1] = new Context((IMessage)parameterTemplete[0]); + parameterTemplete[1] = new Context((IMessage) parameterTemplete[0]); startIndex = 2; } @@ -105,10 +105,10 @@ public Object execute(IMessage message, FunctionContext context) { Entry entry = it.next(); Integer index = entry.getKey(); CompileParameter compileParameter = entry.getValue(); - parameters[index] = functionConfigure.getRealValue(index,compileParameter.getValue(message, context)); + parameters[index] = functionConfigure.getRealValue(index, compileParameter.getValue(message, context)); } } - Object value = scriptExpression.executeFunctionConfigue(message,context,functionConfigure,parameters); + Object value = scriptExpression.executeFunctionConfigue(message, context, functionConfigure, parameters); if (isSimpleNewFieldName && value != null) { message.getMessageBody().put(scriptExpression.getNewFieldName(), value); } else { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/IScriptOptimization.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/IScriptOptimization.java index bf088fce..4511a357 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/IScriptOptimization.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/IScriptOptimization.java @@ -31,17 +31,26 @@ */ public interface IScriptOptimization { + static String getParameterValue(IScriptParamter scriptParamter) { + if (!ScriptParameter.class.isInstance(scriptParamter)) { + return null; + } + ScriptParameter parameter = (ScriptParameter) scriptParamter; + if (parameter.getRigthVarName() != null) { + return null; + } + return FunctionUtils.getConstant(parameter.getLeftVarName()); + } + /** * compile expression list to improve performance - * @param expressions IScriptExpression list of FunctionScript + * + * @param expressions IScriptExpression list of FunctionScript * @param functionScript functionScript object * @return the executor to to improve performance */ IOptimizationCompiler compile(List expressions, IConfigurableIdentification functionScript); - - - /** * the executor can execute expression and return result */ @@ -49,27 +58,16 @@ interface IOptimizationExecutor { FilterResultCache execute(IMessage message, AbstractContext context); } - /** * the executor can execute expression and return result */ - interface IOptimizationCompiler extends IOptimizationExecutor{ + interface IOptimizationCompiler extends IOptimizationExecutor { /** * the IScriptExpression list after Optimization compile + * * @return */ List getOptimizationExpressionList(); } - - static String getParameterValue(IScriptParamter scriptParamter) { - if (!ScriptParameter.class.isInstance(scriptParamter)) { - return null; - } - ScriptParameter parameter = (ScriptParameter) scriptParamter; - if (parameter.getRigthVarName() != null) { - return null; - } - return FunctionUtils.getConstant(parameter.getLeftVarName()); - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/parser/ScriptExpressionParserFactory.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/parser/ScriptExpressionParserFactory.java index f45940de..b105ba80 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/parser/ScriptExpressionParserFactory.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/parser/ScriptExpressionParserFactory.java @@ -31,9 +31,6 @@ public class ScriptExpressionParserFactory implements IScriptExpressionParser { private static List parserList = new ArrayList<>(); - private ScriptExpressionParserFactory() { - } - static { INSTANCE = new ScriptExpressionParserFactory(); parserList.add(new ConditionExpressionParser("if", "{", "else")); @@ -41,6 +38,9 @@ private ScriptExpressionParserFactory() { parserList.add(new FunctionExpressionParser()); } + private ScriptExpressionParserFactory() { + } + public static ScriptExpressionParserFactory getInstance() { return INSTANCE; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/parser/imp/ConditionExpressionParser.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/parser/imp/ConditionExpressionParser.java index 8dd500a2..cbda3e53 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/parser/imp/ConditionExpressionParser.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/parser/imp/ConditionExpressionParser.java @@ -33,10 +33,8 @@ public class ConditionExpressionParser implements IScriptExpressionParser { public final static String LINE_SEPERATOR = System.getProperty("line.separator"); - - private List keywords = new ArrayList<>(); - private static final String ELSEIF = "elseif"; + private List keywords = new ArrayList<>(); public ConditionExpressionParser(String... conditonKeywords) { if (conditonKeywords == null) { @@ -56,7 +54,7 @@ public GroupScriptExpression parse(String itemStr) { } protected GroupScriptExpression parse(String itemStr, GroupScriptExpression current, GroupScriptExpression root, - Map flag2ExpressionStr) { + Map flag2ExpressionStr) { int startIndex = 0; List list = new ArrayList<>(); @@ -152,7 +150,7 @@ protected int getElseIndex(String expressionStr) { } private List parseScriptExpression(String expressionStr, - Map flag2ExpressionStr) { + Map flag2ExpressionStr) { expressionStr = expressionStr.trim(); if (expressionStr.startsWith("if((")) { expressionStr = expressionStr.replace("if((", "if('("); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/parser/imp/FunctionExpressionParser.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/parser/imp/FunctionExpressionParser.java index 1eb1f75e..bce777c9 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/parser/imp/FunctionExpressionParser.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/parser/imp/FunctionExpressionParser.java @@ -95,7 +95,7 @@ public boolean support(String itemStr) { } protected void parseExpression(ScriptExpression scriptExpression, String expressionStr, - Map flag2ExpressionStr) { + Map flag2ExpressionStr) { if (expressionStr == null) { return; } @@ -200,7 +200,7 @@ public String getScriptParameterStr() { }); } else { - parameters = parseParameter(parameterStr, flag2ExpressionStr,0); + parameters = parseParameter(parameterStr, flag2ExpressionStr, 0); } scriptExpression.setFunctionName(functionName.trim()); scriptExpression.setParameters(parameters); @@ -251,7 +251,7 @@ private boolean isFieldParamter(String expressionStr) { * * @param parameterStr */ - protected List parseParameter(String parameterStr, Map flag2ExpressionStr,int flag) { + protected List parseParameter(String parameterStr, Map flag2ExpressionStr, int flag) { if (StringUtil.isEmpty(parameterStr)) { return new ArrayList(); } @@ -275,7 +275,7 @@ protected List parseParameter(String parameterStr, Map parameterMap, - List parameters) { + List parameters) { for (int i = 0; i < parameters.size(); i++) { IScriptParamter scriptParameter = parameters.get(i); if (IScriptExpression.class.isInstance(scriptParameter)) { - IScriptExpression expression = (IScriptExpression)scriptParameter; + IScriptExpression expression = (IScriptExpression) scriptParameter; List parameterList = expression.getScriptParamters(); rebackExpressionParameters(parameterMap, parameterList); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/parser/imp/FunctionParser.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/parser/imp/FunctionParser.java index 3c0497a0..0844ed03 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/parser/imp/FunctionParser.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/parser/imp/FunctionParser.java @@ -44,6 +44,17 @@ public static FunctionParser getInstance() { return functionParser; } + public static void main(String[] args) { + String script = + "if(min_day_prediction>=prediction_day){prediction_day=min_day_prediction;}else{echo();};" + + "until_day=datefirst(now,'dd');"; + + FunctionParser functionParser = new FunctionParser(); + Map map = new HashMap(); + String result = functionParser.doConditionParser(script, map, 1); + System.out.println(result); + } + public List parse(String value) { if (StringUtil.isEmpty(value)) { return new ArrayList<>(); @@ -151,15 +162,4 @@ private int paserFunctionFirstBracketesIndex(String parameterStr) { } return -1; } - - public static void main(String[] args) { - String script = - "if(min_day_prediction>=prediction_day){prediction_day=min_day_prediction;}else{echo();};" - + "until_day=datefirst(now,'dd');"; - - FunctionParser functionParser = new FunctionParser(); - Map map = new HashMap(); - String result = functionParser.doConditionParser(script, map, 1); - System.out.println(result); - } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/impl/ScriptServiceImpl.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/impl/ScriptServiceImpl.java index fea38717..1fe8127d 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/impl/ScriptServiceImpl.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/impl/ScriptServiceImpl.java @@ -34,6 +34,16 @@ import org.apache.rocketmq.streams.script.service.IScriptService; public class ScriptServiceImpl implements IScriptService { + /** + * 创建软引用缓存,尽量缓存script,减少解析成本,当内存不足时自动回收 + */ + private static ICache cache = + new SoftReferenceCache<>(script -> { + FunctionScript functionScript = new FunctionScript(); + functionScript.setScript(script); + functionScript.init(); + return functionScript; + }); /** * 所有实现IScriptInit接口的对象和method */ @@ -46,7 +56,7 @@ public ScriptServiceImpl() { @Override public List executeScript(IMessage message, FunctionContext context, AbstractScript, FunctionContext> script) { script.doMessage(message, context); - if(context.isSplitModel()){ + if (context.isSplitModel()) { return context.getSplitMessages(); } List messages = new ArrayList<>(); @@ -54,17 +64,6 @@ public List executeScript(IMessage message, FunctionContext context, A return messages; } - /** - * 创建软引用缓存,尽量缓存script,减少解析成本,当内存不足时自动回收 - */ - private static ICache cache = - new SoftReferenceCache<>(script -> { - FunctionScript functionScript = new FunctionScript(); - functionScript.setScript(script); - functionScript.init(); - return functionScript; - }); - @Override public List executeScript(final JSONObject jsonObject, String script) { FunctionScript functionScript = cache.get(script); @@ -95,7 +94,7 @@ public List executeScript(final JSONObject jsonObject, AbstractScript< @Override public void scanPackages(String... packageNames) { - functionService.scanePackages(packageNames); + functionService.scanPackages(packageNames); } @Override diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/SimpleUDAFScript.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/SimpleUDAFScript.java index 936df8f3..da787d7a 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/SimpleUDAFScript.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/SimpleUDAFScript.java @@ -16,11 +16,9 @@ */ package org.apache.rocketmq.streams.script.service.udf; -import java.io.Serializable; - public class SimpleUDAFScript extends UDAFScript { - public SimpleUDAFScript(){ + public SimpleUDAFScript() { this.accumulateMethodName = "accumulate"; this.createAccumulatorMethodName = "createAccumulator"; this.getValueMethodName = "getValue"; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/UDAFScript.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/UDAFScript.java index 9301521e..089f7e23 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/UDAFScript.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/UDAFScript.java @@ -94,7 +94,7 @@ public void initMethod() { public ACC createAccumulator() { FunctionConfigure functionConfigure = getCreateAccumulatorFunctionConfigure(); if (functionConfigure != null) { - return (ACC)functionConfigure.execute(new Object[0]); + return (ACC) functionConfigure.execute(new Object[0]); } return null; } @@ -103,7 +103,7 @@ public ACC createAccumulator() { public T getValue(ACC accumulator) { FunctionConfigure functionConfigure = getGetValueFunctionConfigure(); if (functionConfigure != null) { - return (T)functionConfigure.execute(new Object[] {accumulator}); + return (T) functionConfigure.execute(new Object[] {accumulator}); } return null; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/UDFScript.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/UDFScript.java index c8f0e927..149e0e9d 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/UDFScript.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/UDFScript.java @@ -16,15 +16,21 @@ */ package org.apache.rocketmq.streams.script.service.udf; +import com.aliyun.oss.OSSClient; +import com.google.common.collect.Lists; import java.io.File; import java.lang.reflect.Method; import java.net.URL; import java.net.URLClassLoader; +import java.util.Arrays; +import java.util.Date; import java.util.List; import java.util.Map; -import org.apache.commons.logging.LogFactory; +import org.apache.commons.lang3.time.DateUtils; +import org.apache.rocketmq.streams.common.component.ComponentCreator; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.interfaces.IUDF; import org.apache.rocketmq.streams.common.topology.model.AbstractScript; import org.apache.rocketmq.streams.common.utils.FileUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; @@ -33,17 +39,16 @@ import org.apache.rocketmq.streams.script.function.model.FunctionConfigure; import org.apache.rocketmq.streams.script.function.model.FunctionType; import org.apache.rocketmq.streams.script.function.service.IFunctionService; -import org.apache.rocketmq.streams.script.service.IScriptUDFInit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * 主要是为了兼容外部的udf,或者把任意java的方法发布成函数 */ -public class UDFScript extends AbstractScript implements IScriptUDFInit { - - private transient ScriptComponent scriptComponent = ScriptComponent.getInstance(); - - private static final org.apache.commons.logging.Log LOG = LogFactory.getLog(UDFScript.class); +public class UDFScript extends AbstractScript implements IUDF { + private static final Logger LOGGER = LoggerFactory.getLogger(UDFScript.class); + private final transient ScriptComponent scriptComponent = ScriptComponent.getInstance(); protected transient Object instance; protected String fullClassName;//类的全限定名 @@ -62,24 +67,40 @@ public class UDFScript extends AbstractScript implements IScriptUDFInit { * init method对应的参数 */ protected transient Object[] initParameters = null; - + protected String jarPath; /** * 是否完成初始化 */ private transient volatile boolean hasInit = false; + protected static File createFileSupportResourceFile(String fileUrl) { + if (fileUrl.startsWith(FileUtil.CLASS_PATH_FILE_HEADER)) { + fileUrl = fileUrl.replaceFirst(FileUtil.CLASS_PATH_FILE_HEADER, ""); + return FileUtil.getResourceFile(fileUrl); + } else if (fileUrl.startsWith(FileUtil.LOCAL_FILE_HEADER)) { + fileUrl = fileUrl.replaceFirst(FileUtil.LOCAL_FILE_HEADER, ""); + return new File(fileUrl); + } else { + return new File(fileUrl); + } + } + /** * Configuable的初始化方法,由框架执行,执行时自动完成注册 * * @return */ - @Override - protected boolean initConfigurable() { + @Override protected boolean initConfigurable() { + loadUDFInner(); + + return true; + } + + private void loadUDFInner() { registFunctionSerivce(scriptComponent.getFunctionService()); - FunctionConfigure functionConfigure = - scriptComponent.getFunctionService().getFunctionConfigure(createInitMethodName(), this.initParameters); + FunctionConfigure functionConfigure = scriptComponent.getFunctionService().getFunctionConfigure(createInitMethodName(), this.initParameters); if (functionConfigure == null) { - return true; + return; } if (initParameters != null) { functionConfigure.execute(initParameters); @@ -91,13 +112,14 @@ protected boolean initConfigurable() { } functionConfigure.execute(paras); } - return true; + } + + @Override public void loadUDF() { + } /** * 完成函数的注册 - * - * @param iFunctionService */ protected void registFunctionSerivce(IFunctionService iFunctionService) { if (StringUtil.isEmpty(functionName) || hasInit) { @@ -126,8 +148,7 @@ protected void registFunctionSerivce(IFunctionService iFunctionService) { } } - @Override - public void destroy() { + @Override public void destroy() { super.destroy(); FunctionConfigure functionConfigure = scriptComponent.getFunctionService().getFunctionConfigure(getCloseMethodName()); if (functionConfigure == null) { @@ -143,83 +164,70 @@ public void destroy() { /** * 在加载时,初始化对象。应该支持,本地class load,从文件load和从远程下载。远程下载部分待测试 - * - * @param iFunctionService - * @return */ protected boolean initBeanClass(IFunctionService iFunctionService) { ClassLoader classLoader = this.getClass().getClassLoader(); - Class clazz; + Class clazz; try { clazz = classLoader.loadClass(fullClassName); instance = clazz.newInstance(); - return true; } catch (Exception e) { - e.printStackTrace(); - } - try { - String jarUrl = getValue(); -// if (StringUtil.isEmpty(jarUrl)) { -// clazz = classLoader.loadClass(fullClassName); -// instance = clazz.newInstance(); -// return true; -// } - URL url = null; - if (isURL) { - url = new URL(getValue()); - } else { - File file = null; - if (StringUtil.isNotEmpty(jarUrl)) { - if (jarUrl.startsWith("/")) { - file = new File(jarUrl); - } else { - file = downLoadFile(jarUrl); + try { + String jarUrl = getValue(); + URL url = null; + if (isURL) { + url = new URL(getValue()); + } else { + if (StringUtil.isNotEmpty(jarUrl)) { + if (jarUrl.startsWith("/")) { + File file = new File(jarUrl); + url = new URL("file", null, file.getCanonicalPath()); + } else if (jarUrl.startsWith("http://") || jarUrl.startsWith("https://")) { + url = new URL(jarUrl); + } else if (jarUrl.startsWith("oss://")) { + String ossUrl = jarUrl.substring(6); //url以oss://开头 + String accessKeyId = configuration.getProperty(ComponentCreator.UDF_JAR_OSS_ACCESS_ID); + String accesskeySecurity = configuration.getProperty(ComponentCreator.UDF_JAR_OSS_ACCESS_KEY); + + String[] ossInfo = ossUrl.split("/"); + String endPoint = ossInfo.length > 0 ? ossInfo[0] : ""; + String bucketName = ossInfo.length > 1 ? ossInfo[1] : ""; + List objectNames = ossInfo.length > 2 ? Arrays.asList(ossInfo[2].split(",")) : Lists.newArrayList(); + + OSSClient ossClient = new OSSClient(endPoint, accessKeyId, accesskeySecurity); + if (objectNames.size() > 0) { + url = ossClient.generatePresignedUrl(bucketName, objectNames.get(0), DateUtils.addMinutes(new Date(), 30)); + } + } else { + File file = downLoadFile(jarUrl); + url = new URL("file", null, file.getCanonicalPath()); + } } - if (file == null) { - throw new RuntimeException("can not file jar file"); - } - url = new URL("file", null, file.getCanonicalPath()); - } - } - - URL[] urls = new URL[] {url}; - URLClassLoader urlClassLoader = new URLClassLoader(urls, classLoader); - classLoader = urlClassLoader; - clazz = classLoader.loadClass(fullClassName); - instance = clazz.newInstance(); + URL[] urls = new URL[] {url}; + classLoader = new URLClassLoader(urls, classLoader); - } catch (Exception e) { - LOG.error("加载异常," + e.getMessage(), e); - return false; + clazz = classLoader.loadClass(fullClassName); + instance = clazz.newInstance(); + } catch (Exception ex) { + LOGGER.error("加载异常," + ex.getMessage(), ex); + return false; + } } return true; } protected File downLoadFile(String fileUrl) { if (fileUrl.startsWith(FileUtil.LOCAL_FILE_HEADER)) { - File file = createFileSupportResourceFile(fileUrl); - return file; + return createFileSupportResourceFile(fileUrl); } else if (fileUrl.startsWith(FileUtil.CLASS_PATH_FILE_HEADER)) { return createFileSupportResourceFile(fileUrl); } return null; } - protected static File createFileSupportResourceFile(String fileUrl) { - if (fileUrl.startsWith(FileUtil.CLASS_PATH_FILE_HEADER)) { - fileUrl = fileUrl.replaceFirst(FileUtil.CLASS_PATH_FILE_HEADER, ""); - return FileUtil.getResourceFile(fileUrl); - } else if (fileUrl.startsWith(FileUtil.LOCAL_FILE_HEADER)) { - fileUrl = fileUrl.replaceFirst(FileUtil.LOCAL_FILE_HEADER, ""); - return new File(fileUrl); - } else { - return new File(fileUrl); - } - } - protected String createInitMethodName() { return MapKeyUtil.createKey(functionName, initMethodName); } @@ -244,8 +252,7 @@ public void setMethodName(String methodName) { this.methodName = methodName; } - @Override - public String getInitMethodName() { + @Override public String getInitMethodName() { return initMethodName; } @@ -257,8 +264,7 @@ protected FunctionType getFunctionType() { return FunctionType.UDF; } - @Override - public Object getInstance() { + @Override public Object getInstance() { return instance; } @@ -270,6 +276,10 @@ public String getFunctionName() { return functionName; } + public void setFunctionName(String functionName) { + this.functionName = functionName; + } + public boolean isURL() { return isURL; } @@ -278,12 +288,7 @@ public void setURL(boolean URL) { isURL = URL; } - public void setFunctionName(String functionName) { - this.functionName = functionName; - } - - @Override - public Object doMessage(IMessage channelMessage, AbstractContext context) { + @Override public Object doMessage(IMessage channelMessage, AbstractContext context) { return instance; } @@ -295,13 +300,19 @@ public void setCloseMethodName(String closeMethodName) { this.closeMethodName = closeMethodName; } - @Override - public List getScriptsByDependentField(String fieldName) { + @Override public List getScriptsByDependentField(String fieldName) { throw new RuntimeException("can not support this method:getScriptsByDependentField"); } - @Override - public Map> getDependentFields() { + @Override public Map> getDependentFields() { return null; } + + public String getJarPath() { + return jarPath; + } + + public void setJarPath(String jarPath) { + this.jarPath = jarPath; + } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/utils/FunctionUtils.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/utils/FunctionUtils.java index 9bcc1b82..149df88c 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/utils/FunctionUtils.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/utils/FunctionUtils.java @@ -27,6 +27,11 @@ public class FunctionUtils { private static final String FLAG_PREFIX = "constants_";//常量flag的前缀 + private final static String LONG = "^[-\\+]?[\\d]*$"; // 整数 + private final static String DOUBLE = "^[-\\+]?[\\d]*[\\.]?[\\d]*$"; + + private static final String CONTEXT = "'${context}'"; + private static DateDataType dateDataType = new DateDataType(); public static boolean isField(String fieldName) { return !isConstantOrNumber(fieldName); @@ -50,23 +55,16 @@ public static boolean isConstant(String fieldName) { if (fieldName.startsWith("'") && fieldName.endsWith("'")) { return true; } - if (fieldName.startsWith("\"") && fieldName.endsWith("\"")) { - return true; - } - return false; + return fieldName.startsWith("\"") && fieldName.endsWith("\""); } - private final static String LONG = "^[-\\+]?[\\d]*$"; // 整数 - private final static String DOUBLE = "^[-\\+]?[\\d]*[\\.]?[\\d]*$"; - public static boolean isNumberObject(Object object) { - if (Long.class.isInstance(object)) { - return true; - } else if (Double.class.isInstance(object)) { + if (object instanceof Long) { return true; + } else { + return object instanceof Double; } - return false; } public static boolean isNumber(String fieldName) { @@ -80,16 +78,14 @@ public static boolean isLong(String fieldName) { if (StringUtil.isEmpty(fieldName)) { return false; } - boolean match = StringUtil.matchRegex(fieldName, LONG); - return match; + return StringUtil.matchRegex(fieldName, LONG); } public static boolean isDouble(String fieldName) { if (StringUtil.isEmpty(fieldName)) { return false; } - boolean match = StringUtil.matchRegex(fieldName, DOUBLE); - return match; + return StringUtil.matchRegex(fieldName, DOUBLE); } public static boolean isBoolean(String fieldName) { @@ -97,10 +93,7 @@ public static boolean isBoolean(String fieldName) { return false; } String value = fieldName.toLowerCase(); - if ("true".equals(value) || "false".equals(value)) { - return true; - } - return false; + return "true".equals(value) || "false".equals(value); } public static Long getLong(String fieldName) { @@ -130,7 +123,16 @@ public static String getConstant(String fieldName) { } public static Object getValue(IMessage message, AbstractContext context, String fieldName) { - Object value = getFiledValue(message, context, fieldName); + if (CONTEXT.equalsIgnoreCase(fieldName)) { + return message.getMessageBody(); + } + if ("'*'".equalsIgnoreCase(fieldName)) { + return message.getMessageBody(); + } + if ("'\\*'".equalsIgnoreCase(fieldName)) { + fieldName = "'*'"; + } + Object value = getFiledValue(message, fieldName); if (value != null) { return value; } @@ -150,26 +152,23 @@ public static Object getValue(IMessage message, AbstractContext context, String } } - public static Object getFiledValue(IMessage message, AbstractContext context, String fieldName) { - if (context == null || message == null) { + public static Object getFiledValue(IMessage message, String fieldName) { + if (message == null) { return fieldName; } - if (IgnoreMessage.class.isInstance(message)) { + if (message instanceof IgnoreMessage) { return fieldName; } - Object value = message.getMessageBody().get(fieldName); - return value; + return message.getMessageBody().get(fieldName); } - private static DateDataType dateDataType = new DateDataType(); - public static String getValueString(IMessage message, AbstractContext context, String fieldName) { Object value = getValue(message, context, fieldName); if (value == null) { return null; } - if (String.class.isInstance(value)) { + if (value instanceof String) { return (String) value; } if (dateDataType.matchClass(value.getClass())) { @@ -244,7 +243,7 @@ private static String doPreConstants(String scriptOrExpression, Map> fieldConditon, - boolean isAndRelation) { + boolean isAndRelation) { Iterator>> it = fieldConditon.entrySet().iterator(); while (it.hasNext()) { Map.Entry> entry = it.next(); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/utils/RandomStrUtil.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/utils/RandomStrUtil.java index b3195edd..6178d08d 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/utils/RandomStrUtil.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/utils/RandomStrUtil.java @@ -20,12 +20,12 @@ public class RandomStrUtil { + private static final SecureRandom random = new SecureRandom(); private static char[] digits = - {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', + { + '0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o', 'p', 'q', 'r', 's', 't', 'u', 'v', 'w', 'x', 'y', 'z'}; - private static final SecureRandom random = new SecureRandom(); - public static String getRandomStr(int length) { StringBuffer sb = new StringBuffer(); for (int i = 0; i < length; i++) { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/utils/UniqID.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/utils/UniqID.java index 17cf1834..34de97e7 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/utils/UniqID.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/utils/UniqID.java @@ -22,12 +22,10 @@ public class UniqID { - private static char[] digits = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f'}; - private static UniqID me = new UniqID(); - private static final ThreadLocal digestLocal = new ThreadLocal(); - private static final SecureRandom random = new SecureRandom(); + private static char[] digits = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f'}; + private static UniqID me = new UniqID(); private UniqID() { diff --git a/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/FunctionTest.java b/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/FunctionTest.java index 75a0d31b..042a9a00 100644 --- a/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/FunctionTest.java +++ b/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/FunctionTest.java @@ -45,33 +45,33 @@ public void testCast() { @Test public void testDate() { JSONObject message = new JSONObject(); - message.put("name", "Chris"); - message.put("age", 18); - message.put("from", "2019-07-14 00:00:00"); - message.put("last", "2019-07-14 01:00:00"); - message.put("event_type", "alert"); - String scriptValue = "now=now();x =concat(from,last)"; + message.put("a", 1); + message.put("b", 0); + message.put("c", 2); + message.put("g", -1); + message.put("d", 10); + String scriptValue = "x=d+((a-b)/c+8)+(c-abs(g))%7;"; List list = ScriptComponent.getInstance().getService().executeScript(message, scriptValue); for (int i = 0; i < list.size(); i++) { - assertTrue(list.get(i).getMessageBody().getString("from").equals("2019-07-14 00:00:00")); System.out.println(list.get(i).getMessageBody()); } } + @Test - public void testJSON(){ - JSONObject jsonObject=new JSONObject(); - JSONObject person=new JSONObject(); - person.put("name","chris"); - person.put("age",18); - jsonObject.put("persion",person); - JSONArray jsonArray=new JSONArray(); - for(int i=0;i<3;i++){ - JSONObject jsonObject1=new JSONObject(); - jsonObject1.put("address","address"+i); + public void testJSON() { + JSONObject jsonObject = new JSONObject(); + JSONObject person = new JSONObject(); + person.put("name", "chris"); + person.put("age", 18); + jsonObject.put("persion", person); + JSONArray jsonArray = new JSONArray(); + for (int i = 0; i < 3; i++) { + JSONObject jsonObject1 = new JSONObject(); + jsonObject1.put("address", "address" + i); jsonArray.add(jsonObject1); } - jsonObject.put("addresses",jsonArray); + jsonObject.put("addresses", jsonArray); System.out.println(JsonableUtil.formatJson(jsonObject)); } diff --git a/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/aggregation/AccumulatorTest.java b/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/aggregation/AccumulatorTest.java index 965b24fb..fa00bdba 100644 --- a/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/aggregation/AccumulatorTest.java +++ b/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/aggregation/AccumulatorTest.java @@ -56,7 +56,7 @@ public void testAverageAccumulator() { Assert.assertEquals(1.75, averageAccum.value); String objectValue = ReflectUtil.serializeObject(averageAccum).toJSONString(); JSONObject objectJson = JSONObject.parseObject(objectValue); - AverageAccum reflectAvgAccum = (AverageAccum)ReflectUtil.deserializeObject(objectJson); + AverageAccum reflectAvgAccum = (AverageAccum) ReflectUtil.deserializeObject(objectJson); Assert.assertEquals(1.75, reflectAvgAccum.value); averageAccum = averageAccumulator.createAccumulator(); @@ -64,7 +64,7 @@ public void testAverageAccumulator() { averageAccumulator.accumulate(averageAccum, 2); objectValue = ReflectUtil.serializeObject(averageAccum).toJSONString(); objectJson = JSONObject.parseObject(objectValue); - reflectAvgAccum = (AverageAccum)ReflectUtil.deserializeObject(objectJson); + reflectAvgAccum = (AverageAccum) ReflectUtil.deserializeObject(objectJson); Assert.assertEquals(1.5, reflectAvgAccum.value); averageAccum = averageAccumulator.createAccumulator(); @@ -72,7 +72,7 @@ public void testAverageAccumulator() { averageAccumulator.accumulate(averageAccum, 2); objectValue = ReflectUtil.serializeObject(averageAccum).toJSONString(); objectJson = JSONObject.parseObject(objectValue); - reflectAvgAccum = (AverageAccum)ReflectUtil.deserializeObject(objectJson); + reflectAvgAccum = (AverageAccum) ReflectUtil.deserializeObject(objectJson); Assert.assertEquals(2, reflectAvgAccum.value); averageAccum = averageAccumulator.createAccumulator(); @@ -81,7 +81,7 @@ public void testAverageAccumulator() { Assert.assertEquals(2, averageAccum.value); objectValue = ReflectUtil.serializeObject(averageAccum).toJSONString(); objectJson = JSONObject.parseObject(objectValue); - reflectAvgAccum = (AverageAccum)ReflectUtil.deserializeObject(objectJson); + reflectAvgAccum = (AverageAccum) ReflectUtil.deserializeObject(objectJson); Assert.assertEquals(2, reflectAvgAccum.value); } @@ -93,7 +93,7 @@ public void testSumAccumulator() { sumAccumulator.accumulate(sumAccum, 2.5); String dbValue = ReflectUtil.serializeObject(sumAccum).toJSONString(); JSONObject memObject = JSONObject.parseObject(dbValue); - SumAccum reflectSumAccum = (SumAccum)ReflectUtil.deserializeObject(memObject); + SumAccum reflectSumAccum = (SumAccum) ReflectUtil.deserializeObject(memObject); Assert.assertEquals(3.5, reflectSumAccum.sum); sumAccum = sumAccumulator.createAccumulator(); @@ -101,7 +101,7 @@ public void testSumAccumulator() { sumAccumulator.accumulate(sumAccum, 2); dbValue = ReflectUtil.serializeObject(sumAccum).toJSONString(); memObject = JSONObject.parseObject(dbValue); - reflectSumAccum = (SumAccum)ReflectUtil.deserializeObject(memObject); + reflectSumAccum = (SumAccum) ReflectUtil.deserializeObject(memObject); Assert.assertEquals(3, reflectSumAccum.sum); } @@ -113,7 +113,7 @@ public void testMinAccumulator() { minAccumulator.accumulate(minAccum, 12); String dbAccum = ReflectUtil.serializeObject(minAccum).toJSONString(); JSONObject memObject = JSONObject.parseObject(dbAccum); - MinAccum reflectMinAccum = (MinAccum)ReflectUtil.deserializeObject(memObject); + MinAccum reflectMinAccum = (MinAccum) ReflectUtil.deserializeObject(memObject); //TODO 序列化过程中的value都被转成了string 如果碰到这个问题 暂时在sql里加一个类型转换解决 Assert.assertEquals("12", reflectMinAccum.min); @@ -122,7 +122,7 @@ public void testMinAccumulator() { minAccumulator.accumulate(minAccum, "2021-06-18 12:00:00"); dbAccum = ReflectUtil.serializeObject(minAccum).toJSONString(); memObject = JSONObject.parseObject(dbAccum); - reflectMinAccum = (MinAccum)ReflectUtil.deserializeObject(memObject); + reflectMinAccum = (MinAccum) ReflectUtil.deserializeObject(memObject); Assert.assertEquals("2021-06-16 12:00:00", reflectMinAccum.min); minAccum = minAccumulator.createAccumulator(); @@ -130,7 +130,7 @@ public void testMinAccumulator() { minAccumulator.accumulate(minAccum, "127.0.0.2"); dbAccum = ReflectUtil.serializeObject(minAccum).toJSONString(); memObject = JSONObject.parseObject(dbAccum); - reflectMinAccum = (MinAccum)ReflectUtil.deserializeObject(memObject); + reflectMinAccum = (MinAccum) ReflectUtil.deserializeObject(memObject); Assert.assertEquals("127.0.0.1", reflectMinAccum.min); } @@ -142,7 +142,7 @@ public void testMaxAccumulator() { maxAccumulator.accumulate(maxAccum, 12); String dbAccum = ReflectUtil.serializeObject(maxAccum).toJSONString(); JSONObject memObject = JSONObject.parseObject(dbAccum); - MaxAccum reflectMaxAccum = (MaxAccum)ReflectUtil.deserializeObject(memObject); + MaxAccum reflectMaxAccum = (MaxAccum) ReflectUtil.deserializeObject(memObject); //TODO 序列化过程中的value都被转成了string 如果碰到这个问题 暂时在sql里加一个类型转换解决 Assert.assertEquals("13", reflectMaxAccum.max); @@ -151,7 +151,7 @@ public void testMaxAccumulator() { maxAccumulator.accumulate(maxAccum, "2021-06-18 12:00:00"); dbAccum = ReflectUtil.serializeObject(maxAccum).toJSONString(); memObject = JSONObject.parseObject(dbAccum); - reflectMaxAccum = (MaxAccum)ReflectUtil.deserializeObject(memObject); + reflectMaxAccum = (MaxAccum) ReflectUtil.deserializeObject(memObject); Assert.assertEquals("2021-06-18 12:00:00", reflectMaxAccum.max); maxAccum = maxAccumulator.createAccumulator(); @@ -159,7 +159,7 @@ public void testMaxAccumulator() { maxAccumulator.accumulate(maxAccum, "127.0.0.2"); dbAccum = ReflectUtil.serializeObject(maxAccum).toJSONString(); memObject = JSONObject.parseObject(dbAccum); - reflectMaxAccum = (MaxAccum)ReflectUtil.deserializeObject(memObject); + reflectMaxAccum = (MaxAccum) ReflectUtil.deserializeObject(memObject); Assert.assertEquals("127.0.0.2", reflectMaxAccum.max); } @@ -172,7 +172,7 @@ public void testConcatAccumulator() { concatAccumulator.accumulate(concatAccum, "listener"); String dbValue = ReflectUtil.serializeObject(concatAccum).toJSONString(); JSONObject memObject = JSONObject.parseObject(dbValue); - ConcatAccum reflectConcatAccum = (ConcatAccum)ReflectUtil.deserializeObject(memObject); + ConcatAccum reflectConcatAccum = (ConcatAccum) ReflectUtil.deserializeObject(memObject); Assert.assertEquals("be,a,listener", concatAccumulator.getValue(reflectConcatAccum)); concatAccum = concatAccumulator.createAccumulator(); @@ -181,7 +181,7 @@ public void testConcatAccumulator() { concatAccumulator.accumulate(concatAccum, " ", "listener"); dbValue = ReflectUtil.serializeObject(concatAccum).toJSONString(); memObject = JSONObject.parseObject(dbValue); - reflectConcatAccum = (ConcatAccum)ReflectUtil.deserializeObject(memObject); + reflectConcatAccum = (ConcatAccum) ReflectUtil.deserializeObject(memObject); Assert.assertEquals("be a listener", concatAccumulator.getValue(reflectConcatAccum)); } @@ -199,7 +199,7 @@ public void testConcatDistinctAccumulator() { concatDistinctAccumulator.accumulate(concatDistinctAccum, "code"); String dbValue = ReflectUtil.serializeObject(concatDistinctAccum).toJSONString(); JSONObject memObject = JSONObject.parseObject(dbValue); - ConcatDistinctAccum reflectConcatAccum = (ConcatDistinctAccum)ReflectUtil.deserializeObject(memObject); + ConcatDistinctAccum reflectConcatAccum = (ConcatDistinctAccum) ReflectUtil.deserializeObject(memObject); Assert.assertEquals("code,show,me,talk,is,cheap", concatDistinctAccumulator.getValue(reflectConcatAccum)); concatDistinctAccum = concatDistinctAccumulator.createAccumulator(); @@ -213,7 +213,7 @@ public void testConcatDistinctAccumulator() { concatDistinctAccumulator.accumulate(concatDistinctAccum, " ", "code"); dbValue = ReflectUtil.serializeObject(concatDistinctAccum).toJSONString(); memObject = JSONObject.parseObject(dbValue); - reflectConcatAccum = (ConcatDistinctAccum)ReflectUtil.deserializeObject(memObject); + reflectConcatAccum = (ConcatDistinctAccum) ReflectUtil.deserializeObject(memObject); Assert.assertEquals("code show me talk is cheap", concatDistinctAccumulator.getValue(reflectConcatAccum)); } @@ -227,7 +227,7 @@ public void testDistinctAccumulator() { distinctAccumulator.accumulate(distinctAccum, "pua"); String dbValue = ReflectUtil.serializeObject(distinctAccum).toJSONString(); JSONObject memObject = JSONObject.parseObject(dbValue); - DistinctAccum reflectDistinctAccum = (DistinctAccum)ReflectUtil.deserializeObject(memObject); + DistinctAccum reflectDistinctAccum = (DistinctAccum) ReflectUtil.deserializeObject(memObject); Assert.assertEquals(3, distinctAccumulator.getValue(reflectDistinctAccum).size()); } diff --git a/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/function/JavaObjectUDFFunction.java b/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/function/JavaObjectUDFFunction.java index eada4f5d..ea4bcb66 100644 --- a/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/function/JavaObjectUDFFunction.java +++ b/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/function/JavaObjectUDFFunction.java @@ -25,37 +25,40 @@ public class JavaObjectUDFFunction { /** * 这个是UDF + * * @param person 这个是数据流中的列值 * @return */ - public String eval(Person person){ + public String eval(Person person) { return person.name; } /** * 一个类中,UDF可以多个 + * * @param person * @param needAge * @return */ - public String eval(Person person,boolean needAge){ - return person.name+(needAge?person.age:""); + public String eval(Person person, boolean needAge) { + return person.name + (needAge ? person.age : ""); } /** * 这个是UDTF,UDTF如果 + * * @param person * @param splitSign * @return 返回的是拆分后的多行数据,包含多列, * 每列是一个数据,如果没有key的值,请用f0,f1,f2,fn来当作默认的字段名 */ - public List> eval(Person person,String splitSign){ - String name=person.name; - String[] names=name.split(splitSign); - List> rows=new ArrayList<>(); - for(int i=0;i row=new HashMap<>(); - row.put("f"+i,names[i]); + public List> eval(Person person, String splitSign) { + String name = person.name; + String[] names = name.split(splitSign); + List> rows = new ArrayList<>(); + for (int i = 0; i < names.length; i++) { + Map row = new HashMap<>(); + row.put("f" + i, names[i]); rows.add(row); } return rows; diff --git a/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/function/Person.java b/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/function/Person.java index b55d7b25..f01fe336 100644 --- a/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/function/Person.java +++ b/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/function/Person.java @@ -19,8 +19,9 @@ public class Person { protected String name; protected int age; - public Person(String name,int age){ - this.name=name; - this.age=age; + + public Person(String name, int age) { + this.name = name; + this.age = age; } } diff --git a/rocketmq-streams-script/src/test/resources/log4j.xml b/rocketmq-streams-script/src/test/resources/log4j.xml old mode 100755 new mode 100644 diff --git a/rocketmq-streams-serviceloader/pom.xml b/rocketmq-streams-serviceloader/pom.xml index 60177b2e..a19aca93 100755 --- a/rocketmq-streams-serviceloader/pom.xml +++ b/rocketmq-streams-serviceloader/pom.xml @@ -15,14 +15,14 @@ See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 org.apache.rocketmq rocketmq-streams - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT rocketmq-streams-serviceloader jar diff --git a/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponent.java b/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponent.java index 2dc4d5fc..a74243e5 100644 --- a/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponent.java +++ b/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponent.java @@ -30,12 +30,24 @@ public class ServiceLoaderComponent extends AbstractComponent> implements IServiceLoaderService { + static ServiceLoaderComponent nameLoaderComponent = new ServiceLoaderComponent<>(); + + static { + nameLoaderComponent.init(); + nameLoaderComponent.needServiceName = true; + nameLoaderComponent.startComponent(IServiceNameGetter.class.getName()); + } + private Properties properties; private Class interfaceClass; private Map name2Service = new HashMap<>(); private List serviceList = new ArrayList(); private boolean hasRefresh = false; - private boolean needServieName = true; + private boolean needServiceName = true; + + public static ServiceLoaderComponent getInstance(Class interfaceClass) { + return ComponentCreator.getComponent(interfaceClass.getName(), ServiceLoaderComponent.class); + } @Override public boolean stop() { @@ -47,20 +59,14 @@ public IServiceLoaderService getService() { return this; } - public static ServiceLoaderComponent getInstance(Class interfaceClass) { - ServiceLoaderComponent serviceLoaderComponent = - ComponentCreator.getComponent(interfaceClass.getName(), ServiceLoaderComponent.class); - return serviceLoaderComponent; - } - @Override - protected boolean startComponent(String interfaceClassName) { + protected boolean startComponent(String namespace) { try { - Class clazz = Class.forName(interfaceClassName); - this.interfaceClass = clazz; + Class clazz = Class.forName(namespace); + this.interfaceClass = (Class) clazz; refresh(false); } catch (ClassNotFoundException e) { - throw new RuntimeException("class not found " + interfaceClassName, e); + throw new RuntimeException("class not found " + namespace, e); } return true; } @@ -76,7 +82,7 @@ public T loadService(String serviceName) { if (!this.hasRefresh) { refresh(false); } - return (T)this.name2Service.get(serviceName); + return (T) this.name2Service.get(serviceName); } @Override @@ -100,7 +106,7 @@ public void refresh(boolean forceRefresh) { Iterable iterable = ServiceLoader.load(interfaceClass); List allService = new ArrayList<>(); for (T t : iterable) { - if (needServieName) { + if (needServiceName) { List serviceNames = loadServiceName(t); if (serviceNames == null) { name2Service.put(t.getClass().getSimpleName(), t); @@ -119,24 +125,16 @@ public void refresh(boolean forceRefresh) { } } - static ServiceLoaderComponent nameLoaderComponent = new ServiceLoaderComponent(); - - static { - nameLoaderComponent.init(); - nameLoaderComponent.needServieName = true; - nameLoaderComponent.startComponent(IServiceNameGetter.class.getName()); - } - protected List loadServiceName(T t) { - List serviceNames = new ArrayList(); - Class tClass = t.getClass(); + List serviceNames = new ArrayList<>(); + Class tClass = t.getClass(); String serviceName = properties.getProperty(tClass.getName()); if (properties != null && StringUtil.isNotEmpty(serviceName)) { serviceNames.add(serviceName); return serviceNames; } - ServiceName annotation = (ServiceName)tClass.getAnnotation(ServiceName.class); + ServiceName annotation = (ServiceName) tClass.getAnnotation(ServiceName.class); if (annotation == null) { return null; } @@ -144,7 +142,11 @@ protected List loadServiceName(T t) { serviceNames.add(annotation.value()); } if (StringUtil.isNotEmpty(annotation.aliasName())) { - serviceNames.add(annotation.aliasName()); + String[] names= annotation.aliasName().split(","); + for(String name:names){ + serviceNames.add(name); + } + } if (StringUtil.isNotEmpty(annotation.name())) { diff --git a/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/namefinder/impl/AnnotationServiceNameGetter.java b/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/namefinder/impl/AnnotationServiceNameGetter.java index 71d0eafa..529fc2be 100644 --- a/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/namefinder/impl/AnnotationServiceNameGetter.java +++ b/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/namefinder/impl/AnnotationServiceNameGetter.java @@ -30,7 +30,7 @@ public class AnnotationServiceNameGetter implements IServiceNameGetter { @Override public String getServiceName(Class clazz) { - ServiceName annotation = (ServiceName)clazz.getAnnotation(ServiceName.class); + ServiceName annotation = (ServiceName) clazz.getAnnotation(ServiceName.class); if (annotation == null) { return null; } diff --git a/rocketmq-streams-serviceloader/src/test/java/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponentTest.java b/rocketmq-streams-serviceloader/src/test/java/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponentTest.java index 3a88859a..928bbe9d 100644 --- a/rocketmq-streams-serviceloader/src/test/java/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponentTest.java +++ b/rocketmq-streams-serviceloader/src/test/java/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponentTest.java @@ -27,7 +27,7 @@ public class ServiceLoaderComponentTest { @Test public void testLoadService() { ServiceLoaderComponent serviceLoaderComponent = ServiceLoaderComponent.getInstance(IServiceNameGetter.class); - AnnotationServiceNameGetter getter = (AnnotationServiceNameGetter)serviceLoaderComponent.getService().loadService(AnnotationServiceNameGetter.SERVICE_NAME); + AnnotationServiceNameGetter getter = (AnnotationServiceNameGetter) serviceLoaderComponent.getService().loadService(AnnotationServiceNameGetter.SERVICE_NAME); assertTrue(getter != null && AnnotationServiceNameGetter.class.isInstance(getter)); } } diff --git a/rocketmq-streams-serviceloader/src/test/resources/log4j.xml b/rocketmq-streams-serviceloader/src/test/resources/log4j.xml old mode 100755 new mode 100644 diff --git a/rocketmq-streams-state/pom.xml b/rocketmq-streams-state/pom.xml index 8cd731b5..63bd11bc 100644 --- a/rocketmq-streams-state/pom.xml +++ b/rocketmq-streams-state/pom.xml @@ -15,14 +15,14 @@ See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 org.apache.rocketmq rocketmq-streams - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT rocketmq-streams-state jar diff --git a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/IEntryProcessor.java b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/IEntryProcessor.java index 46292b4a..fa1492af 100644 --- a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/IEntryProcessor.java +++ b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/IEntryProcessor.java @@ -19,7 +19,7 @@ import java.util.Map; -public interface IEntryProcessor{ +public interface IEntryProcessor { void processEntry(Map.Entry entry); } diff --git a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/ISchama.java b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/ISchama.java index d103e5f1..a58d1379 100644 --- a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/ISchama.java +++ b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/ISchama.java @@ -23,8 +23,9 @@ public interface ISchama { /** * split namespace 2 mutil fields, key is field name;value is field Value. * use in db storage + * * @param namespace * @return */ - Map parseSchama(String namespace); + Map parseSchama(String namespace); } diff --git a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/IState.java b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/IState.java index 74043617..e443dfb3 100644 --- a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/IState.java +++ b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/IState.java @@ -22,10 +22,11 @@ import java.util.List; import java.util.Map; import java.util.Set; + /** * state ,save stata data */ -public interface IState { +public interface IState { /** * Returns the number of key-value mappings in this map. If the @@ -52,17 +53,16 @@ public interface IState { * * @param key key whose presence in this map is to be tested * @return true if this map contains a mapping for the specified - * key - * @throws ClassCastException if the key is of an inappropriate type for - * this map - * (optional) + * key + * @throws ClassCastException if the key is of an inappropriate type for + * this map + * (optional) * @throws NullPointerException if the specified key is null and this map - * does not permit null keys - * (optional) + * does not permit null keys + * (optional) */ boolean containsKey(K key); - /** * Returns the value to which the specified key is mapped, * or {@code null} if this map contains no mapping for the key. @@ -80,22 +80,17 @@ public interface IState { * * @param key the key whose associated value is to be returned * @return the value to which the specified key is mapped, or - * {@code null} if this map contains no mapping for the key - * @throws ClassCastException if the key is of an inappropriate type for - * this map - * (optional) + * {@code null} if this map contains no mapping for the key + * @throws ClassCastException if the key is of an inappropriate type for + * this map + * (optional) * @throws NullPointerException if the specified key is null and this map - * does not permit null keys - * (optional) + * does not permit null keys + * (optional) */ V get(K key); - - - Map get(List key); - - - + Map get(List key); // Modification Operations @@ -107,21 +102,21 @@ public interface IState { * if {@link #containsKey(Object) m.containsKey(k)} would return * true.) * - * @param key key with which the specified value is to be associated + * @param key key with which the specified value is to be associated * @param value value to be associated with the specified key * @return the previous value associated with key, or - * null if there was no mapping for key. - * (A null return can also indicate that the map - * previously associated null with key, - * if the implementation supports null values.) + * null if there was no mapping for key. + * (A null return can also indicate that the map + * previously associated null with key, + * if the implementation supports null values.) * @throws UnsupportedOperationException if the put operation - * is not supported by this map - * @throws ClassCastException if the class of the specified key or value - * prevents it from being stored in this map - * @throws NullPointerException if the specified key or value is null - * and this map does not permit null keys or values - * @throws IllegalArgumentException if some property of the specified key - * or value prevents it from being stored in this map + * is not supported by this map + * @throws ClassCastException if the class of the specified key or value + * prevents it from being stored in this map + * @throws NullPointerException if the specified key or value is null + * and this map does not permit null keys or values + * @throws IllegalArgumentException if some property of the specified key + * or value prevents it from being stored in this map */ V put(K key, V value); @@ -145,39 +140,38 @@ public interface IState { * * @param key key whose mapping is to be removed from the map * @return the previous value associated with key, or - * null if there was no mapping for key. + * null if there was no mapping for key. * @throws UnsupportedOperationException if the remove operation - * is not supported by this map - * @throws ClassCastException if the key is of an inappropriate type for - * this map - * (optional) - * @throws NullPointerException if the specified key is null and this - * map does not permit null keys - * (optional) + * is not supported by this map + * @throws ClassCastException if the key is of an inappropriate type for + * this map + * (optional) + * @throws NullPointerException if the specified key is null and this + * map does not permit null keys + * (optional) */ V remove(K key); - // Bulk Operations /** * Copies all of the mappings from the specified map to this map * (optional operation). The effect of this call is equivalent to that - * of calling {@link #put(Object,Object) put(k, v)} on this map once + * of calling {@link #put(Object, Object) put(k, v)} on this map once * for each mapping from key k to value v in the * specified map. The behavior of this operation is undefined if the * specified map is modified while the operation is in progress. * * @param m mappings to be stored in this map * @throws UnsupportedOperationException if the putAll operation - * is not supported by this map - * @throws ClassCastException if the class of a key or value in the - * specified map prevents it from being stored in this map - * @throws NullPointerException if the specified map is null, or if - * this map does not permit null keys or values, and the - * specified map contains null keys or values - * @throws IllegalArgumentException if some property of a key or value in - * the specified map prevents it from being stored in this map + * is not supported by this map + * @throws ClassCastException if the class of a key or value in the + * specified map prevents it from being stored in this map + * @throws NullPointerException if the specified map is null, or if + * this map does not permit null keys or values, and the + * specified map contains null keys or values + * @throws IllegalArgumentException if some property of a key or value in + * the specified map prevents it from being stored in this map */ void putAll(Map m); @@ -186,7 +180,7 @@ public interface IState { * The map will be empty after this call returns. * * @throws UnsupportedOperationException if the clear operation - * is not supported by this map + * is not supported by this map */ void clear(); @@ -207,7 +201,6 @@ public interface IState { */ Iterator keyIterator(); - /** * Returns a {@link Set} view of the mappings contained in this map. * The set is backed by the map, so changes to the map are @@ -226,14 +219,9 @@ public interface IState { */ Iterator> entryIterator(); - - void removeKeys(Collection keys); - - void scanEntity(IEntryProcessor processor); - V putIfAbsent(K key, V value); } diff --git a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/LruState.java b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/LruState.java index 7ccb149a..443e2c46 100644 --- a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/LruState.java +++ b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/LruState.java @@ -28,10 +28,8 @@ */ public final class LruState { - private int max_size; - private final T FLAG_VALUE; - + private int max_size; private ConcurrentHashMap> elementMap; private LinkedList elementList; @@ -117,6 +115,15 @@ public synchronized Iterator iterator() { return new LruIterator(); } + /** + * counter + * + * @return + */ + public synchronized int count() { + return elementMap.size(); + } + public class LruIterator implements Iterator { private Element current = elementList.head.next; @@ -132,26 +139,14 @@ public class LruIterator implements Iterator { } } - /** - * counter - * - * @return - */ - public synchronized int count() { - return elementMap.size(); - } - } class Element { - private T value; - - private volatile int counter; - Element next; - Element pre; + private T value; + private volatile int counter; public Element(T element) { this.value = element; diff --git a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/backend/IStateBackend.java b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/backend/IStateBackend.java index ddc35f7d..f997fdc1 100644 --- a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/backend/IStateBackend.java +++ b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/backend/IStateBackend.java @@ -23,7 +23,7 @@ import java.util.Set; import org.apache.rocketmq.streams.state.IEntryProcessor; -public interface IStateBackend { +public interface IStateBackend { /** * Returns the number of key-value mappings in this map. If the @@ -34,8 +34,7 @@ public interface IStateBackend { */ int size(String namespace); - - Map get(String namespace, List key); + Map get(String namespace, List key); /** * Removes the mapping for a key from this map if it is present @@ -57,37 +56,36 @@ public interface IStateBackend { * * @param key key whose mapping is to be removed from the map * @return the previous value associated with key, or - * null if there was no mapping for key. + * null if there was no mapping for key. * @throws UnsupportedOperationException if the remove operation - * is not supported by this map - * @throws ClassCastException if the key is of an inappropriate type for - * this map - * (optional) - * @throws NullPointerException if the specified key is null and this - * map does not permit null keys - * (optional) + * is not supported by this map + * @throws ClassCastException if the key is of an inappropriate type for + * this map + * (optional) + * @throws NullPointerException if the specified key is null and this + * map does not permit null keys + * (optional) */ V remove(String namespace, K key); - /** * Copies all of the mappings from the specified map to this map * (optional operation). The effect of this call is equivalent to that - * of calling {@link #put(Object,Object) put(k, v)} on this map once + * of calling {@link #put(Object, Object) put(k, v)} on this map once * for each mapping from key k to value v in the * specified map. The behavior of this operation is undefined if the * specified map is modified while the operation is in progress. * * @param m mappings to be stored in this map * @throws UnsupportedOperationException if the putAll operation - * is not supported by this map - * @throws ClassCastException if the class of a key or value in the - * specified map prevents it from being stored in this map - * @throws NullPointerException if the specified map is null, or if - * this map does not permit null keys or values, and the - * specified map contains null keys or values - * @throws IllegalArgumentException if some property of a key or value in - * the specified map prevents it from being stored in this map + * is not supported by this map + * @throws ClassCastException if the class of a key or value in the + * specified map prevents it from being stored in this map + * @throws NullPointerException if the specified map is null, or if + * this map does not permit null keys or values, and the + * specified map contains null keys or values + * @throws IllegalArgumentException if some property of a key or value in + * the specified map prevents it from being stored in this map */ void putAll(String namespace, Map m); @@ -96,11 +94,10 @@ public interface IStateBackend { * The map will be empty after this call returns. * * @throws UnsupportedOperationException if the clear operation - * is not supported by this map + * is not supported by this map */ void clear(String namespace); - /** * Returns a {@link Set} view of the keys contained in this map. * The set is backed by the map, so changes to the map are @@ -118,7 +115,6 @@ public interface IStateBackend { */ Iterator keyIterator(String namespace); - /** * Returns a {@link Set} view of the mappings contained in this map. * The set is backed by the map, so changes to the map are @@ -137,15 +133,10 @@ public interface IStateBackend { */ Iterator> entryIterator(String namespace); - - void removeKeys(String namespace, Collection keys); - - void scanEntity(String namespace, IEntryProcessor processor); - V putIfAbsent(String namespace, K key, V value); } diff --git a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/impl/MapState.java b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/impl/MapState.java index 69f7b347..c22614ee 100644 --- a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/impl/MapState.java +++ b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/impl/MapState.java @@ -15,9 +15,10 @@ * limitations under the License. */ package org.apache.rocketmq.streams.state.impl; + import org.apache.rocketmq.streams.state.AbstractState; -public class MapState extends AbstractState { +public class MapState extends AbstractState { public MapState(String namespace, String backendName) { super(namespace, backendName); diff --git a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/rocksdb/RocksDBOperator.java b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/rocksdb/RocksDBOperator.java index a06ca581..32ee72be 100644 --- a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/rocksdb/RocksDBOperator.java +++ b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/rocksdb/RocksDBOperator.java @@ -20,23 +20,11 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.rocketmq.streams.common.utils.FileUtil; import org.apache.rocketmq.streams.common.utils.RuntimeUtil; -import org.apache.rocketmq.streams.common.utils.StringUtil; -import org.rocksdb.BlockBasedTableConfig; -import org.rocksdb.BloomFilter; -import org.rocksdb.Cache; -import org.rocksdb.CompactionStyle; -import org.rocksdb.CompressionType; -import org.rocksdb.Filter; -import org.rocksdb.LRUCache; import org.rocksdb.Options; -import org.rocksdb.RateLimiter; -import org.rocksdb.ReadOptions; import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; -import org.rocksdb.Statistics; import org.rocksdb.TtlDB; import org.rocksdb.WriteOptions; -import org.rocksdb.util.SizeUnit; public class RocksDBOperator { @@ -48,14 +36,14 @@ public class RocksDBOperator { protected static RocksDB rocksDB; - protected WriteOptions writeOptions = new WriteOptions(); - static { RocksDB.loadLibrary(); } + protected WriteOptions writeOptions = new WriteOptions(); + public RocksDBOperator() { - this(FileUtil.concatFilePath( DB_PATH + File.separator + RuntimeUtil.getDipperInstanceId(), "rocksdb")); + this(FileUtil.concatFilePath(DB_PATH + File.separator + RuntimeUtil.getDipperInstanceId(), "rocksdb")); } public RocksDBOperator(String rocksdbFilePath) { @@ -72,7 +60,7 @@ public RocksDBOperator(String rocksdbFilePath) { dir.delete(); } dir.mkdirs(); - // final Filter bloomFilter = new BloomFilter(10); + // final Filter bloomFilter = new BloomFilter(10); // final ReadOptions readOptions = new ReadOptions().setFillCache(false); // final Statistics stats = new Statistics(); // final RateLimiter rateLimiter = new RateLimiter(10000000, 10000, 10); diff --git a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/rocksdb/RocksdbState.java b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/rocksdb/RocksdbState.java index 3e0f38b5..6526cc1a 100644 --- a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/rocksdb/RocksdbState.java +++ b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/rocksdb/RocksdbState.java @@ -43,11 +43,23 @@ */ public class RocksdbState implements IKvState { + private final static Byte SIGN = 1; private static RocksDBOperator operator = new RocksDBOperator(); - private final LruState cache = new LruState<>(100, ""); - private final static Byte SIGN = 1; + /** + * 把byte转化成值 + * + * @param bytes + * @return + */ + protected static String getValueFromByte(byte[] bytes) { + try { + return new String(bytes, UTF8); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } @Override public String get(String key) { try { @@ -69,12 +81,10 @@ public class RocksdbState implements IKvState { } try { Map resultMap = new HashMap<>(keys.size()); - Map map = operator.getInstance().multiGet(keyByteList); - Iterator> it = map.entrySet().iterator(); - while (it.hasNext()) { - Map.Entry entry = it.next(); - String key = getValueFromByte(entry.getKey()); - String value = getValueFromByte(entry.getValue()); + List valueByteList = operator.getInstance().multiGetAsList(keyByteList); + for (int i = 0; i < keyByteList.size(); i++) { + String key = getValueFromByte(keyByteList.get(i)); + String value = getValueFromByte(valueByteList.get(i)); resultMap.put(key, value); } return resultMap; @@ -176,30 +186,13 @@ protected byte[] getKeyBytes(String key) { } } - /** - * 把byte转化成值 - * - * @param bytes - * @return - */ - protected static String getValueFromByte(byte[] bytes) { - try { - return new String(bytes, UTF8); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException(e); - } - } - public static class RocksDBIterator implements Iterator> { protected AtomicBoolean hasInit = new AtomicBoolean(false); - + protected String keyPrefix; private ReadOptions readOptions = new ReadOptions(); - private RocksIterator iter; - protected String keyPrefix; - public RocksDBIterator(String keyPrefix) { readOptions.setPrefixSameAsStart(true).setTotalOrderSeek(true); iter = operator.getInstance().newIterator(readOptions); diff --git a/rocketmq-streams-state/src/test/java/org/apache/rocketmq/streams/state/kv/TestLruState.java b/rocketmq-streams-state/src/test/java/org/apache/rocketmq/streams/state/kv/TestLruState.java index 6dbcf7ed..ef076f04 100644 --- a/rocketmq-streams-state/src/test/java/org/apache/rocketmq/streams/state/kv/TestLruState.java +++ b/rocketmq-streams-state/src/test/java/org/apache/rocketmq/streams/state/kv/TestLruState.java @@ -19,7 +19,7 @@ import java.util.Iterator; import java.util.Random; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; +import org.apache.rocketmq.streams.common.threadpool.ThreadPoolFactory; import org.apache.rocketmq.streams.state.LruState; import org.junit.Assert; import org.junit.Test; @@ -63,7 +63,7 @@ public void testBasic() { public void testConcurrent() { String content = "lru"; final LruState lruState = new LruState<>(10, ""); - ExecutorService poolService = Executors.newFixedThreadPool(10); + ExecutorService poolService = ThreadPoolFactory.createFixedThreadPool(10, TestLruState.class.getName() + "-test_concurrent"); final Random random = new Random(System.currentTimeMillis()); for (int i = 0; i < 1000; i++) { final int index = i; diff --git a/rocketmq-streams-state/src/test/resources/log4j.xml b/rocketmq-streams-state/src/test/resources/log4j.xml old mode 100755 new mode 100644 diff --git a/rocketmq-streams-cep/pom.xml b/rocketmq-streams-storage/pom.xml old mode 100755 new mode 100644 similarity index 55% rename from rocketmq-streams-cep/pom.xml rename to rocketmq-streams-storage/pom.xml index b0f3e2f1..381b0105 --- a/rocketmq-streams-cep/pom.xml +++ b/rocketmq-streams-storage/pom.xml @@ -15,50 +15,44 @@ See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 org.apache.rocketmq rocketmq-streams - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT - rocketmq-streams-ce0 - ROCKETMQ STREAMS :: cep + rocketmq-streams-storage jar + ROCKETMQ STREAMS :: storage + - org.apache.rocketmq - rocketmq-streams-commons - - - org.apache.flink - flink-cep_${scala.binary.version} - ${flink.version} + org.apache.hadoop + hadoop-client + 2.7.7 - org.apache.flink - flink-java - ${flink.version} + org.apache.orc + orc-core + 1.5.4 + - org.apache.flink - flink-core - ${flink.version} + org.apache.rocketmq + rocketmq-streams-state - org.apache.flink - flink-clients_${scala.binary.version} - ${flink.version} + com.google.guava + guava - org.apache.flink - flink-runtime_${scala.binary.version} - ${flink.version} + com.janeluo + ikanalyzer + 2012_u6 - - diff --git a/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/IStorage.java b/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/IStorage.java new file mode 100644 index 00000000..29f666ac --- /dev/null +++ b/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/IStorage.java @@ -0,0 +1,28 @@ +/* + * 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.storage; + +import com.alibaba.fastjson.JSONObject; +import java.util.List; +import org.apache.rocketmq.streams.common.model.Pair; + +public interface IStorage { + + List getRows(int... rowIds); + + List getRowOnlySpecifyColumn(List fieldNames, Pair... filterConditions); +} diff --git a/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/IStorageBlock.java b/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/IStorageBlock.java new file mode 100644 index 00000000..bdf908b2 --- /dev/null +++ b/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/IStorageBlock.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.storage; + +import com.alibaba.fastjson.JSONObject; +import java.util.Iterator; +import java.util.List; +import org.apache.rocketmq.streams.common.metadata.MetaData; + +public interface IStorageBlock { + /** + * return column info + * + * @return + */ + MetaData getMetaData(); + + long insertData(JSONObject... msgs); + + long insertData(List msgs); + + Iterator> queryRows(String... columnNames); + + Iterator> queryRowsByTerms(String columnName, boolean supportPrefix, Object... terms); + + Iterator> queryRowsByBetween(String columnName, Object queryMin, Object queryMax); + + long getRowSize(); + + /** + * 写入完成,后续不会再写此block + */ + void finishWrite(); + +} diff --git a/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/orc/ORCFile.java b/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/orc/ORCFile.java new file mode 100644 index 00000000..fc66535b --- /dev/null +++ b/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/orc/ORCFile.java @@ -0,0 +1,377 @@ +/* + * 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.storage.orc; + +import com.alibaba.fastjson.JSONObject; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; +import org.apache.orc.ColumnStatistics; +import org.apache.orc.OrcFile; +import org.apache.orc.OrcProto; +import org.apache.orc.Reader; +import org.apache.orc.StripeStatistics; +import org.apache.orc.TypeDescription; +import org.apache.orc.Writer; +import org.apache.orc.impl.ColumnStatisticsImpl; +import org.apache.rocketmq.streams.common.datatype.BooleanDataType; +import org.apache.rocketmq.streams.common.datatype.ByteDataType; +import org.apache.rocketmq.streams.common.datatype.DataType; +import org.apache.rocketmq.streams.common.datatype.DateDataType; +import org.apache.rocketmq.streams.common.datatype.DoubleDataType; +import org.apache.rocketmq.streams.common.datatype.FloatDataType; +import org.apache.rocketmq.streams.common.datatype.IntDataType; +import org.apache.rocketmq.streams.common.datatype.ListDataType; +import org.apache.rocketmq.streams.common.datatype.LongDataType; +import org.apache.rocketmq.streams.common.datatype.MapDataType; +import org.apache.rocketmq.streams.common.datatype.ShortDataType; +import org.apache.rocketmq.streams.common.datatype.StringDataType; +import org.apache.rocketmq.streams.common.metadata.MetaData; +import org.apache.rocketmq.streams.common.metadata.MetaDataField; +import org.apache.rocketmq.streams.common.utils.ReflectUtil; +import org.apache.rocketmq.streams.storage.utils.TermMatchUtil; + +public class ORCFile { + protected transient Writer writer; + protected transient TypeDescription schema; + protected transient String orcFilePath; + protected VectorizedRowBatch batch; + protected MetaData metaData; + + protected Map columnName2Index = new HashMap<>();//列名和列在metadata中的index + + public ORCFile(String orcFilePath, MetaData metaData) { + this.orcFilePath = orcFilePath; + this.metaData = metaData; + + TypeDescription schema = TypeDescription.createStruct(); + List fields = metaData.getMetaDataFields(); + int i = 0; + for (MetaDataField field : fields) { + String fieldName = field.getFieldName(); + schema.addField(fieldName, createType(field.getDataType())); + columnName2Index.put(fieldName, i); + i++; + } + this.schema = schema; + } + + public void insertData(List msgs) { + if (msgs == null) { + return; + } + if (this.writer == null || this.batch == null) { + synchronized (this) { + if (this.writer == null || this.batch == null) { + try { + Configuration conf = new Configuration(); + this.writer = OrcFile.createWriter(new Path(orcFilePath), + OrcFile.writerOptions(conf) + .rowIndexStride(10000) + .setSchema(schema)); + this.batch = schema.createRowBatch(); + } catch (IOException e) { + e.printStackTrace(); + } + } + } + + } + + try { + for (int i = 0; i < msgs.size(); i++) { + JSONObject jsonObject = msgs.get(i); + int rowIndex = batch.size++; + for (int j = 0; j < metaData.getMetaDataFields().size(); j++) { + ColumnVector vector = batch.cols[j]; + MetaDataField metaDataField = (MetaDataField) metaData.getMetaDataFields().get(j); + setColumnValue(vector, jsonObject.get(metaDataField.getFieldName()), rowIndex); + + } + if (batch.size == batch.getMaxSize()) { + writer.addRowBatch(batch); + batch.reset(); + } + } + if (batch.size != 0) { + writer.addRowBatch(batch); + batch.reset(); + } + } catch (Exception e) { + throw new RuntimeException("insert data 2 orc error ", e); + } + } + + public void flush() { + try { + writer.close(); + } catch (IOException e) { + e.printStackTrace(); + } + } + + public RowIterator queryRows(String... columnNames) { + TypeDescription schema = TypeDescription.createStruct(); + for (String columnName : columnNames) { + int index = columnName2Index.get(columnName); + MetaDataField field = (MetaDataField) metaData.getMetaDataFields().get(index); + String fieldName = field.getFieldName(); + schema.addField(fieldName, createType(field.getDataType())); + } + return new RowIterator(orcFilePath, schema, columnNames); + } + + public RowIterator queryRowsByTerms(String queryColumn, String viewColumn, boolean supportPrefix, Object... terms) { + return queryRows(queryColumn, viewColumn, supportPrefix, false, terms); + } + + public RowIterator queryRowsByBetween(String queryColumn, String viewColumn, Object min, Object max) { + return queryRows(queryColumn, viewColumn, false, true, new Object[] {min, max}); + } + + /** + * 统计orc文件行条数 + */ + public long count() { + Reader reader = getORCReader(); + OrcProto.FileTail fileTail = reader.getFileTail(); + return fileTail.getFooter().getNumberOfRows(); + } + + /** + * 关闭orc文件 + */ + public void close() { + try { + writer.close(); + } catch (IOException e) { + throw new RuntimeException("close orc file error " + orcFilePath); + } + } + + /** + * 返回包含term的行 + * 获取queryColumn,queryColumn列的值,并根据下推的谓词完成过滤 + * + * @param queryColumn 需要查询的cloumn + * @param viewColumn 额外需要展示的列 + * @param supportPrefix 谓词匹配是否考虑前缀匹配,只针对字符串生效 + * @param terms 待搜索的词 + * @return + */ + protected RowIterator queryRows(String queryColumn, String viewColumn, boolean supportPrefix, boolean isBetween, Object... terms) { + String[] columnNames = new String[] {queryColumn, viewColumn}; + Reader reader = getORCReader(); + List columnStatisticsList = reader.getOrcProtoFileStatistics(); + Integer columnIndex = columnName2Index.get(queryColumn); + if (columnIndex == null) { + throw new RuntimeException(queryColumn + " column is not exist in " + this.orcFilePath + " orc file "); + } + + /** + * 判断搜索词是否在当前文件 + */ + OrcProto.ColumnStatistics columnStatistics = columnStatisticsList.get(columnIndex + 1); + MetaDataField metaDataField = (MetaDataField) metaData.getMetaDataFields().get(columnIndex); + Object min = getMinColumnValue(metaDataField.getDataType(), columnStatistics); + Object max = getMaxColumnValue(metaDataField.getDataType(), columnStatistics); + if (!isMatch(metaDataField.getDataType(), min, max, terms)) { + return null; + } + + /** + * 找出包含搜索词的Stripe + */ + + try { + List stripeIndexs = new ArrayList<>(); + List list = reader.getOrcProtoStripeStatistics(); + List stripeStatistics = reader.getStripeStatistics(); + int stripeIndex = 0; + for (StripeStatistics statistics : stripeStatistics) { + ColumnStatistics statistic = statistics.getColumnStatistics()[columnIndex + 1]; + ColumnStatisticsImpl tripeColumnStatistics = (ColumnStatisticsImpl) statistic; + Object minValue = ReflectUtil.getDeclaredField(tripeColumnStatistics, "minimum"); + Object maxValue = ReflectUtil.getDeclaredField(tripeColumnStatistics, "maximum"); + if (isMatch(metaDataField.getDataType(), minValue, maxValue, terms)) { + stripeIndexs.add(stripeIndex); + } + stripeIndex++; + } + if (stripeIndexs.size() == 0) { + return null; + } + RowIterator rowIterator = new RowIterator(orcFilePath, schema, columnNames); + // rowIterator.setBatchIndexs(stripeIndexs); + rowIterator.setQueryColumn(queryColumn); + rowIterator.setQueryColumnDataType(metaDataField.getDataType()); + rowIterator.setSupportPrefix(supportPrefix); + if (isBetween) { + rowIterator.setMin(terms[0]); + rowIterator.setMax(terms[1]); + } else { + rowIterator.setTerms(terms); + } + + return rowIterator; + } catch (Exception e) { + throw new RuntimeException("read StripeStatistics error"); + } + + } + + private Object getMinColumnValue(DataType dataType, OrcProto.ColumnStatistics statistics) { + return getStatistics(dataType, statistics, true); + } + + private Object getMaxColumnValue(DataType dataType, OrcProto.ColumnStatistics statistics) { + return getStatistics(dataType, statistics, false); + } + + /** + * 创建reader + * + * @return + */ + protected Reader getORCReader() { + try { + Configuration conf = new Configuration(); + Reader reader = OrcFile.createReader(new Path(orcFilePath), + OrcFile.readerOptions(conf)); + return reader; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * 把列值设置到orc的列对象中 + * + * @param vector 列对象 + * @param value 列值 + * @param rowIndex 行号 + */ + protected void setColumnValue(ColumnVector vector, Object value, int rowIndex) { + try { + if (BytesColumnVector.class.isInstance(vector)) { + BytesColumnVector bytesColumnVector = (BytesColumnVector) vector; + byte[] bytes = null; + if (value == null) { + bytes = new byte[0]; + } else { + bytes = value.toString().getBytes("UTF-8"); + } + bytesColumnVector.setVal(rowIndex, bytes); + } else if (LongColumnVector.class.isInstance(vector)) { + LongColumnVector columnVector = (LongColumnVector) vector; + columnVector.vector[rowIndex] = (Long) value; + } else { + throw new RuntimeException("can not support this columnVector " + vector.toString()); + } + + //todo other columnVector impl + } catch (Exception e) { + throw new RuntimeException("set value 2 orc error", e); + } + + } + + /** + * 根据datatype转化成orc的格式 + * + * @param dataType + * @return + */ + protected TypeDescription createType(DataType dataType) { + if (StringDataType.class.isInstance(dataType)) { + return TypeDescription.createString(); + } else if (IntDataType.class.isInstance(dataType)) { + return TypeDescription.createInt(); + } else if (LongDataType.class.isInstance(dataType)) { + return TypeDescription.createLong(); + } else if (DoubleDataType.class.isInstance(dataType)) { + return TypeDescription.createDouble(); + } else if (FloatDataType.class.isInstance(dataType)) { + return TypeDescription.createFloat(); + } else if (DateDataType.class.isInstance(dataType)) { + return TypeDescription.createDate(); + } else if (BooleanDataType.class.isInstance(dataType)) { + return TypeDescription.createBoolean(); + } else if (ListDataType.class.isInstance(dataType)) { + ListDataType listDataType = (ListDataType) dataType; + return TypeDescription.createList(createType(listDataType.getParadigmType())); + } else if (ShortDataType.class.isInstance(dataType)) { + return TypeDescription.createShort(); + } else if (ByteDataType.class.isInstance(dataType)) { + return TypeDescription.createByte(); + } else if (MapDataType.class.isInstance(dataType)) { + MapDataType mapDataType = (MapDataType) dataType; + return TypeDescription.createMap(createType(mapDataType.getKeyParadigmType()), createType(mapDataType.getValueParadigmType())); + } else { + throw new RuntimeException("can not support this datatype " + dataType); + } + + } + + /** + * 根据datatype转化成orc的格式 + * + * @param dataType + * @return + */ + protected Object getStatistics(DataType dataType, OrcProto.ColumnStatistics statistics, boolean isMin) { + if (StringDataType.class.isInstance(dataType)) { + return isMin ? statistics.getStringStatistics().getMinimum() : statistics.getStringStatistics().getMaximum(); + } else if (IntDataType.class.isInstance(dataType) || LongDataType.class.isInstance(dataType) || ShortDataType.class.isInstance(dataType)) { + return isMin ? statistics.getIntStatistics().getMinimum() : statistics.getIntStatistics().getMaximum(); + } else if (DoubleDataType.class.isInstance(dataType) || FloatDataType.class.isInstance(dataType)) { + return isMin ? statistics.getDoubleStatistics().getMinimum() : statistics.getDoubleStatistics().getMaximum(); + } else if (DateDataType.class.isInstance(dataType)) { + return isMin ? statistics.getDateStatistics().getMinimum() : statistics.getDateStatistics().getMaximum(); + } else { + throw new RuntimeException("can not support this datatype " + dataType); + } + + } + + protected boolean isMatch(DataType type, Object min, Object max, Object[] terms) { + if (terms == null) { + return true; + } + + for (Object term : terms) { + boolean isMatch = TermMatchUtil.matchBetween(term, type, min, max); + if (isMatch) { + return true; + } + } + return false; + } + + protected boolean isMatch(DataType type, Object min, Object max, Object queryMin, Object queryMax) { + return isMatch(type, min, max, new Object[] {queryMin, queryMax}); + } + +} diff --git a/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/orc/ORCStorageBlock.java b/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/orc/ORCStorageBlock.java new file mode 100644 index 00000000..f70e2318 --- /dev/null +++ b/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/orc/ORCStorageBlock.java @@ -0,0 +1,130 @@ +/* + * 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.storage.orc; + +import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; +import org.apache.rocketmq.streams.common.datatype.DataType; +import org.apache.rocketmq.streams.common.metadata.MetaData; +import org.apache.rocketmq.streams.storage.IStorageBlock; + +public class ORCStorageBlock extends BasedConfigurable implements IStorageBlock { + protected static String ROW_ID = "_ROW_ID"; + + protected MetaData metaData;//字段信息 + protected boolean isWriteFinished = false;//块是否完成写入 + protected AtomicLong size = new AtomicLong(0);//行数 + protected AtomicLong blockByteSize = new AtomicLong(0);// + + /** + * key:columnName + */ + protected transient Map orcFileMap = new HashMap<>(); + protected transient RocksDBFile rocksDBFile; + + @Override protected boolean initConfigurable() { + return super.initConfigurable(); + } + + @Override public MetaData getMetaData() { + return metaData; + } + + @Override public long insertData(JSONObject... msgs) { + if (msgs == null) { + return blockByteSize.get(); + } + List rows = new ArrayList<>(); + for (JSONObject row : msgs) { + rows.add(row); + } + return insertData(rows); + } + + @Override public long insertData(List msgs) { + if (msgs == null) { + return blockByteSize.get(); + } + try { + for (JSONObject msg : msgs) { + long rowId = size.get(); + blockByteSize.addAndGet(msg.toString().getBytes("UTF-8").length); + rocksDBFile.write2RocksDB(rowId, msg); + size.incrementAndGet(); + } + return blockByteSize.get(); + } catch (Exception e) { + throw new RuntimeException("insert data 2 rocksdb error ", e); + } + + } + + @Override public RowIterator queryRows(String... columnNames) { + if (isWriteFinished) { + return queryRowsFromSortORC(columnNames); + } else { + return rocksDBFile.queryRowsFromRocksDB(columnNames); + } + } + + @Override public Iterator> queryRowsByTerms(String columnName, boolean supportPrefix, Object... terms) { + if (isWriteFinished) { + return queryRowsFromSortORC(columnName, supportPrefix, terms); + } else { + DataType dataType = metaData.getMetaDataField(columnName).getDataType(); + return rocksDBFile.queryRowsFromRocksDB(columnName, dataType, supportPrefix, terms); + } + } + + @Override public Iterator> queryRowsByBetween(String columnName, Object queryMin, Object queryMax) { + if (isWriteFinished) { + return queryRowsFromSortORC(columnName, queryMin, queryMax); + } else { + DataType dataType = metaData.getMetaDataField(columnName).getDataType(); + return rocksDBFile.queryRowsFromRocksDB(columnName, dataType, queryMin, queryMax); + } + } + + private RowIterator queryRowsFromSortORC(String[] columnNames) { + throw new RuntimeException("can not suppoert this method"); + } + + private RowIterator queryRowsFromSortORC(String columnName, boolean supportPrefix, Object[] terms) { + ORCFile orcFile = orcFileMap.get(columnName); + return orcFile.queryRowsByTerms(columnName, ROW_ID, supportPrefix, terms); + } + + private RowIterator queryRowsFromSortORC(String columnName, Object min, Object max) { + ORCFile orcFile = orcFileMap.get(columnName); + return orcFile.queryRowsByBetween(columnName, ROW_ID, min, max); + } + + @Override public long getRowSize() { + return size.get(); + } + + @Override public void finishWrite() { + this.isWriteFinished = true; + } + +} diff --git a/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/orc/RocksDBFile.java b/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/orc/RocksDBFile.java new file mode 100644 index 00000000..4a27bf47 --- /dev/null +++ b/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/orc/RocksDBFile.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.storage.orc; + +import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.rocketmq.streams.common.datatype.DataType; +import org.apache.rocketmq.streams.common.metadata.MetaData; +import org.apache.rocketmq.streams.common.metadata.MetaDataField; +import org.apache.rocketmq.streams.state.kv.rocksdb.RocksDBOperator; +import org.rocksdb.ReadOptions; +import org.rocksdb.RocksDB; +import org.rocksdb.RocksIterator; +import org.rocksdb.WriteBatch; +import org.rocksdb.WriteOptions; + +public class RocksDBFile { + protected static String ROW_PREFIX = "_row"; + protected static String COLUMN_PREFIX = "_columm"; + + protected String rocksdbFilePath; + protected MetaData metaData; + protected transient RocksDB rocksDB; + protected Map columnIndexs = new HashMap<>(); + + public RocksDBFile(String rocksdbFilePath, MetaData metaData) { + this.rocksdbFilePath = rocksdbFilePath; + this.metaData = metaData; + List metaDataFields = metaData.getMetaDataFields(); + int columnIndex = 0; + for (MetaDataField metaDataField : metaDataFields) { + columnIndexs.put(metaDataField.getFieldName(), columnIndex++); + + } + rocksDB = new RocksDBOperator(rocksdbFilePath).getInstance(); + } + + public void write2RocksDB(List rows) { + try { + WriteBatch writeBatch = new WriteBatch(); + List columnIndexList = metaData.getIndexFieldNamesList(); + long rowId = 0; + for (String row : rows) { + JSONObject rowJson = JSONObject.parseObject(row); + if (columnIndexList != null) { + + for (String columnName : columnIndexList) { + Object colummValue = rowJson.get(columnName); + MetaDataField metaDataField = metaData.getMetaDataField(columnName); + + if (colummValue != null) { + String value = metaDataField.getDataType().toDataJson(colummValue); + String key = columnIndexs.get(columnName) + "," + value + "," + rowId; + writeBatch.put(key.getBytes("UTF-8"), new byte[1]); + } + rowJson.remove(columnName); + } + } + + writeBatch.put(((rowId++) + "").getBytes("UTF-8"), rowJson.toJSONString().getBytes("UTF-8")); + + } + WriteOptions writeOptions = new WriteOptions(); + writeOptions.setSync(false); + writeOptions.setDisableWAL(true); + rocksDB.write(writeOptions, writeBatch); + writeBatch.close(); + writeOptions.close(); + } catch (Exception e) { + e.printStackTrace(); + } + + } + + public void write2RocksDB(long rowId, JSONObject row) { + try { + + WriteBatch writeBatch = new WriteBatch(); + List columnIndexList = metaData.getIndexFieldNamesList(); + if (columnIndexList != null) { + for (String columnName : columnIndexList) { + Object colummValue = row.get(columnName); + MetaDataField metaDataField = metaData.getMetaDataField(columnName); + + if (colummValue != null) { + String value = metaDataField.getDataType().toDataJson(colummValue); + String key = columnIndexs.get(columnName) + "," + value + "," + rowId; + writeBatch.put(key.getBytes("UTF-8"), new byte[1]); + if (writeBatch.getDataSize() > 1000) { + + } + } + row.remove(columnName); + } + } + + writeBatch.put((rowId + "").getBytes("UTF-8"), row.toJSONString().getBytes("UTF-8")); + WriteOptions writeOptions = new WriteOptions(); + writeOptions.setSync(false); + writeOptions.setDisableWAL(true); + rocksDB.write(writeOptions, writeBatch); + writeBatch.close(); + writeOptions.close(); + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException("write 2 rocksdb error. rowId=" + rowId); + } + + } + + public RowIterator queryRowsFromRocksDB(String[] columnNames) { + throw new RuntimeException("can not support this method"); + } + + public Iterator> queryRowsFromRocksDB(String columnName, DataType dataType, boolean supportPrefix, Object[] terms) { + return new RocksDBIteratorByTerms(columnName, dataType, terms); + } + + public Iterator> queryRowsFromRocksDB(String columnName, DataType dataType, Object min, Object max) { + return new RocksDBIterator(columnName, min, max, dataType); + } + + protected JSONObject createRowByKey(String columnName, String key) { + String[] columns = key.split(","); + String columnIndex = columns[0]; + String rowIndex = columns[columns.length - 1]; + JSONObject row = new JSONObject(); + row.put(columnName, key.substring(columnIndex.length(), key.length() - rowIndex.length())); + row.put("rowId", rowIndex); + return row; + } + + public class RocksDBIteratorByTerms implements Iterator> { + protected int currentIndex; + List rocksDBIterators = new ArrayList<>(); + + public RocksDBIteratorByTerms(String columnName, DataType dataType, Object... terms) { + if (terms != null) { + for (Object term : terms) { + rocksDBIterators.add(new RocksDBIterator(columnName, term, term, dataType)); + } + } + } + + @Override public boolean hasNext() { + if (currentIndex >= rocksDBIterators.size()) { + return false; + } + if (currentIndex < rocksDBIterators.size() - 1) { + return true; + } + if (currentIndex == rocksDBIterators.size() - 1) { + RocksDBIterator rocksDBIterator = rocksDBIterators.get(currentIndex); + return rocksDBIterator.hasNext; + } + return true; + } + + @Override public List next() { + RocksDBIterator rocksDBIterator = rocksDBIterators.get(currentIndex); + if (rocksDBIterator.hasNext) { + rocksDBIterator.next(); + } else { + currentIndex++; + } + if (hasNext()) { + return next(); + } + return null; + } + } + + public class RocksDBIterator implements Iterator> { + protected volatile boolean hasNext = true; + protected AtomicBoolean hasInit = new AtomicBoolean(false); + protected String min; + protected String max; + protected DataType dataType; + protected String columnName; + ReadOptions readOptions = new ReadOptions(); + private RocksIterator iter; + + public RocksDBIterator(String columnName, Object minObject, Object maxObject, DataType dataType) { + readOptions.setPrefixSameAsStart(true).setTotalOrderSeek(true); + iter = rocksDB.newIterator(readOptions); + this.columnName = columnName; + this.min = dataType.toDataJson(minObject); + if (maxObject == null) { + this.max = min; + } else { + this.max = dataType.toDataJson(maxObject); + ; + } + this.dataType = dataType; + } + + @Override public boolean hasNext() { + if (hasInit.compareAndSet(false, true)) { + iter.seek(min.getBytes()); + } + return iter.isValid() && hasNext; + } + + @Override public List next() { + String key = new String(iter.key()); + if (!key.startsWith(max)) { + hasNext = false; + return null; + } + int batchSize = 1000; + int currentSize = 0; + List rows = new ArrayList<>(); + while (currentSize < batchSize && key.startsWith(max)) { + JSONObject row = createRowByKey(columnName, key); + iter.next(); + currentSize++; + rows.add(row); + key = new String(iter.key()); + } + if (!key.startsWith(max)) { + hasNext = false; + + } + return rows; + } + + } + +} + diff --git a/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/orc/RowIterator.java b/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/orc/RowIterator.java new file mode 100644 index 00000000..399614a3 --- /dev/null +++ b/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/orc/RowIterator.java @@ -0,0 +1,227 @@ +/* + * 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.storage.orc; + +import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; +import org.apache.orc.OrcFile; +import org.apache.orc.Reader; +import org.apache.orc.RecordReader; +import org.apache.orc.TypeDescription; +import org.apache.rocketmq.streams.common.datatype.DataType; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.storage.utils.TermMatchUtil; + +public class RowIterator implements Iterator> { + protected String[] columnNames; + protected transient RecordReader rows; + protected transient VectorizedRowBatch batch; + + /** + * 谓词下推,词搜索 + */ + protected List batchIndexs;//指定扫描的batch + protected Object[] terms; + protected boolean supportPrefix; + protected String queryColumn; + protected DataType queryColumnDataType; + protected transient Integer currentIndex = 0;//当前执行到哪个batch + + /** + * 谓词下推,范围查找 + */ + protected Object min; + protected Object max; + + public RowIterator(String orcFilePath, TypeDescription schema, String... columnNames) { + this.columnNames = columnNames; + try { + Configuration conf = new Configuration(); + Reader reader = OrcFile.createReader(new Path(orcFilePath), + OrcFile.readerOptions(conf)); + Reader.Options readerOptions = new Reader.Options(conf); + this.rows = reader.rows(readerOptions.schema(schema)); + this.batch = schema.createRowBatch(); + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException("create orc read error ", e); + } + + } + + @Override public boolean hasNext() { + try { + if (batchIndexs == null) { + return rows.nextBatch(batch); + } else { + if (currentIndex >= batchIndexs.size()) { + return false; + } + Integer batchIndex = batchIndexs.get(currentIndex); + rows.seekToRow(batchIndex); + currentIndex++; + return rows.nextBatch(batch); + } + + } catch (Exception e) { + throw new RuntimeException("rows nextbatch error ", e); + } + + } + + @Override public List next() { + + ColumnVector[] vectors = new ColumnVector[columnNames.length]; + for (int i = 0; i < vectors.length; i++) { + vectors[i] = batch.cols[i]; + } + try { + List rows = new ArrayList<>(); + for (int rowIndex = 0; rowIndex < batch.size; rowIndex++) { + JSONObject row = new JSONObject(); + for (int i = 0; i < vectors.length; i++) { + ColumnVector vector = vectors[i]; + Object value = getColumnValue(vector, rowIndex); + row.put(columnNames[i], value); + } + boolean isMatch = matchTerm(row); + if (isMatch) { + rows.add(row); + } + + } + return rows; + } catch (Exception e) { + throw new RuntimeException("GET BATCH DATA FROM ORC ERROR ", e); + } + + } + + /** + * 当前列是否包含 + * + * @param row + * @return + */ + protected boolean matchTerm(JSONObject row) { + if (StringUtil.isEmpty(queryColumn) || queryColumnDataType == null) { + return true; + } + + Object columnValue = row.get(queryColumn); + if (columnValue == null) { + return false; + } + if (terms != null) { + for (Object term : terms) { + if (term == null) { + continue; + } + boolean isMatch = TermMatchUtil.matchTerm(columnValue, queryColumnDataType, term, isSupportPrefix()); + if (isMatch) { + return isMatch; + } + } + } else if (min != null || max != null) { + boolean isMatch = TermMatchUtil.matchBetween(columnValue, queryColumnDataType, min, max); + if (isMatch) { + return isMatch; + } + } + + return false; + } + + protected Object getColumnValue(ColumnVector vector, int rowIndex) { + + try { + if (BytesColumnVector.class.isInstance(vector)) { + BytesColumnVector bytesColumnVector = (BytesColumnVector) vector; + return bytesColumnVector.toString(rowIndex); + } else if (LongColumnVector.class.isInstance(vector)) { + LongColumnVector columnVector = (LongColumnVector) vector; + return columnVector.vector[rowIndex]; + } else { + throw new RuntimeException("can not support this columnVector " + vector.toString()); + } + + //todo other columnVector impl + } catch (Exception e) { + throw new RuntimeException("set value 2 orc error", e); + } + } + + public List getBatchIndexs() { + return batchIndexs; + } + + public void setBatchIndexs(List batchIndexs) { + this.batchIndexs = batchIndexs; + } + + public boolean isSupportPrefix() { + return supportPrefix; + } + + public void setSupportPrefix(boolean supportPrefix) { + this.supportPrefix = supportPrefix; + } + + public String getQueryColumn() { + return queryColumn; + } + + public void setQueryColumn(String queryColumn) { + this.queryColumn = queryColumn; + } + + public void setTerms(Object[] terms) { + this.terms = terms; + } + + public DataType getQueryColumnDataType() { + return queryColumnDataType; + } + + public void setQueryColumnDataType(DataType queryColumnDataType) { + this.queryColumnDataType = queryColumnDataType; + } + + public Object getMin() { + return min; + } + + public void setMin(Object min) { + this.min = min; + } + + public Object getMax() { + return max; + } + + public void setMax(Object max) { + this.max = max; + } +} diff --git a/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/utils/TermMatchUtil.java b/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/utils/TermMatchUtil.java new file mode 100644 index 00000000..09cd0f8c --- /dev/null +++ b/rocketmq-streams-storage/src/main/java/org/apache/rocketmq/streams/storage/utils/TermMatchUtil.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.storage.utils; + +import org.apache.rocketmq.streams.common.datatype.BooleanDataType; +import org.apache.rocketmq.streams.common.datatype.DataType; +import org.apache.rocketmq.streams.common.datatype.DoubleDataType; +import org.apache.rocketmq.streams.common.datatype.FloatDataType; +import org.apache.rocketmq.streams.common.datatype.IntDataType; +import org.apache.rocketmq.streams.common.datatype.LongDataType; +import org.apache.rocketmq.streams.common.datatype.ShortDataType; +import org.apache.rocketmq.streams.common.datatype.StringDataType; +import org.apache.rocketmq.streams.common.utils.DataTypeUtil; + +public class TermMatchUtil { + + /** + * @param object + * @param dataType + * @param term + * @return + */ + public static boolean matchTerm(Object object, DataType dataType, Object term, boolean isSupportPrefix) { + if (object == null || dataType == null || term == null) { + return false; + } + if (StringDataType.class.isInstance(dataType)) { + String value = object.toString(); + String termStr = term.toString(); + if (isSupportPrefix) { + if (value.startsWith(termStr)) { + return true; + } else { + return value.equals(termStr); + } + } + } else if (LongDataType.class.isInstance(dataType) || IntDataType.class.isInstance(dataType) || ShortDataType.class.isInstance(dataType)) { + Long value = Long.valueOf(object.toString()); + Long termLong = Long.valueOf(term.toString()); + return (value.equals(termLong)); + } else if (DoubleDataType.class.isInstance(dataType) || FloatDataType.class.isInstance(dataType)) { + Double value = Double.valueOf(object.toString()); + Double termLong = Double.valueOf(term.toString()); + return (value.equals(termLong)); + } else if (BooleanDataType.class.isInstance(dataType)) { + Boolean value = Boolean.valueOf(object.toString()); + Boolean termBoolean = Boolean.valueOf(term.toString()); + return value.equals(termBoolean); + } else { + throw new RuntimeException("can not support the datatype in term match " + dataType); + } + //todo other datatype + return true; + } + + public static boolean matchBetween(Object value, DataType type, Object min, Object max) { + if (value == null) { + return false; + } + if (DataTypeUtil.isString(type)) { + String valueStr = value.toString(); + if (min != null) { + String minStr = min.toString(); + if (valueStr.compareTo(minStr) < 0) { + return false; + } + } + if (max != null) { + String maxStr = max.toString(); + if (valueStr.compareTo(maxStr) > 0) { + return false; + } + } + + } else if (DataTypeUtil.isNumber(type)) { + Double valueDouble = Double.valueOf(value.toString()); + if (min != null) { + Double minDouble = Double.valueOf(min.toString()); + if (valueDouble < minDouble) { + return false; + } + } + if (max != null) { + Double maxDouble = Double.valueOf(max.toString()); + if (valueDouble > maxDouble) { + return false; + } + } + } else if (DataTypeUtil.isDate(type)) { + throw new RuntimeException("can not support date in termmath"); + } else { + throw new RuntimeException("can not support date in termmath " + type.getDataTypeName()); + } + return true; + } + + public static void main(String[] args) { + System.out.println("c".compareTo("b")); + } +} diff --git a/rocketmq-streams-storage/src/test/java/org/apache/rocketmq/streams/storage/BloomFilterTest.java b/rocketmq-streams-storage/src/test/java/org/apache/rocketmq/streams/storage/BloomFilterTest.java new file mode 100644 index 00000000..51f0bada --- /dev/null +++ b/rocketmq-streams-storage/src/test/java/org/apache/rocketmq/streams/storage/BloomFilterTest.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.storage; + +import com.alibaba.fastjson.JSONObject; +import com.google.common.hash.BloomFilter; +import com.google.common.hash.Funnels; +import java.io.BufferedInputStream; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.StringReader; +import java.nio.charset.Charset; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.lucene.analysis.ngram.NGramTokenizer; +import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; +import org.apache.lucene.util.Version; +import org.apache.rocketmq.streams.common.cache.compress.impl.KeySet; +import org.apache.rocketmq.streams.common.channel.impl.file.FileSource; +import org.apache.rocketmq.streams.common.context.AbstractContext; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; +import org.apache.rocketmq.streams.common.threadpool.ThreadPoolFactory; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.junit.Test; + +public class BloomFilterTest { + protected static String fieldNames = + "aliUid\n" + + "buyAegis\n" + + "buySas\n" + + "client_mode\n" + + "cmd_chain\n" + + "cmd_chain_index\n" + + "cmd_index\n" + + "cmdline\n" + + "comm\n" + + "containerhostname\n" + + "containermip\n" + + "containername\n" + + "cwd\n" + + "data_complete\n" + + "delta_t1\n" + + "delta_t2\n" + + "docker_file_path\n" + + "dockercontainerid\n" + + "dockerimageid\n" + + "dockerimagename\n" + + "egroup_id\n" + + "egroup_name\n" + + "euid\n" + + "euid_name\n" + + "file_gid\n" + + "file_gid_name\n" + + "file_name\n" + + "file_path\n" + + "file_uid\n" + + "file_uid_name\n" + + "gcLevel\n" + + "gid\n" + + "gid_name\n" + + "host_uuid\n" + + "index\n" + + "k8sclusterid\n" + + "k8snamespace\n" + + "k8snodeid\n" + + "k8snodename\n" + + "k8spodname\n" + + "logTime\n" + + "log_match\n" + + "parent_cmd_line\n" + + "parent_file_name\n" + + "parent_file_path\n" + + "pcomm\n" + + "perm\n" + + "pid\n" + + "pid_start_time\n" + + "ppid\n" + + "scan_time\n" + + "sid\n" + + "srv_cmd\n" + + "stime\n" + + "tty\n" + + "uid\n" + + "uid_name\n"; + + @Test + public void testBloomFilter() throws IOException, InterruptedException { + long start = System.currentTimeMillis(); + String[] columnNames = new String[] {"aliUid", "cmdline", "file_name", "file_path", "parent_cmd_line", "parent_file_name", "parent_file_path", "host_uuid"};//fieldNames.split("\n"); + FileSource fileSource = new FileSource("/Users/yuanxiaodong/aegis_proc_public_1G.txt"); + fileSource.init(); + BloomFilter filter = BloomFilter.create( + Funnels.stringFunnel(Charset.defaultCharset()), + 335544320, + 0.3); + AtomicInteger count = new AtomicInteger(0); + ExecutorService executorService = ThreadPoolFactory.createThreadPool(10, BloomFilterTest.class.getName() + "-index"); + KeySet keySet = new KeySet(580000 * columnNames.length); + Object object = this; + fileSource.start(new IStreamOperator() { + @Override public Object doMessage(IMessage message, AbstractContext context) { + JSONObject jsonObject = message.getMessageBody(); + count.incrementAndGet(); + if (count.get() % 1000 == 0) { + System.out.println("progress " + (double) count.get() / (double) 580000); + } + for (int i = 0; i < columnNames.length; i++) { + String word = jsonObject.getString(columnNames[i]); + if (StringUtil.isEmpty(word)) { + continue; + } + if (keySet.contains(word)) { + continue; + } else { + keySet.add(word); + } + Set splitsWords = null; + try { + splitsWords = splitWords(word); + } catch (IOException e) { + e.printStackTrace(); + } + for (String w : splitsWords) { + filter.put(w); + } + } + +// executorService.execute(new Runnable() { +// @Override public void run() { +// +// } +// }); + return null; + } + }); + filter.writeTo(new FileOutputStream("/Users/yuanxiaodong/orc_1/bloom_test_3")); + System.out.println("cost is:" + (System.currentTimeMillis() - start)); + +// +// +// +// long startTime=System.currentTimeMillis(); +//filter =BloomFilter.readFrom(new BufferedInputStream(new FileInputStream("/tmp/orc/bloom_test_3")),Funnels.stringFunnel(Charset.defaultCharset())); +// System.out.println(filter.mightContain("fdsfsdfdsdfds")); +// +// String cmdline="/bash/dat33"; +// words=splitWords(cmdline); +// boolean isMatch=true; +// for(String word:words){ +// if(!filter.mightContain(word)){ +// isMatch=false; +// break; +// } +// } +// System.out.println(isMatch); + // System.out.println(filter.mightContain(cmdline)); +// +// System.out.println(filter.mightContain(cmdline.substring(0,10))); +// +// System.out.println(filter.mightContain(cmdline.substring(cmdline.length()-10))); + +// System.out.println("cost is "+(System.currentTimeMillis()-startTime)); + } + + @Test + public void testBloom() throws IOException { + BloomFilter filter = BloomFilter.readFrom(new BufferedInputStream(new FileInputStream("/Users/yuanxiaodong/orc_1/bloom_test_3")), Funnels.stringFunnel(Charset.defaultCharset())); + System.out.println(filter.mightContain(StringUtil.createMD5Str("java"))); + } + + private Set splitWords(String text) throws IOException { + if (StringUtil.isEmpty(text)) { + return new HashSet<>(); + } + Set words = new HashSet<>(); + if (text.length() < 3) { + words.add(text); + return words; + } + StringReader reader = new StringReader(text); + NGramTokenizer gramTokenizer = new NGramTokenizer(Version.LUCENE_47, reader, 3, 10); + CharTermAttribute charTermAttribute = gramTokenizer.addAttribute(CharTermAttribute.class); + gramTokenizer.reset(); + + while (gramTokenizer.incrementToken()) { + String token = charTermAttribute.toString(); + words.add(token); + + } + gramTokenizer.end(); + gramTokenizer.close(); + return words; + } + + @Test + public void testWords() throws IOException { + Set words = splitWords("%bash%/dat33%"); + for (String word : words) { + System.out.println(word); + } + } +} diff --git a/rocketmq-streams-storage/src/test/java/org/apache/rocketmq/streams/storage/ORCFileTest.java b/rocketmq-streams-storage/src/test/java/org/apache/rocketmq/streams/storage/ORCFileTest.java new file mode 100644 index 00000000..a84e384e --- /dev/null +++ b/rocketmq-streams-storage/src/test/java/org/apache/rocketmq/streams/storage/ORCFileTest.java @@ -0,0 +1,177 @@ +/* + * 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.storage; + +import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; +import java.util.List; +import org.apache.rocketmq.streams.common.datatype.StringDataType; +import org.apache.rocketmq.streams.common.metadata.MetaData; +import org.apache.rocketmq.streams.common.metadata.MetaDataField; +import org.apache.rocketmq.streams.common.utils.FileUtil; +import org.apache.rocketmq.streams.storage.orc.ORCFile; +import org.apache.rocketmq.streams.storage.orc.RowIterator; +import org.junit.Test; + +public class ORCFileTest { + protected static String fieldNames = + "aliUid\n" + + "buyAegis\n" + + "buySas\n" + + "client_mode\n" + + "cmd_chain\n" + + "cmd_chain_index\n" + + "cmd_index\n" + + "cmdline\n" + + "comm\n" + + "containerhostname\n" + + "containermip\n" + + "containername\n" + + "cwd\n" + + "data_complete\n" + + "delta_t1\n" + + "delta_t2\n" + + "docker_file_path\n" + + "dockercontainerid\n" + + "dockerimageid\n" + + "dockerimagename\n" + + "egroup_id\n" + + "egroup_name\n" + + "euid\n" + + "euid_name\n" + + "file_gid\n" + + "file_gid_name\n" + + "file_name\n" + + "file_path\n" + + "file_uid\n" + + "file_uid_name\n" + + "gcLevel\n" + + "gid\n" + + "gid_name\n" + + "host_uuid\n" + + "index\n" + + "k8sclusterid\n" + + "k8snamespace\n" + + "k8snodeid\n" + + "k8snodename\n" + + "k8spodname\n" + + "logTime\n" + + "log_match\n" + + "parent_cmd_line\n" + + "parent_file_name\n" + + "parent_file_path\n" + + "pcomm\n" + + "perm\n" + + "pid\n" + + "pid_start_time\n" + + "ppid\n" + + "scan_time\n" + + "sid\n" + + "srv_cmd\n" + + "stime\n" + + "tty\n" + + "uid\n" + + "uid_name\n"; + protected MetaData metaData; + + public ORCFileTest() { + String[] columns = fieldNames.split("\n"); + this.metaData = new MetaData(); + for (String column : columns) { + MetaDataField metaDataField = new MetaDataField(); + metaDataField.setFieldName(column); + metaDataField.setDataType(new StringDataType()); + metaData.getMetaDataFields().add(metaDataField); + } + metaData.setIndexFieldNamesList(new ArrayList<>()); + metaData.getIndexFieldNamesList().add("aliUid"); + metaData.getIndexFieldNamesList().add("cmdline"); + metaData.getIndexFieldNamesList().add("file_name"); + metaData.getIndexFieldNamesList().add("file_path"); + metaData.getIndexFieldNamesList().add("host_uuid"); + metaData.getIndexFieldNamesList().add("parent_cmd_line"); + metaData.getIndexFieldNamesList().add("parent_file_name"); + metaData.getIndexFieldNamesList().add("parent_file_path"); + + } + + @Test + public void testWrite() { + long start = System.currentTimeMillis(); + List rows = FileUtil.loadFileLine("/Users/yuanxiaodong/aegis_proc_public_1G.txt"); + System.out.println("start writing"); + for (Object object : metaData.getMetaDataFields()) { + MetaDataField metaDataField = (MetaDataField) object; + testColumnWrite(metaDataField.getFieldName(), metaDataField.getDataType().getDataTypeName(), rows); + System.out.println("finish " + metaDataField.getFieldName()); + } + + System.out.println("insert rocksdb cost " + (System.currentTimeMillis() - start)); + + } + + public void testColumnWrite(String columnName, String typeName, List rows) { + MetaData metaData = new MetaData(); + metaData.addMetaDataField("rowId", "long", true); + metaData.addMetaDataField(columnName, typeName, false); + + ORCFile orcFile = new ORCFile("/Users/yuanxiaodong/orc_1/orc_" + columnName + ".orc", metaData); + + List msgs = new ArrayList<>(); + long start = System.currentTimeMillis(); + long rowId = 0; + for (int i = 0; i < 10; i++) { + for (String rowStr : rows) { + JSONObject row = JSONObject.parseObject(rowStr); + JSONObject columnRow = new JSONObject(); + columnRow.put(columnName, row.get(columnName)); + columnRow.put("rowId", rowId++); + msgs.add(columnRow); + if (msgs.size() % 1000 == 0) { + orcFile.insertData(msgs); + msgs = new ArrayList<>(); + } + } + if (msgs.size() > 0) { + orcFile.insertData(msgs); + msgs = new ArrayList<>(); + } + } + orcFile.flush(); + } + + @Test + public void testRead() { + MetaData metaData = new MetaData(); + metaData.addMetaDataField("rowId", "long", true); + metaData.addMetaDataField("buyAegis", "string", false); + ORCFile orcFile = new ORCFile("/Users/yuanxiaodong/orc/orc_buyAegis.orc", metaData); + long start = System.currentTimeMillis(); + RowIterator rowIterator = orcFile.queryRowsByTerms("rowId", "buyAegis", false, "1000"); + int count = 0; + if (rowIterator != null) { + while (rowIterator.hasNext()) { + List jsonObjects = rowIterator.next(); + for (JSONObject row : jsonObjects) { + count++; + } + } + } + + System.out.println("query count is " + count + ", cost is " + (System.currentTimeMillis() - start)); + } +} diff --git a/rocketmq-streams-storage/src/test/java/org/apache/rocketmq/streams/storage/RocksDBFileTest.java b/rocketmq-streams-storage/src/test/java/org/apache/rocketmq/streams/storage/RocksDBFileTest.java new file mode 100644 index 00000000..acae7baf --- /dev/null +++ b/rocketmq-streams-storage/src/test/java/org/apache/rocketmq/streams/storage/RocksDBFileTest.java @@ -0,0 +1,133 @@ +/* + * 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.storage; + +import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; +import java.util.List; +import org.apache.rocketmq.streams.common.datatype.StringDataType; +import org.apache.rocketmq.streams.common.metadata.MetaData; +import org.apache.rocketmq.streams.common.metadata.MetaDataField; +import org.apache.rocketmq.streams.common.utils.FileUtil; +import org.apache.rocketmq.streams.storage.orc.RocksDBFile; +import org.junit.Test; + +public class RocksDBFileTest { + protected static String fieldNames = + "aliUid\n" + + "buyAegis\n" + + "buySas\n" + + "client_mode\n" + + "cmd_chain\n" + + "cmd_chain_index\n" + + "cmd_index\n" + + "cmdline\n" + + "comm\n" + + "containerhostname\n" + + "containermip\n" + + "containername\n" + + "cwd\n" + + "data_complete\n" + + "delta_t1\n" + + "delta_t2\n" + + "docker_file_path\n" + + "dockercontainerid\n" + + "dockerimageid\n" + + "dockerimagename\n" + + "egroup_id\n" + + "egroup_name\n" + + "euid\n" + + "euid_name\n" + + "file_gid\n" + + "file_gid_name\n" + + "file_name\n" + + "file_path\n" + + "file_uid\n" + + "file_uid_name\n" + + "gcLevel\n" + + "gid\n" + + "gid_name\n" + + "host_uuid\n" + + "index\n" + + "k8sclusterid\n" + + "k8snamespace\n" + + "k8snodeid\n" + + "k8snodename\n" + + "k8spodname\n" + + "logTime\n" + + "log_match\n" + + "parent_cmd_line\n" + + "parent_file_name\n" + + "parent_file_path\n" + + "pcomm\n" + + "perm\n" + + "pid\n" + + "pid_start_time\n" + + "ppid\n" + + "scan_time\n" + + "sid\n" + + "srv_cmd\n" + + "stime\n" + + "tty\n" + + "uid\n" + + "uid_name\n"; + protected RocksDBFile rocksDBFile; + + public RocksDBFileTest() { + String[] columns = fieldNames.split("\n"); + MetaData metaData = new MetaData(); + for (String column : columns) { + MetaDataField metaDataField = new MetaDataField(); + metaDataField.setFieldName(column); + metaDataField.setDataType(new StringDataType()); + metaData.getMetaDataFields().add(metaDataField); + } + metaData.setIndexFieldNamesList(new ArrayList<>()); + metaData.getIndexFieldNamesList().add("aliUid"); + metaData.getIndexFieldNamesList().add("cmdline"); + metaData.getIndexFieldNamesList().add("file_name"); + metaData.getIndexFieldNamesList().add("file_path"); + metaData.getIndexFieldNamesList().add("host_uuid"); + metaData.getIndexFieldNamesList().add("parent_cmd_line"); + metaData.getIndexFieldNamesList().add("parent_file_name"); + metaData.getIndexFieldNamesList().add("parent_file_path"); + FileUtil.deleteFile("/tmp/rocksdb"); + rocksDBFile = new RocksDBFile("/tmp/rocksdb", metaData); + } + + @Test + public void testWrite() { + + List rows = FileUtil.loadFileLine("/Users/yuanxiaodong/aegis_proc_public_1G.txt"); + +// rocksDBFile.write2RocksDB(rows); + List msgs = new ArrayList<>(); + long rowIndex = 0; + long start = System.currentTimeMillis(); + for (String rowStr : rows) { + JSONObject row = JSONObject.parseObject(rowStr); + rocksDBFile.write2RocksDB(rowIndex++, row); + } + + System.out.println("insert rocksdb cost " + (System.currentTimeMillis() - start)); + } + + @Test + public void testRead() { + + } +} diff --git a/rocketmq-streams-sts/.gitignore b/rocketmq-streams-sts/.gitignore new file mode 100644 index 00000000..b83d2226 --- /dev/null +++ b/rocketmq-streams-sts/.gitignore @@ -0,0 +1 @@ +/target/ diff --git a/rocketmq-streams-sts/pom.xml b/rocketmq-streams-sts/pom.xml new file mode 100755 index 00000000..7e6eccbe --- /dev/null +++ b/rocketmq-streams-sts/pom.xml @@ -0,0 +1,41 @@ + + + + 4.0.0 + + rocketmq-streams + org.apache.rocketmq + 2.0.1-SNAPSHOT + + rocketmq-streams-sts + ROCKETMQ STREAMS :: sts + jar + + + + + org.apache.rocketmq + rocketmq-streams-serviceloader + + + + + com.aliyun + aliyun-java-sdk-sts-internal + + + com.aliyun.ram + ram-auth-client + + + com.aliyun + aliyun-java-sdk-ram-inner + + + com.aliyun + aliyun-java-sdk-core + + + diff --git a/rocketmq-streams-sts/src/main/java/org/apache/rocketmq/streams/sts/CheckParametersUtils.java b/rocketmq-streams-sts/src/main/java/org/apache/rocketmq/streams/sts/CheckParametersUtils.java new file mode 100644 index 00000000..dae461e1 --- /dev/null +++ b/rocketmq-streams-sts/src/main/java/org/apache/rocketmq/streams/sts/CheckParametersUtils.java @@ -0,0 +1,31 @@ +package org.apache.rocketmq.streams.sts; + + +public class CheckParametersUtils { + + public static final T checkIfNullReturnDefault(T t1, T t2) { + if (t1 == null) { + return t2; + } else { + return t1; + } + } + + public static final T checkIfNullThrowException(T t, String ExceptionMsg) { + + if (t == null) { + throw new NullPointerException("check parameter error : {}. "); + } + return t; + } + + public static final int checkIfNullReturnDefaultInt(Object o, int t) { + + if (o == null) { + return t; + } + return Integer.parseInt(String.valueOf(o)); + + } + +} diff --git a/rocketmq-streams-sts/src/main/java/org/apache/rocketmq/streams/sts/ClientTransformer.java b/rocketmq-streams-sts/src/main/java/org/apache/rocketmq/streams/sts/ClientTransformer.java new file mode 100644 index 00000000..26a668b4 --- /dev/null +++ b/rocketmq-streams-sts/src/main/java/org/apache/rocketmq/streams/sts/ClientTransformer.java @@ -0,0 +1,131 @@ +/* + * 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.sts; + +import java.io.IOException; +import java.lang.instrument.ClassFileTransformer; +import java.lang.instrument.IllegalClassFormatException; +import java.security.ProtectionDomain; +import javassist.CannotCompileException; +import javassist.ClassPool; +import javassist.CtClass; +import javassist.CtField; +import javassist.CtMethod; +import javassist.CtNewMethod; +import javassist.Modifier; +import javassist.NotFoundException; + + +public class ClientTransformer implements ClassFileTransformer { + + final String name = "com.aliyun.openservices.log.Client.SendData(java.lang.String,com.aliyun.openservices.log.http.client.HttpMethod,java.lang.String,java.util.Map,java.util.Map,byte[],java.util.Map,java.lang.String)"; + final String methodBody = "\n" + + "\n" + + "final boolean isDipperRefresh = (headers.get(\"dipper_sts_refresh\") != null);\n" + + "if(isDipperRefresh){\n" + + "\theader.remove(\"dipper_sts_refresh\");\n" + + "}\n" + + "if (body.length > 0) {\n" + + "\theaders.put(Consts.CONST_CONTENT_MD5, DigestUtils.md5Crypt(body));\n" + + "}\n" + + "if (resourceOwnerAccount != null && !resourceOwnerAccount.isEmpty()) {\n" + + "\theaders.put(Consts.CONST_X_LOG_RESOURCEOWNERACCOUNT, resourceOwnerAccount);\n" + + "}\n" + + "headers.put(Consts.CONST_CONTENT_LENGTH, String.valueOf(body.length));\n" + + "DigestUtils.addSignature(credentials, method.toString(), headers, resourceUri, parameters);\n" + + "URI uri;\n" + + "if (serverIp == null) {\n" + + "\turi = GetHostURI(project);\n" + + "} else {\n" + + "\turi = GetHostURIByIp(serverIp);\n" + + "}\n" + + "RequestMessage request = BuildRequest(uri, method,\n" + + "\t\tresourceUri, parameters, headers,\n" + + "\t\tnew ByteArrayInputStream(body), body.length);\n" + + "ResponseMessage response = null;\n" + + "try {\n" + + "\tresponse = this.serviceClient.sendRequest(request, Consts.UTF_8_ENCODING);\n" + + "\tExtractResponseBody(response);\n" + + "\tif (outputHeader != null) {\n" + + "\t\toutputHeader.putAll(response.getHeaders());\n" + + "\t}\n" + + "\tint statusCode = response.getStatusCode();\n" + + "\tif (statusCode != Consts.CONST_HTTP_OK) {\n" + + "\t\tString requestId = GetRequestId(response.getHeaders());\n" + + "\t\ttry {\n" + + "\t\t\tJSONObject object = parseResponseBody(response, requestId);\n" + + "\t\t\tif(object.getString(Consts.CONST_ERROR_CODE).equals(\"Unauthorized\") && !isDipperRefresh){\n" + + "\t\t\t\tStsIdentity stsIdentity = stsService.getStsIdentity();\n" + + "\t\t\t\tcredentials.setAccessKeyId(stsIdentity.getAccessKeyId());\n" + + "\t\t\t\tcredentials.setAccessKeySecret(stsIdentity.getAccessKeySecret());\n" + + "\t\t\t\tcredentials.setSecurityToken(stsIdentity.getSecurityToken());\n" + + "\t\t\t\theaders.put(\"dipper_sts_refresh\", \"1\");\n" + + "\t\t\t\treturn SendData(project, method, resourceUri, parameters, headers, body, outputHeader, serverIp)\n" + + "\t\t\t}\n" + + "\t\t\tErrorCheck(object, requestId, statusCode);\n" + + "\t\t} catch (LogException ex) {\n" + + "\t\t\tex.SetHttpCode(response.getStatusCode());\n" + + "\t\t\tthrow ex;\n" + + "\t\t}\n" + + "\t}\n" + + "} catch (ServiceException e) {\n" + + "\tthrow new LogException(\"RequestError\", \"Web request failed: \" + e.getMessage(), e, \"\");\n" + + "} catch (ClientException e) {\n" + + "\tthrow new LogException(\"RequestError\", \"Web request failed: \" + e.getMessage(), e, \"\");\n" + + "} finally {\n" + + "\ttry {\n" + + "\t\tif (response != null) {\n" + + "\t\t\tresponse.close();\n" + + "\t\t}\n" + + "\t} catch (IOException ignore) {}\n" + + "}\n" + + "return response;"; + + @Override + public byte[] transform(ClassLoader loader, String className, Class classBeingRedefined, ProtectionDomain protectionDomain, byte[] classfileBuffer) throws IllegalClassFormatException { + if (className.contains("Client")) { + final ClassPool classPool = ClassPool.getDefault(); + final CtClass clazz; + try { + clazz = classPool.get("com.aliyun.openservices.log.Client"); + CtField ctField = new CtField(classPool.get("org.apache.rocketmq.streams.sls.source.sts.StsService"), "stsService", clazz); + ctField.setModifiers(Modifier.PUBLIC); + clazz.addMethod(CtNewMethod.setter("setStsService", ctField)); + CtMethod[] convertToAbbr = clazz.getDeclaredMethods("SendData"); + for (CtMethod ctm : convertToAbbr) { + if (name.equals(ctm.getLongName())) { + ctm.setBody(this.methodBody); + break; + } + } + byte[] byteCode = clazz.toBytecode(); + //detach的意思是将内存中曾经被javassist加载过的Date对象移除,如果下次有需要在内存中找不到会重新走javassist加载 + clazz.detach(); + return byteCode; + } catch (NotFoundException e) { + e.printStackTrace(); + } catch (CannotCompileException e) { + e.printStackTrace(); + } catch (IOException e) { + e.printStackTrace(); + } + + } + // 如果返回null则字节码不会被修改 + return null; + } +} diff --git a/rocketmq-streams-sts/src/main/java/org/apache/rocketmq/streams/sts/StsIdentity.java b/rocketmq-streams-sts/src/main/java/org/apache/rocketmq/streams/sts/StsIdentity.java new file mode 100644 index 00000000..c0958793 --- /dev/null +++ b/rocketmq-streams-sts/src/main/java/org/apache/rocketmq/streams/sts/StsIdentity.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.sts; + +public class StsIdentity { + + String accessKeyId; + String accessKeySecret; + String securityToken; + String expiration; + + public StsIdentity() { + } + + public StsIdentity(String accessKeyId, String accessKeySecret, String securityToken, String expiration) { + this.accessKeyId = accessKeyId; + this.accessKeySecret = accessKeySecret; + this.securityToken = securityToken; + this.expiration = expiration; + } + + public String getAccessKeyId() { + return accessKeyId; + } + + public void setAccessKeyId(String accessKeyId) { + this.accessKeyId = accessKeyId; + } + + public String getAccessKeySecret() { + return accessKeySecret; + } + + public void setAccessKeySecret(String accessKeySecret) { + this.accessKeySecret = accessKeySecret; + } + + public String getSecurityToken() { + return securityToken; + } + + public void setSecurityToken(String securityToken) { + this.securityToken = securityToken; + } + + public String getExpiration() { + return expiration; + } + + public void setExpiration(String expiration) { + this.expiration = expiration; + } + + @Override + public String toString() { + return "StsIdentity{" + "\n" + + "accessKeyId='" + accessKeyId + '\'' + "\n" + + "accessKeySecret='" + accessKeySecret + '\'' + "\n" + + "securityToken='" + securityToken + '\'' + "\n" + + "expiration='" + expiration + '\'' + "\n" + + '}'; + } +} diff --git a/rocketmq-streams-sts/src/main/java/org/apache/rocketmq/streams/sts/StsService.java b/rocketmq-streams-sts/src/main/java/org/apache/rocketmq/streams/sts/StsService.java new file mode 100644 index 00000000..9dcca8c3 --- /dev/null +++ b/rocketmq-streams-sts/src/main/java/org/apache/rocketmq/streams/sts/StsService.java @@ -0,0 +1,177 @@ +/* + * 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.sts; + +import com.aliyuncs.DefaultAcsClient; +import com.aliyuncs.exceptions.ClientException; +import com.aliyuncs.http.MethodType; +import com.aliyuncs.profile.DefaultProfile; +import com.aliyuncs.profile.IClientProfile; +import com.aliyuncs.sts.model.v20150401.AssumeRoleWithServiceIdentityRequest; +import com.aliyuncs.sts.model.v20150401.AssumeRoleWithServiceIdentityResponse; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; + +import java.util.Properties; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class StsService { + + public static final String STS_ACCESS_ID = "stsAccessId"; + public static final String STS_ACCESS_KEY = "stsAccessKey"; + public static final String STS_ROLE_ARN = "stsRoleArn"; + public static final String STS_ASSUME_ROLE_FOR = "stsAssumeRoleFor"; + public static final String STS_TIME_OUT_SECOND = "stsTimeoutSecond"; + public static final String STS_RAM_ENDPOINT = "ramEndpoint"; + public static final String STS_SESSION_NAME_PREFIX = "stsSessionPrefix"; + private static final Logger logger = LoggerFactory.getLogger(StsService.class); + private static final String CACHE_STS_KEY = "stsCacheKey"; + private static final String DEFAULT_RAM_ENDPOINT = "sts-inner.aliyuncs.com"; + private static final int DEFAULT_STS_EXPIRE_SECOND = 86400; + + String accessId; + String accessKey; + String roleArn; + String ramEndPoint; + String stsSessionPrefix; + String stsAssumeRoleFor; + int stsExpireSeconds; + String stsSessionName; + LoadingCache cacheLoader; + + public StsService() { + + } + + public StsService(Properties properties) { + this.accessId = CheckParametersUtils.checkIfNullThrowException(properties.getProperty(STS_ACCESS_ID), STS_ACCESS_ID + " is null, pls check params"); + this.accessKey = CheckParametersUtils.checkIfNullThrowException(properties.getProperty(STS_ACCESS_KEY), STS_ACCESS_KEY + " is null, pls check params"); + this.roleArn = CheckParametersUtils.checkIfNullThrowException(properties.getProperty(STS_ROLE_ARN), STS_ROLE_ARN + " is null, pls check params"); + this.stsAssumeRoleFor = CheckParametersUtils.checkIfNullThrowException(properties.getProperty(STS_ASSUME_ROLE_FOR), STS_ASSUME_ROLE_FOR + " is null, pls check params"); + this.ramEndPoint = CheckParametersUtils.checkIfNullReturnDefault(properties.getProperty(STS_RAM_ENDPOINT), DEFAULT_RAM_ENDPOINT); + this.stsSessionPrefix = CheckParametersUtils.checkIfNullReturnDefault(properties.getProperty(STS_SESSION_NAME_PREFIX), "DEFAULT_SESSION_NAME"); + this.stsExpireSeconds = CheckParametersUtils.checkIfNullReturnDefaultInt(properties.get(STS_TIME_OUT_SECOND), DEFAULT_STS_EXPIRE_SECOND); + createCacheLoader(); + } + + public String getAccessId() { + return accessId; + } + + public void setAccessId(String accessId) { + this.accessId = accessId; + } + + public String getAccessKey() { + return accessKey; + } + + public void setAccessKey(String accessKey) { + this.accessKey = accessKey; + } + + public String getRoleArn() { + return roleArn; + } + + public void setRoleArn(String roleArn) { + this.roleArn = roleArn; + } + + public String getRamEndPoint() { + return ramEndPoint; + } + + public void setRamEndPoint(String ramEndPoint) { + this.ramEndPoint = ramEndPoint; + } + + public String getStsSessionPrefix() { + return stsSessionPrefix; + } + + public void setStsSessionPrefix(String stsSessionPrefix) { + this.stsSessionPrefix = stsSessionPrefix; + } + + public String getStsAssumeRoleFor() { + return stsAssumeRoleFor; + } + + public void setStsAssumeRoleFor(String stsAssumeRoleFor) { + this.stsAssumeRoleFor = stsAssumeRoleFor; + } + + public int getStsExpireSeconds() { + return stsExpireSeconds; + } + + public void setStsExpireSeconds(int stsExpireSeconds) { + this.stsExpireSeconds = stsExpireSeconds; + } + + public int getRefreshTimeSecond() { + return stsExpireSeconds / 2; + } + + public StsIdentity getStsIdentity() throws ExecutionException { + if (cacheLoader == null) { + createCacheLoader(); + } + return cacheLoader.get(CACHE_STS_KEY); + } + + private final void createCacheLoader() { + this.cacheLoader = CacheBuilder.newBuilder().concurrencyLevel(5).initialCapacity(1).maximumSize(3).expireAfterWrite(getRefreshTimeSecond(), TimeUnit.SECONDS).build(new StsIdentityCacheLoader()); + } + + private DefaultAcsClient getRamClient() throws ClientException { + DefaultProfile.addEndpoint("", "", "Sts", ramEndPoint); + // 构造default profile(参数留空,无需添加region ID) + IClientProfile profile = DefaultProfile.getProfile("", accessId, accessKey); + // 用profile构造client + return new DefaultAcsClient(profile); + } + + private final String refreshSessionName() { + this.stsSessionName = stsSessionPrefix + "_" + System.currentTimeMillis(); + return stsSessionName; + } + + class StsIdentityCacheLoader extends CacheLoader { + + @Override + public StsIdentity load(String s) throws ClientException { + DefaultAcsClient client = getRamClient(); + final AssumeRoleWithServiceIdentityRequest request = new AssumeRoleWithServiceIdentityRequest(); + request.setMethod(MethodType.POST); + request.setRoleArn(roleArn); + request.setRoleSessionName(refreshSessionName()); + request.setAssumeRoleFor(stsAssumeRoleFor); + request.setDurationSeconds((long)stsExpireSeconds); + final AssumeRoleWithServiceIdentityResponse response = client.getAcsResponse(request); + StsIdentity stsIdentity = new StsIdentity(response.getCredentials().getAccessKeyId(), response.getCredentials().getAccessKeySecret(), response.getCredentials().getSecurityToken(), response.getCredentials().getExpiration()); + logger.info("CacheLoader refresh stsToken success, accessId is {}. ", stsIdentity.getAccessKeyId()); + return stsIdentity; + } + } +} diff --git a/rocketmq-streams-tasks/pom.xml b/rocketmq-streams-tasks/pom.xml new file mode 100644 index 00000000..6e1eb5af --- /dev/null +++ b/rocketmq-streams-tasks/pom.xml @@ -0,0 +1,31 @@ + + + + rocketmq-streams + org.apache.rocketmq + 2.0.1-SNAPSHOT + + 4.0.0 + + rocketmq-streams-tasks + ROCKETMQ STREAMS :: tasks + + + + + org.apache.rocketmq + rocketmq-streams-dispatcher + + + org.apache.rocketmq + rocketmq-streams-db-operator + + + junit + junit + + + + \ No newline at end of file diff --git a/rocketmq-streams-tasks/src/main/java/org/apache/rocketmq/streams/tasks/entity/WeightChainPipeline.java b/rocketmq-streams-tasks/src/main/java/org/apache/rocketmq/streams/tasks/entity/WeightChainPipeline.java new file mode 100644 index 00000000..715350c3 --- /dev/null +++ b/rocketmq-streams-tasks/src/main/java/org/apache/rocketmq/streams/tasks/entity/WeightChainPipeline.java @@ -0,0 +1,42 @@ +/* + * 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.tasks.entity; + +import java.io.Serializable; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; + +public class WeightChainPipeline implements Serializable { + + private ChainPipeline chainPipeline; + private Integer weight; + + public ChainPipeline getChainPipeline() { + return chainPipeline; + } + + public void setChainPipeline(ChainPipeline chainPipeline) { + this.chainPipeline = chainPipeline; + } + + public Integer getWeight() { + return weight; + } + + public void setWeight(Integer weight) { + this.weight = weight; + } +} diff --git a/rocketmq-streams-tasks/src/test/java/org/apache/rocketmq/streams/dispatcher/DispatcherTest.java b/rocketmq-streams-tasks/src/test/java/org/apache/rocketmq/streams/dispatcher/DispatcherTest.java new file mode 100644 index 00000000..8a9e0404 --- /dev/null +++ b/rocketmq-streams-tasks/src/test/java/org/apache/rocketmq/streams/dispatcher/DispatcherTest.java @@ -0,0 +1,42 @@ +/* + * 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.dispatcher; + +import java.util.ArrayList; +import java.util.List; +import org.apache.rocketmq.streams.common.utils.FileUtil; +import org.apache.rocketmq.streams.dispatcher.impl.RocketmqDispatcher; +import org.junit.Test; + +public class DispatcherTest { + private transient RocketmqDispatcher dispatcher; + private transient IDispatcherCallback balanceCallback; + private String filePath = "/tmp/dipper/task.txt"; + + @Test + public void testTask() { + List tasks = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + tasks.add("task_" + i); + } + + tasks.remove("task_" + 11); + tasks.remove("task_" + 12); + FileUtil.write(filePath, tasks); + } + +} diff --git a/rocketmq-streams-tasks/src/test/java/org/apache/rocketmq/streams/dispatcher/TestStreamsDispatcherV2.java b/rocketmq-streams-tasks/src/test/java/org/apache/rocketmq/streams/dispatcher/TestStreamsDispatcherV2.java new file mode 100644 index 00000000..b5b03abb --- /dev/null +++ b/rocketmq-streams-tasks/src/test/java/org/apache/rocketmq/streams/dispatcher/TestStreamsDispatcherV2.java @@ -0,0 +1,56 @@ +/* + * 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.dispatcher; + +import java.util.List; +import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; +import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyContext; +import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyStatus; +import org.apache.rocketmq.client.consumer.listener.MessageListenerOrderly; +import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.common.message.MessageExt; +import org.junit.Before; +import org.junit.Test; + +public class TestStreamsDispatcherV2 { + + @Before + public void init() { + try { + DefaultMQPushConsumer defaultMQPushConsumer = new DefaultMQPushConsumer("test"); + defaultMQPushConsumer.setNamesrvAddr("localhost:9876"); + defaultMQPushConsumer.setConsumerGroup("please_rename_unique_group_name_4"); + defaultMQPushConsumer.subscribe("TopicTest", "*"); + defaultMQPushConsumer.setInstanceName("instance_1"); + defaultMQPushConsumer.registerMessageListener(new MessageListenerOrderly() { + @Override public ConsumeOrderlyStatus consumeMessage(List list, ConsumeOrderlyContext context) { + return null; + } + }); + defaultMQPushConsumer.start(); + + } catch (MQClientException e) { + throw new RuntimeException(e); + } + } + + @Test + public void testDispatcher() { + System.out.println("hello world"); + } + +} diff --git a/rocketmq-streams-transport-minio/pom.xml b/rocketmq-streams-transport-minio/pom.xml index ac347d4c..951d3889 100755 --- a/rocketmq-streams-transport-minio/pom.xml +++ b/rocketmq-streams-transport-minio/pom.xml @@ -15,14 +15,14 @@ See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 org.apache.rocketmq rocketmq-streams - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT rocketmq-streams-transport-minio jar diff --git a/rocketmq-streams-transport-minio/src/main/java/org/apache/rocketmq/streams/transport/minio/MinioFileTransport.java b/rocketmq-streams-transport-minio/src/main/java/org/apache/rocketmq/streams/transport/minio/MinioFileTransport.java index 79b51546..b6da97b9 100644 --- a/rocketmq-streams-transport-minio/src/main/java/org/apache/rocketmq/streams/transport/minio/MinioFileTransport.java +++ b/rocketmq-streams-transport-minio/src/main/java/org/apache/rocketmq/streams/transport/minio/MinioFileTransport.java @@ -25,8 +25,8 @@ import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.configuration.SystemContext; import org.apache.rocketmq.streams.common.model.ServiceName; import org.apache.rocketmq.streams.common.transport.AbstractFileTransport; import org.apache.rocketmq.streams.common.transport.IFileTransport; @@ -44,10 +44,10 @@ public class MinioFileTransport extends AbstractFileTransport { protected MinioClient minioClient; public MinioFileTransport() { - this.ak = ComponentCreator.getProperties().getProperty(ConfigureFileKey.FILE_TRANSPORT_AK); - this.sk = ComponentCreator.getProperties().getProperty(ConfigureFileKey.FILE_TRANSPORT_SK); - this.endpoint = ComponentCreator.getProperties().getProperty(ConfigureFileKey.FILE_TRANSPORT_ENDPOINT); - this.dirpperDir = ComponentCreator.getProperties().getProperty(ConfigureFileKey.FILE_TRANSPORT_DIPPER_DIR); + this.ak = SystemContext.getProperty(ConfigurationKey.FILE_TRANSPORT_AK); + this.sk = SystemContext.getProperty(ConfigurationKey.FILE_TRANSPORT_SK); + this.endpoint = SystemContext.getProperty(ConfigurationKey.FILE_TRANSPORT_ENDPOINT); + this.dirpperDir = SystemContext.getProperty(ConfigurationKey.FILE_TRANSPORT_DIPPER_DIR); if (StringUtil.isEmpty(this.dirpperDir)) { this.dirpperDir = "dipper_files"; } diff --git a/rocketmq-streams-transport-minio/src/test/java/org/apache/rocketmq/streams/configuable/model/Person.java b/rocketmq-streams-transport-minio/src/test/java/org/apache/rocketmq/streams/configuable/model/Person.java index 04b99bba..7972c6c3 100644 --- a/rocketmq-streams-transport-minio/src/test/java/org/apache/rocketmq/streams/configuable/model/Person.java +++ b/rocketmq-streams-transport-minio/src/test/java/org/apache/rocketmq/streams/configuable/model/Person.java @@ -34,7 +34,7 @@ public static Person createPerson(String namespace) { Person person = new Person(); person.setNameSpace(namespace); person.setType("person"); - person.setConfigureName("Chris"); + person.setName("Chris"); person.setName("Chris"); List addresses = new ArrayList<>(); addresses.add("huilongguan"); diff --git a/rocketmq-streams-transport-minio/src/test/java/org/apache/rocketmq/streams/configurable/model/Person.java b/rocketmq-streams-transport-minio/src/test/java/org/apache/rocketmq/streams/configurable/model/Person.java index 29aa27d7..5df61a64 100644 --- a/rocketmq-streams-transport-minio/src/test/java/org/apache/rocketmq/streams/configurable/model/Person.java +++ b/rocketmq-streams-transport-minio/src/test/java/org/apache/rocketmq/streams/configurable/model/Person.java @@ -33,7 +33,7 @@ public static Person createPerson(String namespace) { Person person = new Person(); person.setNameSpace(namespace); person.setType("person"); - person.setConfigureName("Chris"); + person.setName("Chris"); person.setName("Chris"); List addresses = new ArrayList<>(); addresses.add("huilongguan"); diff --git a/rocketmq-streams-transport-minio/src/test/resources/log4j.xml b/rocketmq-streams-transport-minio/src/test/resources/log4j.xml old mode 100755 new mode 100644 diff --git a/rocketmq-streams-window/pom.xml b/rocketmq-streams-window/pom.xml index 7a2be2dd..f468b810 100755 --- a/rocketmq-streams-window/pom.xml +++ b/rocketmq-streams-window/pom.xml @@ -15,14 +15,14 @@ See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 org.apache.rocketmq rocketmq-streams - 1.0.2-SNAPSHOT + 2.0.1-SNAPSHOT rocketmq-streams-window jar @@ -47,9 +47,9 @@ rocketmq-streams-dim - org.rocksdb - rocksdbjni + org.apache.rocketmq + rocketmq-streams-state - + diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/stage/ShuffleOutputChainStage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/stage/ShuffleOutputChainStage.java new file mode 100644 index 00000000..ba4c5a78 --- /dev/null +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/stage/ShuffleOutputChainStage.java @@ -0,0 +1,196 @@ +/* + * 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.stage; + +import com.alibaba.fastjson.JSONObject; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; +import org.apache.rocketmq.streams.common.channel.sink.AbstractSupportShuffleSink; +import org.apache.rocketmq.streams.common.channel.source.ISource; +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.checkpoint.CheckPointState; +import org.apache.rocketmq.streams.common.configurable.annotation.ConfigurableReference; +import org.apache.rocketmq.streams.common.context.AbstractContext; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.context.MessageHeader; +import org.apache.rocketmq.streams.common.model.SQLCompileContextForSource; +import org.apache.rocketmq.streams.common.topology.IWindow; +import org.apache.rocketmq.streams.common.topology.model.AbstractChainStage; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; +import org.apache.rocketmq.streams.common.utils.TraceUtil; +import org.apache.rocketmq.streams.window.WindowConstants; +import org.apache.rocketmq.streams.window.operator.AbstractShuffleWindow; +import org.apache.rocketmq.streams.window.operator.AbstractWindow; +import org.apache.rocketmq.streams.window.shuffle.ShuffleManager; +import org.apache.rocketmq.streams.window.shuffle.ShuffleSink; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ShuffleOutputChainStage extends AbstractChainStage { + private static final Logger LOGGER = LoggerFactory.getLogger(ShuffleOutputChainStage.class); + protected transient ShuffleSink shuffleSink; + @ConfigurableReference protected AbstractShuffleWindow window; + protected transient AtomicBoolean isFirstMessage = new AtomicBoolean(true); + + public ShuffleOutputChainStage() { + this.setDiscription("Shuffle_Send"); + super.entityName = "window"; + } + + @Override + protected IMessage handleMessage(IMessage message, AbstractContext context) { + shuffleSend(message, context); + if (!window.isSynchronous()) { + context.breakExecute(); + } + + return message; + } + + @Override public void startJob() { + ISource source = ((ChainPipeline) getPipeline()).getSource(); + if (source == null) { + /** + * sql场景,会用嵌套的pipelinebuilder 创建,source可能无法传递过来 + */ + source = SQLCompileContextForSource.getInstance().get(); + } + Pair, AbstractSupportShuffleSink> shuffleSourceAndSink = ShuffleManager.createOrGetShuffleSourceAndSink(source, window); + + if (shuffleSourceAndSink == null) { + return; + } + ShuffleSink shuffleSink = new ShuffleSink(); + shuffleSink.setNameSpace(getNameSpace()); + shuffleSink.setSink(shuffleSourceAndSink.getRight()); + shuffleSink.setWindow(window); + shuffleSink.init(); + window.start(); + window.setShuffleSink(shuffleSink); + this.shuffleSink = shuffleSink; + } + + @Override public void stopJob() { + super.destroy(); + this.shuffleSink.destroy(); + this.window.destroy(); + this.isFirstMessage.set(true); + } + + protected void shuffleSend(IMessage message, AbstractContext context) { + if (isFirstMessage.compareAndSet(true, false) && !message.getHeader().isSystemMessage()) { + //当收到第一条消息的时候 + JSONObject msg = new JSONObject(); + msg.put(window.getTimeFieldName(), message.getMessageBody().getString(window.getTimeFieldName())); + shuffleSink.sendNotifyToShuffleSource(message.getHeader().getQueueId(), msg); + } + + if (StringUtils.isNotEmpty(window.getSizeVariable())) { + if (message.getMessageBody().containsKey(window.getSizeVariable())) { + try { + window.setSizeInterval(window.getSizeAdjust() * message.getMessageBody().getInteger(window.getSizeVariable())); + } catch (Exception e) { + LOGGER.error("failed in getting the size value, message = " + message.toString(), e); + } + } + } + if (StringUtils.isNotEmpty(window.getSlideVariable())) { + if (message.getMessageBody().containsKey(window.getSlideVariable())) { + try { + window.setSlideInterval(window.getSlideAdjust() * message.getMessageBody().getInteger(window.getSlideVariable())); + } catch (Exception e) { + LOGGER.error("failed in getting the slide value, message = " + message.toString(), e); + } + } + } + JSONObject msg = message.getMessageBody(); + msg.put(MessageHeader.class.getSimpleName(), message.getHeader()); + msg.put(AbstractWindow.class.getSimpleName(), this); + shuffleSink.batchAdd(message); + TraceUtil.debug(message.getHeader().getTraceId(), "origin message in"); + } + + @Override + public boolean isAsyncNode() { + if (window != null) { + return !window.isSynchronous(); + } + return false; + } + + @Override + public void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage) { + if (shuffleSink == null) {//over window windowcache is null + return; + } + if (message.getHeader().isNeedFlush()) { + if (shuffleSink != null && message.getHeader().getCheckpointQueueIds() != null && message.getHeader().getCheckpointQueueIds().size() > 0) { + shuffleSink.checkpoint(message.getHeader().getCheckpointQueueIds()); + } else { + if (shuffleSink != null) { + Set queueIds = new HashSet<>(); + queueIds.add(message.getHeader().getQueueId()); + shuffleSink.checkpoint(queueIds); + } + + } + + } + CheckPointState checkPointState = new CheckPointState(); + checkPointState.setQueueIdAndOffset(shuffleSink.getFinishedQueueIdAndOffsets(checkPointMessage)); + checkPointMessage.reply(checkPointState); + } + + @Override + public void addNewSplit(IMessage message, AbstractContext context, NewSplitMessage newSplitMessage) { + + } + + @Override + public void removeSplit(IMessage message, AbstractContext context, RemoveSplitMessage removeSplitMessage) { + + } + + @Override + public void batchMessageFinish(IMessage message, AbstractContext context, BatchFinishMessage checkPointMessage) { + checkPointMessage.getMsg().getMessageBody().put(WindowConstants.ORIGIN_MESSAGE_HEADER, JSONObject.toJSONString(message.getHeader())); + shuffleSink.finishBatchMsg(checkPointMessage); + } + + public IWindow getWindow() { + return window; + } + + public void setWindow(AbstractShuffleWindow window) { + if (window == null) { + throw new RuntimeException("can not get shuffle window"); + } + this.window = window; + } + + @Override + public String getEntityName() { + return super.entityName; + } + +} diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/stage/ShuffleSourceChainStage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/stage/ShuffleSourceChainStage.java new file mode 100644 index 00000000..479b3bb6 --- /dev/null +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/stage/ShuffleSourceChainStage.java @@ -0,0 +1,440 @@ +/* + * 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.stage; + +import com.alibaba.fastjson.JSONArray; +import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; +import org.apache.rocketmq.streams.common.channel.sink.AbstractSupportShuffleSink; +import org.apache.rocketmq.streams.common.channel.source.ISource; +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.channel.source.systemmsg.WaterMarkNotifyMessage; +import org.apache.rocketmq.streams.common.checkpoint.CheckPointMessage; +import org.apache.rocketmq.streams.common.checkpoint.CheckPointState; +import org.apache.rocketmq.streams.common.configurable.annotation.ConfigurableReference; +import org.apache.rocketmq.streams.common.context.AbstractContext; +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.MessageOffset; +import org.apache.rocketmq.streams.common.interfaces.INeedAalignWaiting; +import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; +import org.apache.rocketmq.streams.common.interfaces.ISystemMessage; +import org.apache.rocketmq.streams.common.model.NameCreator; +import org.apache.rocketmq.streams.common.model.SQLCompileContextForSource; +import org.apache.rocketmq.streams.common.topology.IWindow; +import org.apache.rocketmq.streams.common.topology.model.AbstractChainStage; +import org.apache.rocketmq.streams.common.topology.model.ChainPipeline; +import org.apache.rocketmq.streams.common.utils.CollectionUtil; +import org.apache.rocketmq.streams.common.utils.CompressUtil; +import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.common.utils.IdUtil; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; +import org.apache.rocketmq.streams.common.utils.ReflectUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.common.utils.TraceUtil; +import org.apache.rocketmq.streams.db.driver.orm.ORMUtil; +import org.apache.rocketmq.streams.window.WindowConstants; +import org.apache.rocketmq.streams.window.debug.DebugWriter; +import org.apache.rocketmq.streams.window.model.WindowInstance; +import org.apache.rocketmq.streams.window.operator.AbstractShuffleWindow; +import org.apache.rocketmq.streams.window.operator.AbstractWindow; +import org.apache.rocketmq.streams.window.operator.impl.WindowOperator; +import org.apache.rocketmq.streams.window.shuffle.ShuffleCache; +import org.apache.rocketmq.streams.window.shuffle.ShuffleManager; +import org.apache.rocketmq.streams.window.sqlcache.impl.SQLElement; +import org.apache.rocketmq.streams.window.storage.ShufflePartitionManager; +import org.apache.rocketmq.streams.window.trigger.WindowTrigger; +import org.apache.rocketmq.streams.window.util.ShuffleUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ShuffleSourceChainStage extends AbstractChainStage { + public static final String SHUFFLE_OFFSET = "SHUFFLE_OFFSET"; + public static final String IS_COMPRESSION_MSG = "_is_compress_msg"; + public static final String COMPRESSION_MSG_DATA = "_compress_msg"; + public static final String MSG_FROM_SOURCE = "msg_from_source"; + public static final String ORIGIN_OFFSET = "origin_offset"; + public static final String ORIGIN_QUEUE_ID = "origin_queue_id"; + public static final String ORIGIN_QUEUE_IS_LONG = "origin_offset_is_LONG"; + protected static final String SHUFFLE_MESSAGES = "SHUFFLE_MESSAGES"; + private static final Logger LOGGER = LoggerFactory.getLogger(ShuffleSourceChainStage.class); + private static final String SHUFFLE_TRACE_ID = "SHUFFLE_TRACE_ID"; + /** + * 消息所属的window + */ + protected String MSG_OWNER = "MSG_OWNER"; + protected transient ISource shuffleSource;//消费shuffle的数据 + @ConfigurableReference protected AbstractShuffleWindow window; + + protected transient ShuffleCache shuffleCache;//消费的数据先攒批,然后调回调接口IShuffleCallback做窗口计算(accumulate) + protected transient WindowTrigger windowTrigger;//触发窗口,当窗口触发时调回调接口IShuffleCallback做fire,fire后会调回调接口clearWindowInstance清理资源,sendFireMessage发送结果给后续的节点 + + /** + * 每个分片,已经确定处理的最大offset + */ + protected transient Map split2MaxOffsets; + protected transient boolean isWindowTest = false; + protected transient AtomicLong COUNT; + /** + * 当前管理的分片 + */ + private Set currentQueueIds; + + public ShuffleSourceChainStage() { + setDiscription("Shuffle_Receive"); + } + + @Override public void startJob() { + COUNT = new AtomicLong(0); + this.split2MaxOffsets = new HashMap<>(); + this.window.setFireReceiver(getReceiverAfterCurrentNode()); + this.shuffleCache = new ShuffleCache(window); + this.shuffleCache.init(); + this.shuffleCache.openAutoFlush(); + + this.windowTrigger = new WindowTrigger(window); + this.windowTrigger.start(); + window.setWindowTrigger(windowTrigger); + this.window.start(); + window.getEventTimeManager().setSource(((ChainPipeline) this.getPipeline()).getSource()); + initShuffleSource(); + if (shuffleSource != null) {//view场景,有可能source为null + shuffleSource.start((IStreamOperator) (message, context) -> { + consumeShuffleMessage(message, context); + stageMetric.outCalculate(); + return null; + }); + } + + } + + @Override public void stopJob() { + this.shuffleCache.destroy(); + this.windowTrigger.destroy(); + this.shuffleSource.destroy(); + this.window.destroy(); + } + + protected void initShuffleSource() { + ISource source = ((ChainPipeline) getPipeline()).getSource(); + if (source == null) { + /** + * sql场景,会用嵌套的pipelinebuilder 创建,source可能无法传递过来 + */ + source = SQLCompileContextForSource.getInstance().get(); + } + Pair, AbstractSupportShuffleSink> shuffleSourceAndSink = ShuffleManager.createOrGetShuffleSourceAndSink(source, window); + if (shuffleSourceAndSink == null) { + return; + } + this.shuffleSource = shuffleSourceAndSink.getLeft(); + this.shuffleSource.init(); + } + + protected void consumeShuffleMessage(IMessage oriMessage, AbstractContext context) { + if (oriMessage.getHeader().isSystemMessage()) { + doSystemMessage(oriMessage, context); + return; + + } + if (oriMessage.getMessageBody().getBooleanValue(IS_COMPRESSION_MSG)) { + byte[] bytes = oriMessage.getMessageBody().getBytes(COMPRESSION_MSG_DATA); + String msgStr = CompressUtil.unGzip(bytes); + oriMessage.setMessageBody(JSONObject.parseObject(msgStr)); + } + /* + 过滤不是这个window的消息,一个shuffle通道,可能多个window共享,这里过滤掉非本window的消息 + */ + boolean isFilter = filterNotOwnerMessage(oriMessage); + if (isFilter) { + return; + } + String queueId = oriMessage.getHeader().getQueueId(); + JSONArray messages = oriMessage.getMessageBody().getJSONArray(SHUFFLE_MESSAGES); + if (messages == null) { + return; + } + + String traceId = oriMessage.getMessageBody().getString(SHUFFLE_TRACE_ID); + if (!StringUtil.isEmpty(traceId)) { + TraceUtil.debug(traceId, "shuffle message in", "received message size:" + messages.size()); + } + for (Object obj : messages) { + + IMessage message = restoreMsg((JSONObject) obj); + + if (message.getHeader().isSystemMessage()) { + doSystemMessage(message, context); + continue; + } + stageMetric.startCalculate(message); + message.getHeader().setQueueId(queueId); + message.getMessageBody().put(SHUFFLE_OFFSET, oriMessage.getHeader().getOffset()); + window.updateMaxEventTime(message); + if (isRepeatMessage(message, queueId)) { + continue; + } + List windowInstances = window.queryOrCreateWindowInstance(message, queueId); + if (windowInstances == null || windowInstances.isEmpty()) { + LOGGER.warn("[{}][{}] WindowInstance_CreateNull_Error_TimeOn({})]", IdUtil.instanceId(), NameCreator.getFirstPrefix(window.getName(), IWindow.TYPE), DateUtil.longToString(WindowInstance.getOccurTime(this.window, message))); + continue; + } + for (WindowInstance windowInstance : windowInstances) { + String windowInstanceId = windowInstance.createWindowInstanceId(); + //new instance, not need load data from remote + if (windowInstance.isNewWindowInstance()) { + window.getSqlCache().addCache(new SQLElement(windowInstance.getSplitId(), windowInstanceId, ORMUtil.createBatchReplaceSQL(windowInstance))); + windowInstance.setNewWindowInstance(false); + ShufflePartitionManager.getInstance().setWindowInstanceFinished(windowInstance.createWindowInstanceId()); + } + } + + message.getMessageBody().put(WindowInstance.class.getSimpleName(), windowInstances); + message.getMessageBody().put(AbstractWindow.class.getSimpleName(), window); + +// if (DebugWriter.getDebugWriter(window.getName()).isOpenDebug()) { +// List msgs = new ArrayList<>(); +// msgs.add(message); +// DebugWriter.getDebugWriter(window.getName()).writeShuffleReceiveBeforeCache(window, msgs, queueId); +// } + + for (WindowInstance windowInstance : windowInstances) { + windowTrigger.updateWindowInstanceLastUpdateTime(windowInstance); + } + shuffleCache.batchAdd(message); + } + if (isWindowTest) { + long count = COUNT.addAndGet(messages.size()); + LOGGER.debug("[{}] receive shuffle msg count is {}", window.getName(), count); + } + + } + + protected IMessage restoreMsg(JSONObject obj) { + Message message = new Message(obj); + if (obj.get(WindowConstants.IS_SYSTEME_MSG) != null && obj.getBoolean(WindowConstants.IS_SYSTEME_MSG)) { + message.getHeader().setSystemMessage(true); + message.setSystemMessage((ISystemMessage) obj.getObject(WindowConstants.SYSTEME_MSG, ReflectUtil.forClass(obj.getString(WindowConstants.SYSTEME_MSG_CLASS)))); + message.getHeader().setQueueId(obj.getString(ORIGIN_QUEUE_ID)); + } + return message; + } + + /** + * do system message + * + * @param oriMessage + * @param context + */ + protected void doSystemMessage(IMessage oriMessage, AbstractContext context) { + ISystemMessage systemMessage = oriMessage.getSystemMessage(); + MessageHeader messageHeader = ShuffleUtil.getMessageHeader(oriMessage.getMessageBody()); + if (messageHeader != null) { + oriMessage.setHeader(messageHeader); + } + if (systemMessage.getSystemMessageType() == ISystemMessage.CHECK_POINT) { + this.checkpoint(oriMessage, context, (CheckPointMessage) systemMessage); + } else if (systemMessage.getSystemMessageType() == ISystemMessage.SPLIT_ADD) { + this.addNewSplit(oriMessage, context, (NewSplitMessage) systemMessage); + } else if (systemMessage.getSystemMessageType() == ISystemMessage.SPLIT_REMOVE) { + this.removeSplit(oriMessage, context, (RemoveSplitMessage) systemMessage); + } else if (systemMessage.getSystemMessageType() == ISystemMessage.BATCH_FINISH) { + boolean canSend = true; + if (this.window instanceof INeedAalignWaiting) { + INeedAalignWaiting needAalignWaiting = (INeedAalignWaiting) window; + canSend = needAalignWaiting.alignWaiting(oriMessage); + } + if (!canSend) { + context.breakExecute(); + return; + } + this.batchMessageFinish(oriMessage, context, (BatchFinishMessage) systemMessage); + } else if (systemMessage.getSystemMessageType() == ISystemMessage.WATER_MARK) { + this.notifyWaterMarkMessage(oriMessage, context, (WaterMarkNotifyMessage) systemMessage); + } else { + throw new RuntimeException("can not support this system message " + systemMessage.getClass().getName() + "," + systemMessage.getSystemMessageType()); + } + afterFlushCallback(oriMessage, context); + } + + protected void notifyWaterMarkMessage(IMessage message, AbstractContext context, WaterMarkNotifyMessage message1) { + window.updateMaxEventTime(message); + + } + + /** + * load ori split consume offset + * + * @param newSplitMessage + */ + protected void loadSplitProgress(NewSplitMessage newSplitMessage) { + for (String queueId : newSplitMessage.getSplitIds()) { + Map result = window.getWindowMaxValueManager().loadOffsets(window.getName(), queueId); + if (result != null) { + this.split2MaxOffsets.putAll(result); + } + } + } + + @Override public void addNewSplit(IMessage message, AbstractContext context, NewSplitMessage newSplitMessage) { + this.currentQueueIds = newSplitMessage.getCurrentSplitIds(); + loadSplitProgress(newSplitMessage); + + List allWindowInstances = WindowInstance.queryAllWindowInstance(DateUtil.getCurrentTimeString(), window, newSplitMessage.getSplitIds()); + if (CollectionUtil.isNotEmpty(allWindowInstances)) { + for (WindowInstance windowInstance : allWindowInstances) { + windowInstance.setNewWindowInstance(false); + window.registerWindowInstance(windowInstance); + windowTrigger.registFireWindowInstanceIfNotExist(windowInstance, window); + String queueId = windowInstance.getSplitId(); + window.getStorage().loadSplitData2Local(queueId, windowInstance.createWindowInstanceId(), window.getWindowBaseValueClass(), new WindowOperator.WindowRowOperator(windowInstance, queueId, window)); + window.initWindowInstanceMaxSplitNum(windowInstance); + } + + } else { + for (String queueId : newSplitMessage.getSplitIds()) { + ShufflePartitionManager.getInstance().setSplitFinished(queueId); + } + } + getReceiverAfterCurrentNode().doMessage(message, context); + } + + @Override public void removeSplit(IMessage message, AbstractContext context, RemoveSplitMessage removeSplitMessage) { + this.currentQueueIds = removeSplitMessage.getCurrentSplitIds(); + Set queueIds = removeSplitMessage.getSplitIds(); + if (queueIds != null) { + for (String queueId : queueIds) { + ShufflePartitionManager.getInstance().setSplitInValidate(queueId); + window.clearCache(queueId); + } + window.getWindowMaxValueManager().removeKeyPrefixFromLocalCache(queueIds); + //window.getWindowFireSource().removeSplit(queueIds); + } + getReceiverAfterCurrentNode().doMessage(message, context); + } + + @Override public void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage) { + if (message.getHeader().isNeedFlush()) { + shuffleCache.flush(message.getHeader().getCheckpointQueueIds()); + window.getSqlCache().flush(message.getHeader().getCheckpointQueueIds()); + } + CheckPointState checkPointState = new CheckPointState(); + checkPointState.setQueueIdAndOffset(this.shuffleCache.getFinishedQueueIdAndOffsets(checkPointMessage)); + checkPointMessage.reply(checkPointState); + } + + /** + * if the message offset is old filter the repeate message + * + * @param message + * @param queueId + * @return + */ + protected boolean isRepeatMessage(IMessage message, String queueId) { + boolean isOrigOffsetLong = message.getMessageBody().getBoolean(ORIGIN_QUEUE_IS_LONG); + String oriQueueId = message.getMessageBody().getString(ORIGIN_QUEUE_ID); + String oriOffset = message.getMessageBody().getString(ORIGIN_OFFSET); + String key = MapKeyUtil.createKey(window.getName(), queueId, oriQueueId); + String offset = this.split2MaxOffsets.get(key); + if (offset != null) { + MessageOffset messageOffset = new MessageOffset(oriOffset, isOrigOffsetLong); + if (!messageOffset.greaterThan(offset)) { + LOGGER.debug("[{}] the message offset is old, the message is discard ", this.getName()); + return true; + } + } + return false; + } + + @Override public void batchMessageFinish(IMessage message, AbstractContext context, BatchFinishMessage batchFinishMessage) { + if (window.supportBatchMsgFinish()) { + // System.out.println("start fire window by fininsh flag "); + long startTime = System.currentTimeMillis(); + Set splitIds = new HashSet<>(); + splitIds.add(message.getHeader().getQueueId()); + if (shuffleCache != null) { + shuffleCache.flush(splitIds); + } + window.getSqlCache().flush(splitIds); + windowTrigger.fireWindowInstance(message.getHeader().getQueueId()); + IMessage cpMsg = batchFinishMessage.getMsg().copy(); + getReceiverAfterCurrentNode().doMessage(cpMsg, context); + LOGGER.debug("[{}] batch message finish cost is {}", this.getName(), (System.currentTimeMillis() - startTime)); + } + + } + + /** + * 过滤掉不是这个window的消息 + * + * @param oriMessage + * @return + */ + protected boolean filterNotOwnerMessage(IMessage oriMessage) { + String owner = oriMessage.getMessageBody().getString(MSG_OWNER); + return owner == null || !owner.equals(getDynamicPropertyValue()); + } + + /** + * 每次checkpoint的回调函数,默认是空实现,子类可以扩展实现 + * + * @param oriMessage + * @param context + */ + protected void afterFlushCallback(IMessage oriMessage, AbstractContext context) { + } + + @Override protected T handleMessage(T t, AbstractContext context) { + return null; + } + + @Override public boolean isAsyncNode() { + return false; + } + + protected String getDynamicPropertyValue() { + String dynamicPropertyValue = MapKeyUtil.createKey(window.getNameSpace(), window.getName(), window.getUpdateFlag() + ""); + dynamicPropertyValue = dynamicPropertyValue.replaceAll("\\.", "_").replaceAll(";", "_"); + return dynamicPropertyValue; + } + + public void setShuffleSource(ISource source) { + this.shuffleSource = source; + } + + public void setWindow(AbstractShuffleWindow window) { + this.window = window; + } + + public Set getCurrentQueueIds() { + return currentQueueIds; + } + + public void setCurrentQueueIds(Set currentQueueIds) { + this.currentQueueIds = currentQueueIds; + } +} diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/WindowConstants.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/WindowConstants.java new file mode 100644 index 00000000..907c03e7 --- /dev/null +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/WindowConstants.java @@ -0,0 +1,45 @@ +/* + * 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.window; + +public class WindowConstants { + + public static final String SHUFFLE_OFFSET = "SHUFFLE_OFFSET"; + public static final String SPLIT_SIGN = "##"; + public static final String IS_COMPRESSION_MSG = "_is_compress_msg"; + public static final String COMPRESSION_MSG_DATA = "_compress_msg"; + public static final String MSG_FROM_SOURCE = "msg_from_source"; + public static final String ORIGIN_OFFSET = "origin_offset"; + public static final String ORIGIN_QUEUE_ID = "origin_queue_id"; + public static final String ORIGIN_QUEUE_IS_LONG = "origin_offset_is_LONG"; + public static final String ORIGIN_MESSAGE_HEADER = "origin_message_header"; + public static final String ORIGIN_SOURCE_NAME = "origin_offset_name"; + public static final String SHUFFLE_KEY = "SHUFFLE_KEY"; + public static final String ORIGIN_MESSAGE_TRACE_ID = "origin_request_id"; + public static final int DEFAULT_WINDOW_SIZE = 10; + public static final int DEFAULT_WINDOW_SESSION_TIMEOUT = 10; + public static final String IS_SYSTEME_MSG = "__is_system_msg"; + public static final String SYSTEME_MSG = "__system_msg"; + public static final String SYSTEME_MSG_CLASS = "__system_msg_class"; + protected static final String SHUFFLE_QUEUE_ID = "SHUFFLE_QUEUE_ID"; + protected static final String SHUFFLE_MESSAGES = "SHUFFLE_MESSAGES"; + private static final String SHUFFLE_TRACE_ID = "SHUFFLE_TRACE_ID"; + /** + * 消息所属的window + */ + protected String MSG_OWNER = "MSG_OWNER"; +} diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/builder/WindowBuilder.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/builder/WindowBuilder.java index abe4847d..14a3fc51 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/builder/WindowBuilder.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/builder/WindowBuilder.java @@ -16,9 +16,9 @@ */ package org.apache.rocketmq.streams.window.builder; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; -import org.apache.rocketmq.streams.common.topology.model.IWindow; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.configuration.SystemContext; +import org.apache.rocketmq.streams.common.topology.IWindow; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.window.operator.AbstractWindow; import org.apache.rocketmq.streams.window.operator.impl.OverWindow; @@ -62,9 +62,9 @@ public static JoinWindow createDefaultJoinWindow() { if (TEST_MODE) { joinWindow = new JoinWindow(); } - joinWindow.setSizeInterval(getIntValue(ConfigureFileKey.DIPPER_WINDOW_JOIN_DEFAULT_ITERVA_SIZE, 5));//默认5分钟一个窗口 - joinWindow.setSlideInterval(getIntValue(ConfigureFileKey.DIPPER_WINDOW_JOIN_DEFAULT_ITERVA_SIZE, 5)); - joinWindow.setRetainWindowCount(getIntValue(ConfigureFileKey.DIPPER_WINDOW_JOIN_RETAIN_WINDOW_COUNT, 6));//join的时间窗口是20分钟 + joinWindow.setSizeInterval(getIntValue(ConfigurationKey.DIPPER_WINDOW_JOIN_DEFAULT_INTERVAL_SIZE, 5));//默认5分钟一个窗口 + joinWindow.setSlideInterval(getIntValue(ConfigurationKey.DIPPER_WINDOW_JOIN_DEFAULT_INTERVAL_SIZE, 5)); + joinWindow.setRetainWindowCount(getIntValue(ConfigurationKey.DIPPER_WINDOW_JOIN_RETAIN_WINDOW_COUNT, 6));//join的时间窗口是20分钟 joinWindow.setWindowType(AbstractWindow.TUMBLE_WINDOW); // joinWindow.setFireDelaySecond(getIntValue(ConfigureFileKey.DIPPER_WINDOW_DEFAULT_FIRE_DELAY_SECOND,5));//延迟5分钟触发 joinWindow.setTimeFieldName(""); @@ -77,7 +77,7 @@ public static OverWindow createOvertWindow(String groupBy, String rowNumName) { overWindow.setGroupByFieldName(groupBy); overWindow.setRowNumerName(rowNumName); overWindow.setTimeFieldName(""); - overWindow.setSizeInterval(getIntValue(ConfigureFileKey.DIPPER_WINDOW_OVER_DEFAULT_ITERVA_SIZE, 60)); + overWindow.setSizeInterval(getIntValue(ConfigurationKey.DIPPER_WINDOW_OVER_DEFAULT_INTERVAL_SIZE, 60)); overWindow.setSlideInterval(overWindow.getSizeInterval()); return overWindow; } @@ -90,7 +90,7 @@ public static OverWindow createOvertWindow(String groupBy, String rowNumName) { * @return */ public static int getIntValue(String propertyKey, int defalutValue) { - String value = ComponentCreator.getProperties().getProperty(propertyKey); + String value = SystemContext.getProperty(propertyKey); if (StringUtil.isNotEmpty(value)) { return Integer.valueOf(value); } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/debug/DebugAnalysis.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/debug/DebugAnalysis.java index 6a01c886..b6fd32b9 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/debug/DebugAnalysis.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/debug/DebugAnalysis.java @@ -35,10 +35,10 @@ import static junit.framework.TestCase.assertTrue; public class DebugAnalysis { - private String dir; protected String sumFieldName; protected int expectValue; protected String timeFieldName; + private String dir; public DebugAnalysis(String dir, String sumFieldName, int expectValue, String timeFieldName) { this.dir = dir; diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/debug/DebugWriter.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/debug/DebugWriter.java index eea8c23c..54798d48 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/debug/DebugWriter.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/debug/DebugWriter.java @@ -24,19 +24,19 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.configuration.SystemContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.utils.DateUtil; import org.apache.rocketmq.streams.common.utils.FileUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; -import org.apache.rocketmq.streams.window.model.WindowCache; +import org.apache.rocketmq.streams.window.WindowConstants; import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.operator.AbstractWindow; import org.apache.rocketmq.streams.window.state.impl.WindowValue; public class DebugWriter { - protected String filePath = "/tmp/rocketmq-streams/window_debug"; protected static Map debugWriterMap = new HashMap<>(); + protected String filePath = "/tmp/rocketmq-streams/window_debug"; protected boolean openDebug = false; protected String countFileName; protected boolean openRocksDBTest = false; @@ -45,19 +45,19 @@ public DebugWriter(String windowName) { filePath = filePath + "/" + windowName; File file = new File(filePath); file.deleteOnExit(); - String value = ComponentCreator.getProperties().getProperty("window.debug"); + String value = SystemContext.getProperty("window.debug"); if (StringUtil.isNotEmpty(value)) { openDebug = Boolean.valueOf(value); } - value = ComponentCreator.getProperties().getProperty("window.debug.countFileName"); + value = SystemContext.getProperty("window.debug.countFileName"); if (StringUtil.isNotEmpty(value)) { countFileName = value; } - value = ComponentCreator.getProperties().getProperty("window.debug.dir"); + value = SystemContext.getProperty("window.debug.dir"); if (StringUtil.isNotEmpty(value)) { filePath = value; } - value = ComponentCreator.getProperties().getProperty("window.debug.rocksdb"); + value = SystemContext.getProperty("window.debug.rocksdb"); if (StringUtil.isNotEmpty(value)) { openRocksDBTest = Boolean.valueOf(value); } @@ -163,10 +163,10 @@ public synchronized void writeShuffleReceive(AbstractWindow window, List eventTimeManagerMap = new HashMap<>(); protected ISource source; - + /** + * key:shuffle 数据源的分片id + * value:管理数据源分片的时间 + */ + private Map eventTimeManagerMap = new HashMap<>(); private Map> eventTimeIncreasementMap = new ConcurrentHashMap<>(); public void updateEventTime(IMessage message, AbstractWindow window) { diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/fire/SplitEventTimeManager.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/fire/SplitEventTimeManager.java index b4daf916..20ecf481 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/fire/SplitEventTimeManager.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/fire/SplitEventTimeManager.java @@ -22,49 +22,49 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; 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.context.IMessage; import org.apache.rocketmq.streams.common.utils.StringUtil; -import org.apache.rocketmq.streams.window.model.WindowCache; +import org.apache.rocketmq.streams.window.WindowConstants; import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.operator.AbstractWindow; +/** + * 对齐等待,当所有的数据源分片的数据都收到,才会返回事件时间 + */ public class SplitEventTimeManager { - protected static final Log LOG = LogFactory.getLog(SplitEventTimeManager.class); - protected static Map messageSplitId2MaxTime = new HashMap<>(); - private AtomicInteger queueIdCount = new AtomicInteger(0); + private static Long splitReadyTime; + private final AtomicInteger queueIdCount = new AtomicInteger(0); + protected Map messageSplitId2MaxTime = new HashMap<>(); protected Long lastUpdateTime; - protected volatile Integer allSplitSize; - protected volatile Integer workingSplitSize; - protected Map> splitsGroupByInstance; - protected ISource source; - + protected Map>> splitsGroupByInstance; + protected ISource source; protected volatile boolean isAllSplitReceived = false; protected transient String queueId; - private static Long splitReadyTime; - - public SplitEventTimeManager(ISource source, String queueId) { + public SplitEventTimeManager(ISource source, String queueId) { this.source = source; this.queueId = queueId; - if (source instanceof AbstractSource) { - AbstractSource abstractSource = (AbstractSource) source; - List splits = abstractSource.getAllSplits(); - if (splits == null) { - this.allSplitSize = -1; - } else { - this.allSplitSize = splits.size(); - } + AbstractSource abstractSource = (AbstractSource) source; + List> splits = abstractSource.fetchAllSplits(); + if (splits == null) { + this.allSplitSize = -1; + } else { + this.allSplitSize = splits.size(); } } + /** + * 更新数据源分片的事件时间,取当前收到消息的最大值 + * + * @param message + * @param window + */ public void updateEventTime(IMessage message, AbstractWindow window) { - String oriQueueId = message.getMessageBody().getString(WindowCache.ORIGIN_QUEUE_ID); + String oriQueueId = message.getMessageBody().getString(WindowConstants.ORIGIN_QUEUE_ID); if (StringUtil.isEmpty(oriQueueId)) { return; } @@ -80,6 +80,17 @@ public void updateEventTime(IMessage message, AbstractWindow window) { } } + /** + * 获取事件时间,对齐等待,期望获取所有分片的数据后开始,如果一直没有数据,则等1分钟后开始 + * 对齐等待:先判断是否所有的数据源分片的数据都收到过,这块是通过数据源接口获取所有分片,然后根据收到信息的分片做对比,这种实现有两个风险: + * 1.数据分片不均匀,会导致某些分片的数据一直收不到,导致窗口一直未能触发,尤其是数据在分片中按shuffle key存储 + * 2.要求所有的数据源必须提供获取所有分片的接口,不友好 + * 最新实现: + * 1.所有数据源启动后或在窗口shuffle前,收到第一条消息后,发一个系统消息 + * 2.通过系统消息,确保分片已经正常工作,且能和数据源的时间快速对齐 + * + * @return + */ public Long getMaxEventTime() { if (!isSplitsReceiver()) { @@ -110,29 +121,26 @@ protected boolean isSplitsReceiver() { if (lastUpdateTime == null) { lastUpdateTime = System.currentTimeMillis(); } - if (allSplitSize == null && workingSplitSize == null) { + int workingSplitSize = this.messageSplitId2MaxTime.size(); + if (allSplitSize == null) { if (source == null) { return false; } - if (source instanceof AbstractSource) { - AbstractSource abstractSource = (AbstractSource) source; - List splits = abstractSource.getAllSplits(); - if (splits == null) { - this.allSplitSize = -1; - } else { - this.allSplitSize = splits.size(); - } + AbstractSource abstractSource = (AbstractSource) source; + List> splits = abstractSource.fetchAllSplits(); + if (splits == null) { + this.allSplitSize = -1; + } else { + this.allSplitSize = splits.size(); } } if (allSplitSize == -1) { return true; } - if (allSplitSize != -1 && workingSplitSize == null) { - workingSplitSize = 0; - } + if (allSplitSize != -1 && allSplitSize > workingSplitSize) { if (System.currentTimeMillis() - lastUpdateTime > 1000) { - workingSplitSize = calcuteWorkingSplitSize(); + workingSplitSize = this.messageSplitId2MaxTime.size(); lastUpdateTime = System.currentTimeMillis(); if (allSplitSize > workingSplitSize) { return false; @@ -162,25 +170,8 @@ protected boolean isSplitsReceiver() { return false; } - private Integer calcuteWorkingSplitSize() { - if (source instanceof AbstractSource) { - AbstractSource abstractSource = (AbstractSource) source; - Map> splits = abstractSource.getWorkingSplitsGroupByInstances(); - if (splits == null) { - return 0; - } - this.splitsGroupByInstance = splits; - int count = 0; - for (List splitList : splits.values()) { - count += splitList.size(); - } - return count; - } - return 0; - - } - public void setSource(ISource source) { this.source = source; } + } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/minibatch/MiniBatchMsgCache.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/minibatch/MiniBatchMsgCache.java index 39629cc1..16735cc7 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/minibatch/MiniBatchMsgCache.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/minibatch/MiniBatchMsgCache.java @@ -16,59 +16,35 @@ */ package org.apache.rocketmq.streams.window.minibatch; -import com.alibaba.fastjson.JSONArray; -import com.alibaba.fastjson.JSONObject; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; import org.apache.commons.lang3.tuple.Pair; import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageFlushCallBack; import org.apache.rocketmq.streams.common.channel.sinkcache.impl.AbstractMultiSplitMessageCache; import org.apache.rocketmq.streams.common.channel.sinkcache.impl.MessageCache; import org.apache.rocketmq.streams.common.channel.split.ISplit; 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.topology.shuffle.IShuffleKeyGenerator; -import org.apache.rocketmq.streams.common.utils.MapKeyUtil; -import org.apache.rocketmq.streams.common.utils.ReflectUtil; -import org.apache.rocketmq.streams.common.utils.StringUtil; -import org.apache.rocketmq.streams.script.operator.impl.AggregationScript; -import org.apache.rocketmq.streams.window.model.WindowInstance; +import org.apache.rocketmq.streams.common.topology.IShuffleKeyGenerator; import org.apache.rocketmq.streams.window.operator.AbstractShuffleWindow; -import org.apache.rocketmq.streams.window.operator.AbstractWindow; -import org.apache.rocketmq.streams.window.state.impl.WindowValue; -import org.apache.rocketmq.streams.window.util.ShuffleUtil; - -public class MiniBatchMsgCache extends AbstractMultiSplitMessageCache> { - public static String SHUFFLE_KEY="shuffle_key"; - +public class MiniBatchMsgCache extends AbstractMultiSplitMessageCache, IMessage>> { + public static String SHUFFLE_KEY = "shuffle_key"; protected transient IShuffleKeyGenerator shuffleKeyGenerator; protected transient AbstractShuffleWindow window; - - - public MiniBatchMsgCache( - IMessageFlushCallBack> flushCallBack, IShuffleKeyGenerator shuffleKeyGenerator, + IMessageFlushCallBack, IMessage>> flushCallBack, IShuffleKeyGenerator shuffleKeyGenerator, AbstractShuffleWindow window) { super(flushCallBack); - this.shuffleKeyGenerator=shuffleKeyGenerator; - this.window=window; + this.shuffleKeyGenerator = shuffleKeyGenerator; + this.window = window; } - - @Override protected String createSplitId(Pair msg) { + @Override protected String createSplitId(Pair, IMessage> msg) { return msg.getLeft().getQueueId(); } @Override protected MessageCache createMessageCache() { - ShuffleMessageCache messageCache=new ShuffleMessageCache(this.flushCallBack); + ShuffleMessageCache messageCache = new ShuffleMessageCache(this.flushCallBack); messageCache.setWindow(window); messageCache.setShuffleKeyGenerator(shuffleKeyGenerator); return messageCache; diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/minibatch/ShuffleMessageCache.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/minibatch/ShuffleMessageCache.java index aa394291..d4374bb2 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/minibatch/ShuffleMessageCache.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/minibatch/ShuffleMessageCache.java @@ -27,15 +27,15 @@ import org.apache.rocketmq.streams.common.channel.sinkcache.impl.MessageCache; import org.apache.rocketmq.streams.common.channel.split.ISplit; import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; 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.topology.shuffle.IShuffleKeyGenerator; +import org.apache.rocketmq.streams.common.topology.IShuffleKeyGenerator; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.script.operator.impl.AggregationScript; -import org.apache.rocketmq.streams.window.model.WindowCache; +import org.apache.rocketmq.streams.window.WindowConstants; import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.operator.AbstractShuffleWindow; import org.apache.rocketmq.streams.window.operator.AbstractWindow; @@ -43,81 +43,47 @@ import org.apache.rocketmq.streams.window.state.impl.WindowValue; import org.apache.rocketmq.streams.window.util.ShuffleUtil; -public class ShuffleMessageCache extends MessageCache> { - - protected Map groupBy2WindowValue=new HashMap<>(); +public class ShuffleMessageCache extends MessageCache, IMessage>> { + protected Map groupBy2WindowValue = new HashMap<>(); protected transient IShuffleKeyGenerator shuffleKeyGenerator; protected transient AbstractShuffleWindow window; + protected transient AtomicLong SUM = new AtomicLong(0); + public ShuffleMessageCache( - IMessageFlushCallBack> flushCallBack) { + IMessageFlushCallBack, IMessage>> flushCallBack) { super(flushCallBack); } - protected class MiniBatch{ - protected WindowValue windowValue; - protected IMessage message; - - public MiniBatch(){ - windowValue=new WindowValue(); - } - public IMessage calculate(AbstractWindow window, IMessage msg,String groupByValue) { - windowValue.calculate(window,msg); - JSONObject mergeMsg=createMsg(groupByValue,windowValue,msg.getHeader(),msg.getMessageBody().getJSONObject(WindowCache.ORIGIN_MESSAGE_HEADER)); - if(window.getTimeFieldName()!=null){ - mergeMsg.put(window.getTimeFieldName(),msg.getMessageBody().getString(window.getTimeFieldName())); - } - if(msg.getMessageBody().get(WindowInstance.class.getSimpleName())!=null){ - mergeMsg.put(WindowInstance.class.getSimpleName(),msg.getMessageBody().get(WindowInstance.class.getSimpleName())); - } - if(msg.getMessageBody().get(AbstractWindow.class.getSimpleName())!=null){ - mergeMsg.put(AbstractWindow.class.getSimpleName(),msg.getMessageBody().get(AbstractWindow.class.getSimpleName())); - } - - - - if(this.message==null){ - this.message=new Message(mergeMsg); - return message; - }else { - this.message.getMessageBody().putAll(mergeMsg); - } - - - return null; - } - - } - - @Override public synchronized int addCache(Pair pair) { - boolean openMiniBatch=isOpenMiniBatch(); - ISplit split=pair.getLeft(); - IMessage message=pair.getRight(); - if(openMiniBatch){ + @Override public synchronized int addCache(Pair, IMessage> pair) { + boolean openMiniBatch = isOpenMiniBatch(); + ISplit split = pair.getLeft(); + IMessage message = pair.getRight(); + if (openMiniBatch) { String groupByValue = shuffleKeyGenerator.generateShuffleKey(message); if (StringUtil.isEmpty(groupByValue)) { groupByValue = ""; } List windowInstances = (List) message.getMessageBody().get(WindowInstance.class.getSimpleName()); - if(windowInstances==null){ - windowInstances=this.window.queryOrCreateWindowInstanceOnly(message,split.getQueueId()); + if (windowInstances == null) { + windowInstances = this.window.queryOrCreateWindowInstanceOnly(message, split.getQueueId()); } - for(WindowInstance windowInstance:windowInstances){ - String key= MapKeyUtil.createKey(windowInstance.createWindowInstanceId(),groupByValue); + for (WindowInstance windowInstance : windowInstances) { + String key = MapKeyUtil.createKey(windowInstance.createWindowInstanceId(), groupByValue); MiniBatch miniBatch = groupBy2WindowValue.get(key); - if(miniBatch==null) { + if (miniBatch == null) { miniBatch = new MiniBatch(); groupBy2WindowValue.put(key, miniBatch); } - IMessage newMergeMessage=miniBatch.calculate(this.window,message,groupByValue); - if(newMergeMessage!=null){ + IMessage newMergeMessage = miniBatch.calculate(this.window, message, groupByValue); + if (newMergeMessage != null) { pair.setValue(newMergeMessage); return super.addCache(pair); } } - }else { + } else { return super.addCache(pair); } @@ -125,30 +91,25 @@ public IMessage calculate(AbstractWindow window, IMessage msg,String groupByValu } protected boolean isOpenMiniBatch() { - if(!WindowOperator.class.isInstance(window)){ + if (!(window instanceof WindowOperator)) { return false; } - if(window.getGroupByFieldName()==null&&WindowOperator.class.getSimpleName().equals(window.getClass().getSimpleName())){ - return true; + if (window.isEmptyGroupBy() && WindowOperator.class.getSimpleName().equals(window.getClass().getSimpleName())) { + return false; } - boolean isOpenMiniBatch= ComponentCreator.getPropertyBooleanValue(ConfigureFileKey.WINDOW_MINIBATCH_SWITCH); - return isOpenMiniBatch; + return ComponentCreator.getPropertyBooleanValue(ConfigurationKey.WINDOW_MINI_BATCH_SWITCH); } - protected transient AtomicLong SUM=new AtomicLong(0); - protected JSONObject createMsg(String shuffleKey,WindowValue windowValue, MessageHeader messageHeader,JSONObject msgHeader) { + protected JSONObject createMsg(String shuffleKey, WindowValue windowValue, MessageHeader messageHeader, JSONObject msgHeader) { JSONObject message = new JSONObject(); - long start=System.currentTimeMillis(); message.put(WindowValue.class.getName(), windowValue); - // long sum=SUM.addAndGet(System.currentTimeMillis()-start); - // System.out.println("create msg "+sum); - message.put(AggregationScript.INNER_AGGREGATION_COMPUTE_KEY,AggregationScript.INNER_AGGREGATION_COMPUTE_MULTI); - IMessage windowValueMsg=new Message(message); + message.put(AggregationScript.INNER_AGGREGATION_COMPUTE_KEY, AggregationScript.INNER_AGGREGATION_COMPUTE_MULTI); + IMessage windowValueMsg = new Message(message); windowValueMsg.setHeader(messageHeader); - ShuffleUtil.createShuffleMsg(windowValueMsg,shuffleKey,msgHeader); + ShuffleUtil.createShuffleMsg(windowValueMsg, shuffleKey, msgHeader); - if (JSONObject.class.isInstance(windowValue.getcomputedResult())) { + if (windowValue.getcomputedResult() instanceof JSONObject) { message.putAll(windowValue.getcomputedResult()); } else { Iterator> it = windowValue.iteratorComputedColumnResult(); @@ -162,9 +123,8 @@ protected JSONObject createMsg(String shuffleKey,WindowValue windowValue, Messag } - @Override public synchronized int flush() { - this.groupBy2WindowValue=new HashMap<>(); + this.groupBy2WindowValue = new HashMap<>(); return super.flush(); } @@ -184,4 +144,37 @@ public AbstractShuffleWindow getWindow() { public void setWindow(AbstractShuffleWindow window) { this.window = window; } + + protected class MiniBatch { + protected WindowValue windowValue; + protected IMessage message; + + public MiniBatch() { + windowValue = new WindowValue(); + } + + public IMessage calculate(AbstractWindow window, IMessage msg, String groupByValue) { + windowValue.calculate(window, msg); + JSONObject mergeMsg = createMsg(groupByValue, windowValue, msg.getHeader(), msg.getMessageBody().getJSONObject(WindowConstants.ORIGIN_MESSAGE_HEADER)); + if (window.getTimeFieldName() != null) { + mergeMsg.put(window.getTimeFieldName(), msg.getMessageBody().getString(window.getTimeFieldName())); + } + if (msg.getMessageBody().get(WindowInstance.class.getSimpleName()) != null) { + mergeMsg.put(WindowInstance.class.getSimpleName(), msg.getMessageBody().get(WindowInstance.class.getSimpleName())); + } + if (msg.getMessageBody().get(AbstractWindow.class.getSimpleName()) != null) { + mergeMsg.put(AbstractWindow.class.getSimpleName(), msg.getMessageBody().get(AbstractWindow.class.getSimpleName())); + } + + if (this.message == null) { + this.message = new Message(mergeMsg); + return message; + } else { + this.message.getMessageBody().putAll(mergeMsg); + } + + return null; + } + + } } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowCache.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowCache.java deleted file mode 100644 index 28ed5da0..00000000 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowCache.java +++ /dev/null @@ -1,204 +0,0 @@ -/* - * 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.window.model; - -import com.alibaba.fastjson.JSONArray; -import com.alibaba.fastjson.JSONObject; - -import java.util.List; -import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.commons.lang3.tuple.MutablePair; -import org.apache.commons.lang3.tuple.Pair; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; -import org.apache.rocketmq.streams.common.channel.sink.AbstractSink; -import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageFlushCallBack; -import org.apache.rocketmq.streams.common.channel.split.ISplit; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.context.Message; -import org.apache.rocketmq.streams.common.topology.model.IWindow; -import org.apache.rocketmq.streams.common.topology.shuffle.IShuffleKeyGenerator; -import org.apache.rocketmq.streams.common.utils.CompressUtil; -import org.apache.rocketmq.streams.window.minibatch.MiniBatchMsgCache; -import org.apache.rocketmq.streams.window.shuffle.ShuffleChannel; -import org.apache.rocketmq.streams.window.util.ShuffleUtil; - -/** - * 缓存数据,flush时,刷新完成数据落盘 - */ -public abstract class WindowCache extends - AbstractSink implements IWindow.IWindowCheckpoint { - - private static final Log LOG = LogFactory.getLog(WindowCache.class); - - public static final String SPLIT_SIGN = "##"; - - public static final String IS_COMPRESSION_MSG = "_is_compress_msg"; - public static final String COMPRESSION_MSG_DATA = "_compress_msg"; - public static final String MSG_FROM_SOURCE = "msg_from_source"; - public static final String ORIGIN_OFFSET = "origin_offset"; - - public static final String ORIGIN_QUEUE_ID = "origin_queue_id"; - - public static final String ORIGIN_QUEUE_IS_LONG = "origin_offset_is_LONG"; - - public static final String ORIGIN_MESSAGE_HEADER = "origin_message_header"; - - public static final String ORIGIN_SOURCE_NAME = "origin_offset_name"; - - public static final String SHUFFLE_KEY = "SHUFFLE_KEY"; - - public static final String ORIGIN_MESSAGE_TRACE_ID = "origin_request_id"; - protected transient boolean isWindowTest = false; - protected transient AtomicLong COUNT = new AtomicLong(0); - /** - * 分片转发channel - */ - protected transient ShuffleChannel shuffleChannel; - - public void initMiniBatch() { - shuffleMsgCache = new MiniBatchMsgCache(new WindowCache.MutilMsgMergerAndCompressFlushCallBack(),(IShuffleKeyGenerator) shuffleChannel.getWindow(),shuffleChannel.getWindow()); - - - shuffleMsgCache.openAutoFlush(); - } - -// protected transient AtomicLong insertCount=new AtomicLong(0); -// protected transient AtomicLong shuffleCount=new AtomicLong(0); -// protected transient AtomicLong SUM=new AtomicLong(0); - - protected class MutilMsgMergerAndCompressFlushCallBack implements IMessageFlushCallBack> { - - @Override - public boolean flushMessage(List> messages) { - if (messages == null || messages.size() == 0) { - return true; - } - long start=System.currentTimeMillis(); - ISplit split = messages.get(0).getLeft(); - JSONArray allMsgs =new JSONArray(); - long sum=0; - for (int i = 0; i < messages.size(); i++) { - Pair pair = messages.get(i); - IMessage message = pair.getRight(); - allMsgs.add(message.getMessageBody()); - // sum=SUM.addAndGet(message.getMessageBody().getLong("total")); - } - //System.out.println("before shuffle sum is "+sum); - JSONObject jsonObject=shuffleChannel.createMsg(allMsgs,split); -// JSONObject zipJsonObject = new JSONObject(); -// zipJsonObject.put(COMPRESSION_MSG_DATA, CompressUtil.gZip(jsonObject.toJSONString())); - shuffleChannel.getProducer().batchAdd(new Message(jsonObject), split); - shuffleChannel.getProducer().flush(split.getQueueId()); - long cost=System.currentTimeMillis()-start; - // shuffleCount.addAndGet(cost); - return true; - } - } - - - protected transient MiniBatchMsgCache shuffleMsgCache ; - - @Override - protected boolean initConfigurable() { - - isWindowTest = ComponentCreator.getPropertyBooleanValue("window.fire.isTest"); - return super.initConfigurable(); - } - - @Override - protected boolean batchInsert(List messageList) { - long start=System.currentTimeMillis(); - for (IMessage msg : messageList) { - String shuffleKey = generateShuffleKey(msg); - IMessage message= ShuffleUtil.createShuffleMsg(msg,shuffleKey); - if(message==null){ - continue; - } - addPropertyToMessage(msg, message.getMessageBody()); - Integer index = shuffleChannel.hash(shuffleKey); - ISplit split = shuffleChannel.getSplit(index); - shuffleMsgCache.addCache(new MutablePair(split, message)); - } - if (isWindowTest) { - long count = COUNT.addAndGet(messageList.size()); - System.out.println(shuffleChannel.getWindow().getConfigureName() + " send shuffle msg count is " + count); - shuffleMsgCache.flush(); - } - long cost=System.currentTimeMillis()-start; - //shuffleCount.addAndGet(cost); - return true; - } - - @Override - public void finishBatchMsg(BatchFinishMessage batchFinishMessage) { - long start=System.currentTimeMillis(); - if (shuffleChannel != null && shuffleChannel.getProducer() != null) { - this.flush(); - shuffleMsgCache.flush(); - for (ISplit split : shuffleChannel.getQueueList()) { - IMessage message = batchFinishMessage.getMsg().deepCopy(); - message.getMessageBody().put(ORIGIN_QUEUE_ID, message.getHeader().getQueueId()); - shuffleChannel.getProducer().batchAdd(message, split); - } - shuffleChannel.getProducer().flush(); - } - // System.out.println("insert cost is "+insertCount.get()+" shuffle cost is "+shuffleCount.get()+" finish batch cost is "+(System.currentTimeMillis()-start)); - - } - - - - /** - * 根据message生成shuffle key - * - * @param message - * @return - */ - protected abstract String generateShuffleKey(IMessage message); - - @Override - public boolean checkpoint(Set queueIds) { - this.flush(queueIds); - this.shuffleMsgCache.flush(queueIds); - return true; - } - - /** - * 如果需要额外的字段附加到shuffle前的message,通过实现这个子类增加 - * - * @param oriJson - */ - protected void addPropertyToMessage(IMessage oriMessage, JSONObject oriJson) { - - } - - public ShuffleChannel getShuffleChannel() { - return shuffleChannel; - } - - public MiniBatchMsgCache getShuffleMsgCache() { - return this.shuffleMsgCache; - } - - public void setShuffleChannel(ShuffleChannel shuffleChannel) { - this.shuffleChannel = shuffleChannel; - } -} diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowInstance.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowInstance.java index fed18863..4e2a98cb 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowInstance.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowInstance.java @@ -25,12 +25,13 @@ import java.util.List; import java.util.concurrent.TimeUnit; import org.apache.commons.lang3.tuple.Pair; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.model.Entity; +import org.apache.rocketmq.streams.common.model.NameCreator; +import org.apache.rocketmq.streams.common.topology.IWindow; import org.apache.rocketmq.streams.common.utils.CollectionUtil; import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.common.utils.IdUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.common.utils.SQLUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; @@ -38,36 +39,35 @@ import org.apache.rocketmq.streams.window.operator.AbstractWindow; import org.apache.rocketmq.streams.window.sqlcache.SQLCache; import org.apache.rocketmq.streams.window.sqlcache.impl.SQLElement; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * 具体的窗口实例 */ public class WindowInstance extends Entity implements Serializable { - protected static final Log LOG = LogFactory.getLog(WindowInstance.class); + private static final Logger LOGGER = LoggerFactory.getLogger(WindowInstance.class); private static final long serialVersionUID = 6893491128670330569L; - + private static final String SEPARATOR = "_"; + private static SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); /** * 窗口实例的开始时间 */ protected String startTime; - /** * 窗口实例的结束时间 */ protected String endTime; - /** * fire! */ protected String fireTime; - /** * 使用configName */ protected String windowName; - protected String splitId; protected boolean canClearResource = true; /** @@ -75,7 +75,6 @@ public class WindowInstance extends Entity implements Serializable { */ protected String windowNameSpace; protected String windowInstanceName;//默认等于窗口名,需要区分不同窗口时使用 - /** * splitId,windowNameSpace,windowName,windowInstanceName,windowInstanceName 数据库中存储的是MD5值 */ @@ -84,61 +83,15 @@ public class WindowInstance extends Entity implements Serializable { * windowInstanceId, splitId,windowNameSpace,windowName,windowInstanceName,windowInstanceName,startTime,endTime" 数据库中存储的是MD5值 */ protected String windowInstanceKey; - protected transient Boolean isNewWindowInstance = false;//当第一次创建时设置为true,否则设置为false - /** * 0:待计算;1:已经计算结束;-1:已经取消; */ protected int status = 0; - //todo 建议之后改个名字,为了测试方便,暂时用这个字段 protected Integer version = 1;//用于标识channel的状态,如果值是1,表示第一次消费,否则是第二次消费 - - private static SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); - - private static final String SEPARATOR = "_"; - protected transient Long lastMaxUpdateTime;//last max update time for session window - public WindowInstance copy() { - WindowInstance windowInstance = new WindowInstance(); - windowInstance.setCanClearResource(this.canClearResource); - windowInstance.setFireTime(this.fireTime); - windowInstance.setNewWindowInstance(isNewWindowInstance); - windowInstance.setLastMaxUpdateTime(this.lastMaxUpdateTime); - windowInstance.setEndTime(this.endTime); - windowInstance.setStartTime(this.startTime); - windowInstance.setSplitId(this.splitId); - windowInstance.setWindowInstanceSplitName(this.windowInstanceSplitName); - windowInstance.setGmtModified(new Date()); - windowInstance.setGmtCreate(new Date()); - windowInstance.setWindowInstanceName(this.windowInstanceName); - windowInstance.setWindowInstanceKey(this.windowInstanceKey); - windowInstance.setWindowName(this.windowName); - windowInstance.setWindowNameSpace(this.windowNameSpace); - windowInstance.setStatus(this.status); - windowInstance.setVersion(this.version); - return windowInstance; - } - - /** - * 创建window instance的唯一ID - * - * @return - */ - public String createWindowInstanceId() { - return MapKeyUtil.createKey(splitId, windowNameSpace, windowName, windowInstanceName, startTime, endTime); - } - - public String createWindowInstanceIdWithoutSplitid() { - return MapKeyUtil.createKey(windowNameSpace, windowName, windowInstanceName, startTime, endTime); - } - - public String createWindowInstanceTriggerId() { - return MapKeyUtil.createKey(splitId, windowNameSpace, windowName, windowInstanceName, startTime, endTime, fireTime); - } - /** * 创建window instance对象列表 * @@ -158,10 +111,6 @@ public static List createWindowInstances(AbstractWindow window, return lostInstanceList; } - public String createWindowInstancePartitionId() { - return StringUtil.createMD5Str(MapKeyUtil.createKey(windowNameSpace, windowName, windowInstanceName, startTime, endTime, splitId)); - } - /** * 触发时间比lastTime小的所有的有效的instance * @@ -177,7 +126,7 @@ public static List queryAllWindowInstance(String lastTime, Abstr splitIdList.addAll(splitIds); String[] splitNames = new String[splitIds.size()]; for (int i = 0; i < splitNames.length; i++) { - splitNames[i] = MapKeyUtil.createKey(window.getNameSpace(), window.getConfigureName(), splitIdList.get(i)); + splitNames[i] = MapKeyUtil.createKey(window.getNameSpace(), window.getName(), splitIdList.get(i)); splitNames[i] = StringUtil.createMD5Str(splitNames[i]); } String sql = "select * from window_instance where " @@ -187,7 +136,7 @@ public static List queryAllWindowInstance(String lastTime, Abstr try { dbWindowInstanceList = ORMUtil.queryForList(sql, null, WindowInstance.class); } catch (Exception e) { - LOG.error("failed in getting unfired window instances", e); + LOGGER.error("failed in getting unfired window instances", e); } return dbWindowInstanceList; } @@ -246,6 +195,7 @@ public static Long getOccurTime(AbstractWindow window, IMessage message) { } return occurTime; } + /** * 查询或者创建Window的实例,滑动窗口有可能返回多个,滚动窗口返回一个 * @@ -256,8 +206,9 @@ public static Long getOccurTime(AbstractWindow window, IMessage message) { */ public static List getOrCreateWindowInstance(AbstractWindow window, Long occurTime, int timeUnitAdjust, String queueId) { - return getOrCreateWindowInstance(window,occurTime,timeUnitAdjust,queueId,false); + return getOrCreateWindowInstance(window, occurTime, timeUnitAdjust, queueId, false); } + /** * 查询或者创建Window的实例,滑动窗口有可能返回多个,滚动窗口返回一个 * @@ -316,8 +267,8 @@ public static List getOrCreateWindowInstance(AbstractWindow wind WindowInstance lastWindowInstance = window.createWindowInstance(DateUtil.format(begin), DateUtil.format(end), DateUtil.format(clearWindowInstanceFireTime), queueId); lastWindowInstance.setCanClearResource(true); window.registerWindowInstance(lastWindowInstance); - window.getSqlCache().addCache(new SQLElement(queueId, lastWindowInstance.createWindowInstanceId(), ORMUtil.createBatchReplacetSQL(lastWindowInstance))); - window.getWindowFireSource().registFireWindowInstanceIfNotExist(lastWindowInstance, window); + window.getSqlCache().addCache(new SQLElement(queueId, lastWindowInstance.createWindowInstanceId(), ORMUtil.createBatchReplaceSQL(lastWindowInstance))); + window.getWindowTrigger().registFireWindowInstanceIfNotExist(lastWindowInstance, window); } } else { @@ -325,8 +276,8 @@ public static List getOrCreateWindowInstance(AbstractWindow wind if (window.getEmitAfterValue() != null && window.getEmitAfterValue() > 0 && window.getMaxDelay() != null && window.getMaxDelay() > 0) { fire = DateUtil.addDate(TimeUnit.SECONDS, fire, window.getMaxDelay().intValue()); } - if (maxEventTime != null && maxEventTime - fire.getTime() > 0) { - LOG.warn("*********************the message is discard, because the fire time is exceed****************** " + DateUtil.format(begin) + "-" + DateUtil.format(end) + "---" + DateUtil.format(fire)); + if (maxEventTime != null && maxEventTime - fire.getTime() >= 3000) { + LOGGER.warn("[{}][{}] Window_Instance_Exceed_On(startTime:{}-endTime:{}-fireTime:{}-maxEventTime:{})--MsgTime:{}", IdUtil.instanceId(), NameCreator.getFirstPrefix(window.getName(), IWindow.TYPE), DateUtil.format(begin), DateUtil.format(end), DateUtil.format(fire), DateUtil.longToString(maxEventTime), DateUtil.longToString(occurTime)); break; } } @@ -347,14 +298,17 @@ public static List getOrCreateWindowInstance(AbstractWindow wind List lostInstanceList = null; lostInstanceList = WindowInstance.createWindowInstances(window, lostWindowTimeList, lostFireList, queueId); instanceList.addAll(lostInstanceList); - if (CollectionUtil.isNotEmpty(lostInstanceList)&&!isCreateOnly) { + if (CollectionUtil.isNotEmpty(lostInstanceList) && !isCreateOnly) { for (WindowInstance windowInstance : instanceList) { List emitInstances = createEmitWindowInstance(window, windowInstance); if (emitInstances != null && emitInstances.size() > 0) { for (WindowInstance emitBeforeInstance : emitInstances) { + if (StringUtil.isEmpty(window.getTimeFieldName()) && System.currentTimeMillis() - DateUtil.parse(emitBeforeInstance.getFireTime()).getTime() >= 0) { + continue; + } window.registerWindowInstance(emitBeforeInstance); - window.getSqlCache().addCache(new SQLElement(queueId, emitBeforeInstance.createWindowInstanceId(), ORMUtil.createBatchReplacetSQL(emitBeforeInstance))); - window.getWindowFireSource().registFireWindowInstanceIfNotExist(emitBeforeInstance, window); + window.getSqlCache().addCache(new SQLElement(queueId, emitBeforeInstance.createWindowInstanceId(), ORMUtil.createBatchReplaceSQL(emitBeforeInstance))); + window.getWindowTrigger().registFireWindowInstanceIfNotExist(emitBeforeInstance, window); } } window.registerWindowInstance(windowInstance); @@ -421,6 +375,48 @@ private static List createEmitAfterWindowInstance(AbstractWindow return windowInstances; } + public WindowInstance copy() { + WindowInstance windowInstance = new WindowInstance(); + windowInstance.setCanClearResource(this.canClearResource); + windowInstance.setFireTime(this.fireTime); + windowInstance.setNewWindowInstance(isNewWindowInstance); + windowInstance.setLastMaxUpdateTime(this.lastMaxUpdateTime); + windowInstance.setEndTime(this.endTime); + windowInstance.setStartTime(this.startTime); + windowInstance.setSplitId(this.splitId); + windowInstance.setWindowInstanceSplitName(this.windowInstanceSplitName); + windowInstance.setGmtModified(new Date()); + windowInstance.setGmtCreate(new Date()); + windowInstance.setWindowInstanceName(this.windowInstanceName); + windowInstance.setWindowInstanceKey(this.windowInstanceKey); + windowInstance.setWindowName(this.windowName); + windowInstance.setWindowNameSpace(this.windowNameSpace); + windowInstance.setStatus(this.status); + windowInstance.setVersion(this.version); + return windowInstance; + } + + /** + * 创建window instance的唯一ID + * + * @return + */ + public String createWindowInstanceId() { + return MapKeyUtil.createKey(splitId, windowNameSpace, windowName, windowInstanceName, startTime, endTime); + } + + public String createWindowInstanceIdWithoutSplitid() { + return MapKeyUtil.createKey(windowNameSpace, windowName, windowInstanceName, startTime, endTime); + } + + public String createWindowInstanceTriggerId() { + return MapKeyUtil.createKey(splitId, windowNameSpace, windowName, windowInstanceName, startTime, endTime, fireTime); + } + + public String createWindowInstancePartitionId() { + return StringUtil.createMD5Str(MapKeyUtil.createKey(windowNameSpace, windowName, windowInstanceName, startTime, endTime, splitId)); + } + public String getStartTime() { return startTime; } @@ -481,6 +477,10 @@ public String getWindowInstanceKey() { return windowInstanceKey; } + public void setWindowInstanceKey(String windowInstanceKey) { + this.windowInstanceKey = windowInstanceKey; + } + public String getWindowInstanceName() { return windowInstanceName; } @@ -489,10 +489,6 @@ public void setWindowInstanceName(String windowInstanceName) { this.windowInstanceName = windowInstanceName; } - public void setWindowInstanceKey(String windowInstanceKey) { - this.windowInstanceKey = windowInstanceKey; - } - public Boolean isNewWindowInstance() { return isNewWindowInstance; } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/IWindowMaxValueManager.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/IWindowMaxValueManager.java index ae114006..7e21691a 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/IWindowMaxValueManager.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/IWindowMaxValueManager.java @@ -34,14 +34,14 @@ public interface IWindowMaxValueManager { */ Long incrementAndGetSplitNumber(WindowInstance instance, String splitId); - WindowMaxValue querySplitNum(WindowInstance instance, String splitId); + /** + * */ void initMaxSplitNum(WindowInstance windowInstance, Long maxSplitNum); - -// /** + // /** // * load mutil window instance split's max split num // * // * @param keys @@ -58,22 +58,20 @@ public interface IWindowMaxValueManager { void resetSplitNum(WindowInstance instance, String splitId); -// void resetSplitNum(String key); + // void resetSplitNum(String key); // void deleteSplitNum(WindowInstance instance, String splitId); /** * save window saved max offset,can filter the less offset + * * @param name * @param oriQueueId2Offsets */ - Map saveMaxOffset(boolean isLong, String name, String shuffleId, + Map saveMaxOffset(boolean isLong, String name, String shuffleId, Map oriQueueId2Offsets); - - Map loadOffsets(String name, String shuffleId); - Map queryOffsets(String name, String shuffleId, Set oriQueueIds); } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValue.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValue.java index 7173a787..b66194f4 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValue.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValue.java @@ -24,17 +24,17 @@ * save windowintance max offset */ public class WindowMaxValue extends Entity { - public static long MAX_VALUE_BASE_VALUE=100000000; + public static long MAX_VALUE_BASE_VALUE = 100000000; protected String msgKey; - protected AtomicLong maxValue=new AtomicLong(MAX_VALUE_BASE_VALUE); - protected String maxOffset="-1"; + protected AtomicLong maxValue = new AtomicLong(MAX_VALUE_BASE_VALUE); + protected String maxOffset = "-1"; protected boolean isMaxOffsetLong; - protected AtomicLong maxEventTime=new AtomicLong();//只有window需要 + protected AtomicLong maxEventTime = new AtomicLong();//只有window需要 - public WindowMaxValue(){ - this.gmtModified=new Date(); - this.gmtCreate=new Date(); + public WindowMaxValue() { + this.gmtModified = new Date(); + this.gmtCreate = new Date(); } public Long getMaxEventTime() { @@ -42,10 +42,10 @@ public Long getMaxEventTime() { } public void setMaxEventTime(Long maxEventTime) { - if(maxEventTime==null){ + if (maxEventTime == null) { return; } - this.maxEventTime.set( maxEventTime); + this.maxEventTime.set(maxEventTime); } public String getMsgKey() { @@ -64,19 +64,19 @@ public void setMaxValue(Long maxValue) { this.maxValue.set(maxValue); } - public long comareAndSet(Long eventTime){ - if(eventTime==null){ + public long comareAndSet(Long eventTime) { + if (eventTime == null) { return maxEventTime.get(); } - long old=maxEventTime.get(); - if(old>=eventTime){ + long old = maxEventTime.get(); + if (old >= eventTime) { return old; } - boolean updateSuccess=false; - while (!updateSuccess){ - old=maxEventTime.get(); - if(eventTime>old){ - updateSuccess= maxEventTime.compareAndSet(old,eventTime); + boolean updateSuccess = false; + while (!updateSuccess) { + old = maxEventTime.get(); + if (eventTime > old) { + updateSuccess = maxEventTime.compareAndSet(old, eventTime); } else { break; } @@ -100,7 +100,7 @@ public void setMaxOffset(String maxOffset) { this.maxOffset = maxOffset; } - public long incrementAndGetMaxOffset(){ + public long incrementAndGetMaxOffset() { return maxValue.incrementAndGet(); } } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueManager.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueManager.java index eaca69a1..1fbc7a86 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueManager.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueManager.java @@ -19,10 +19,6 @@ import java.util.HashMap; import java.util.Map; import java.util.Set; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.operator.AbstractWindow; import org.apache.rocketmq.streams.window.sqlcache.SQLCache; @@ -30,7 +26,7 @@ public class WindowMaxValueManager implements IWindowMaxValueManager { protected AbstractWindow window; protected Map windowMaxValueProcessorMap = new HashMap<>(); - // protected transient ExecutorService executorService; + // protected transient ExecutorService executorService; protected transient SQLCache sqlCache; public WindowMaxValueManager(AbstractWindow window, SQLCache sqlCache) { diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueProcessor.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueProcessor.java index 4aa86ae4..370a68f8 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueProcessor.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueProcessor.java @@ -41,6 +41,7 @@ public class WindowMaxValueProcessor { protected AbstractWindow window; protected String splitId; protected SQLCache sqlCache; + protected Map windowOffsetMap = new HashMap<>();//all window offsets public WindowMaxValueProcessor(String splitId, AbstractWindow window, SQLCache sqlCache) { @@ -49,8 +50,6 @@ public WindowMaxValueProcessor(String splitId, AbstractWindow window, this.sqlCache = sqlCache; } - protected Map windowOffsetMap = new HashMap<>();//all window offsets - public Long incrementAndGetSplitNumber(WindowInstance instance) { String key = createSplitNumberKey(instance, splitId); WindowMaxValue windowMaxValue = queryOrCreateWindowOffset(key, true); @@ -100,7 +99,7 @@ public Map saveMaxOffset(boolean isLong, String name, Ma if (windowMaxValue.getMaxOffset().equals("-1")) { windowMaxValue.setMaxOffset(currentOffset); } else { - if (messageOffset.greateThan(windowMaxValue.getMaxOffset())) { + if (messageOffset.greaterThan(windowMaxValue.getMaxOffset())) { windowMaxValue.setMaxOffset(currentOffset); } } @@ -127,7 +126,7 @@ public Map loadOffset(String name) { } String keyPrefix = MapKeyUtil.createKey(name, splitId); - String sql = "select * from " + ORMUtil.getTableName(WindowMaxValue.class) + " where configure_name like '%" + name + "%' and partition like '%" + splitId + "%'"; + String sql = "select * from " + ORMUtil.getTableName(WindowMaxValue.class) + " where msg_key like '%" + keyPrefix + "%'"; List windowMaxValues = ORMUtil.queryForList(sql, null, WindowMaxValue.class); if (windowMaxValues == null || windowMaxValues.size() == 0) { return result; diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractShuffleWindow.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractShuffleWindow.java index 5338734f..0403c1fe 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractShuffleWindow.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractShuffleWindow.java @@ -16,81 +16,192 @@ */ package org.apache.rocketmq.streams.window.operator; +import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.rocketmq.streams.common.context.AbstractContext; +import org.apache.rocketmq.streams.common.channel.impl.memory.MemorySource; +import org.apache.rocketmq.streams.common.context.Context; import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.topology.shuffle.IShuffleKeyGenerator; -import org.apache.rocketmq.streams.window.minibatch.MiniBatchMsgCache; -import org.apache.rocketmq.streams.window.model.WindowCache; +import org.apache.rocketmq.streams.common.context.Message; +import org.apache.rocketmq.streams.common.model.NameCreator; +import org.apache.rocketmq.streams.common.model.NameCreatorContext; +import org.apache.rocketmq.streams.common.optimization.MessageTrace; +import org.apache.rocketmq.streams.common.topology.IWindow; +import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; +import org.apache.rocketmq.streams.common.topology.model.AbstractChainStage; +import org.apache.rocketmq.streams.common.topology.model.SectionPipeline; +import org.apache.rocketmq.streams.common.topology.stages.ShuffleChainStage; +import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.common.utils.IdUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.filter.builder.ExpressionBuilder; +import org.apache.rocketmq.streams.stage.ShuffleOutputChainStage; +import org.apache.rocketmq.streams.stage.ShuffleSourceChainStage; +import org.apache.rocketmq.streams.window.debug.DebugWriter; import org.apache.rocketmq.streams.window.model.WindowInstance; -import org.apache.rocketmq.streams.window.shuffle.ShuffleChannel; +import org.apache.rocketmq.streams.window.shuffle.ShuffleSink; +import org.apache.rocketmq.streams.window.state.impl.WindowValue; import org.apache.rocketmq.streams.window.storage.WindowStorage; -import org.apache.rocketmq.streams.window.trigger.WindowTrigger; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public abstract class AbstractShuffleWindow extends AbstractWindow { - - protected transient ShuffleChannel shuffleChannel; - protected transient AtomicBoolean hasCreated = new AtomicBoolean(false); +public abstract class AbstractShuffleWindow extends AbstractWindow implements IShuffleCallback { + protected static final Logger LOGGER = LoggerFactory.getLogger(AbstractShuffleWindow.class); + /** + * 聚合后的数据,继续走规则引擎的规则 + */ + protected static MemorySource memorySource = new MemorySource(); + protected SectionPipeline fireReceiver; + protected transient ShuffleSink shuffleSink; - @Override - protected boolean initConfigurable() { + @Override protected void startWindow() { + super.startWindow(); storage = new WindowStorage(); storage.setLocalStorageOnly(isLocalStorageOnly); - return super.initConfigurable(); } @Override - public void windowInit() { + public int fire(WindowInstance windowInstance) { + try { + Set splitIds = new HashSet<>(); + splitIds.add(windowInstance.getSplitId()); + if (shuffleSink != null) { + shuffleSink.flush(splitIds); + } + return fireWindowInstance(windowInstance, windowInstance.getSplitId()); + } catch (Exception e) { + Long time = this.eventTimeManager.getMaxEventTime(windowInstance.getSplitId()); + String maxEventTime = null; + if (time != null) { + maxEventTime = DateUtil.longToString(time); + } + LOGGER.error("[{}][{}] Window_Fire_Error_On({})_Window(startTime{}-endTime{}-fireTime{}-maxEventTime{})_ErrorMsg({})", IdUtil.instanceId(), NameCreator.getFirstPrefix(getName(), IWindow.TYPE), this.getClass().getName(), windowInstance.getStartTime(), windowInstance.getEndTime(), windowInstance.getFireTime(), maxEventTime, e.getMessage(), e); + throw new RuntimeException(e); + } } - @Override - public AbstractContext doMessage(IMessage message, AbstractContext context) { - if (hasCreated.get()==false||this.shuffleChannel==null) { - synchronized (this){ - if(hasCreated.get()==false||this.shuffleChannel==null){ - this.windowFireSource = new WindowTrigger(this); - this.windowFireSource.init(); - this.windowFireSource.start(getFireReceiver()); - this.shuffleChannel = new ShuffleChannel(this); - this.shuffleChannel.init(); - windowCache.setBatchSize(5000); - windowCache.setShuffleChannel(shuffleChannel); - windowCache.initMiniBatch(); - shuffleChannel.startChannel(); - hasCreated.set(true); + @Override public void sendFireMessage(List windowValueList, String queueId) { + try { + int count = 0; + List msgs = new ArrayList<>(); + for (WindowValue windowValue : windowValueList) { + JSONObject message = new JSONObject(); + + if (windowValue.getcomputedResult() instanceof JSONObject) { + message = (JSONObject) windowValue.getcomputedResult(); + } else { + Iterator> it = windowValue.iteratorComputedColumnResult(); + while (it.hasNext()) { + Map.Entry entry = it.next(); + message.put(entry.getKey(), entry.getValue()); + } + } + + if (StringUtil.isNotEmpty(havingExpression)) { + boolean isMatch = ExpressionBuilder.executeExecute("tmp", havingExpression, message); + if (!isMatch) { + continue; + } + } + Long fireTime = DateUtil.parseTime(windowValue.getFireTime()).getTime(); + long baseTime = 1577808000000L;//set base time from 2021-01-01 00:00:00 + int sameFireCount = 0; + if (fireMode != 0) { + Long endTime = DateUtil.parseTime(windowValue.getEndTime()).getTime(); + sameFireCount = (int) ((fireTime - endTime) / 1000) / sizeInterval * timeUnitAdjust; + if (sameFireCount >= 1) { + sameFireCount = 1; + } + } + //can keep offset in order + Long offset = ((fireTime - baseTime) / 1000 * 10 + sameFireCount) * 100000000 + windowValue.getPartitionNum(); + message.put("window_start", windowValue.getStartTime()); + message.put("window_end", windowValue.getEndTime()); + Message newMessage = memorySource.createMessage(message, queueId, offset + "", false); + newMessage.getHeader().setOffsetIsLong(true); + if (count == windowValueList.size() - 1) { + newMessage.getHeader().setNeedFlush(true); + } + + msgs.add(newMessage); + MessageTrace.joinMessage(newMessage);//关联全局监控器 + this.getFireReceiver().doMessage(newMessage, new Context(newMessage)); + + count++; + } + + if (DebugWriter.getDebugWriter(this.getName()).isOpenDebug()) { + DebugWriter.getDebugWriter(this.getName()).writeWindowFire(this, msgs, queueId); + } + } catch (Exception e) { + if (windowValueList != null) { + WindowValue windowValue = windowValueList.get(0); + Long time = this.eventTimeManager.getMaxEventTime(windowValue.getPartition()); + String maxEventTime = null; + if (time != null) { + maxEventTime = DateUtil.longToString(time); } + LOGGER.error("[{}][{}] Window_SendMsg2NextStage_Error_On({})_Window(startTime{}-endTime{}-fireTime{}-maxEventTime{})_ErrorMsg({})", IdUtil.instanceId(), NameCreator.getFirstPrefix(getName(), IWindow.TYPE), this.getClass().getName(), windowValue.getStartTime(), windowValue.getEndTime(), windowValue.getFireTime(), maxEventTime, e.getMessage(), e); + } + throw new RuntimeException(e); } - return super.doMessage(message, context); + } @Override - public int fireWindowInstance(WindowInstance windowInstance, Map queueId2Offset) { - Set splitIds = new HashSet<>(); - splitIds.add(windowInstance.getSplitId()); - shuffleChannel.flush(splitIds); - return fireWindowInstance(windowInstance, windowInstance.getSplitId(), queueId2Offset); + public AbstractChainStage createStageChain(PipelineBuilder pipelineBuilder) { + pipelineBuilder.addConfigurables(this);//必须放在第一个,为了是可以自动生成windowname,后续很多名称都是基于window创建的 + + ShuffleOutputChainStage shuffleOutputChainStage = new ShuffleOutputChainStage(); + shuffleOutputChainStage.setWindow(this); + shuffleOutputChainStage.setNameSpace(getNameSpace()); + shuffleOutputChainStage.setLabel(NameCreatorContext.get().createName("shuffle_out")); + shuffleOutputChainStage.setName(shuffleOutputChainStage.getLabel()); + + ShuffleSourceChainStage shuffleSourceChainStage = new ShuffleSourceChainStage(); + shuffleSourceChainStage.setWindow(this); + shuffleSourceChainStage.setLabel(NameCreatorContext.get().createName("shuffle_source")); + shuffleSourceChainStage.setName(shuffleSourceChainStage.getLabel()); + + ShuffleChainStage shuffleChainStage = new ShuffleChainStage(); + shuffleChainStage.setConsumeChainStage(shuffleSourceChainStage); + shuffleChainStage.setOutputChainStage(shuffleOutputChainStage); + + return shuffleChainStage; } - /** - * 接收shuffle后的消息进行计算,子类实现具体计算逻辑 - * - * @param messages - * @param instance - */ - public abstract void shuffleCalculate(List messages, WindowInstance instance, String queueId); + @Override public void addConfigurables(PipelineBuilder pipelineBuilder) { + + } + + public ShuffleSink getShuffleSink() { + return shuffleSink; + } + + public void setShuffleSink(ShuffleSink shuffleSink) { + this.shuffleSink = shuffleSink; + } /** * 触发window * * @param instance */ - protected abstract int fireWindowInstance(WindowInstance instance, String queueId, Map queueId2Offset); + protected abstract int fireWindowInstance(WindowInstance instance, String queueId); public abstract void clearCache(String queueId); + + public SectionPipeline getFireReceiver() { + return fireReceiver; + } + + @Override public void setFireReceiver(SectionPipeline fireReceiver) { + this.fireReceiver = fireReceiver; + } } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractWindow.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractWindow.java index 8fd07737..1132da95 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractWindow.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractWindow.java @@ -26,61 +26,50 @@ import java.util.Map; import java.util.Map.Entry; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; -import javafx.util.Pair; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.channel.sink.ISink; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.context.AbstractContext; +import org.apache.rocketmq.streams.common.configurable.annotation.ConfigurableReference; 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.functions.MapFunction; -import org.apache.rocketmq.streams.common.topology.ChainStage; -import org.apache.rocketmq.streams.common.topology.SectionPipeline; -import org.apache.rocketmq.streams.common.topology.builder.IStageBuilder; -import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; -import org.apache.rocketmq.streams.common.topology.model.IWindow; -import org.apache.rocketmq.streams.common.topology.stages.WindowChainStage; -import org.apache.rocketmq.streams.common.topology.stages.udf.IReducer; -import org.apache.rocketmq.streams.common.utils.Base64Utils; +import org.apache.rocketmq.streams.common.model.Pair; +import org.apache.rocketmq.streams.common.topology.IStageBuilder; +import org.apache.rocketmq.streams.common.topology.IWindow; +import org.apache.rocketmq.streams.common.topology.model.AbstractChainStage; import org.apache.rocketmq.streams.common.utils.CollectionUtil; -import org.apache.rocketmq.streams.common.utils.DateUtil; -import org.apache.rocketmq.streams.common.utils.InstantiationUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; +import org.apache.rocketmq.streams.common.utils.ReflectUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; -import org.apache.rocketmq.streams.common.utils.TraceUtil; import org.apache.rocketmq.streams.db.driver.orm.ORMUtil; -import org.apache.rocketmq.streams.filter.builder.ExpressionBuilder; import org.apache.rocketmq.streams.script.operator.expression.ScriptExpression; +import org.apache.rocketmq.streams.script.operator.expression.ScriptParameter; import org.apache.rocketmq.streams.script.operator.impl.AggregationScript; import org.apache.rocketmq.streams.script.operator.impl.FunctionScript; import org.apache.rocketmq.streams.script.parser.imp.FunctionParser; import org.apache.rocketmq.streams.script.service.IAccumulator; import org.apache.rocketmq.streams.script.service.IScriptExpression; import org.apache.rocketmq.streams.script.service.IScriptParamter; -import org.apache.rocketmq.streams.window.debug.DebugWriter; import org.apache.rocketmq.streams.window.fire.EventTimeManager; import org.apache.rocketmq.streams.window.model.FunctionExecutor; -import org.apache.rocketmq.streams.window.model.WindowCache; import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.offset.IWindowMaxValueManager; import org.apache.rocketmq.streams.window.offset.WindowMaxValueManager; import org.apache.rocketmq.streams.window.sqlcache.SQLCache; -import org.apache.rocketmq.streams.window.state.impl.WindowValue; import org.apache.rocketmq.streams.window.storage.WindowStorage; import org.apache.rocketmq.streams.window.trigger.WindowTrigger; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * window definition in the pipeline, created by user's configure in WindowChainStage */ -public abstract class AbstractWindow extends BasedConfigurable implements IAfterConfigurableRefreshListener, IWindow, IStageBuilder { - - protected static final Log LOG = LogFactory.getLog(AbstractWindow.class); +public abstract class AbstractWindow extends BasedConfigurable implements IWindow, IStageBuilder> { + private static final Logger LOGGER = LoggerFactory.getLogger(AbstractWindow.class); + public static String WINDOW_START = "window_start"; + public static String WINDOW_END = "window_end"; + public static String WINDOW_TIME = "window_time"; /** * tumble or hop window 目前不再使用了 @@ -91,27 +80,16 @@ public abstract class AbstractWindow extends BasedConfigurable implements IAfter * 用消息中的哪个字段做时间字段 */ protected String timeFieldName; - - /** - * having column in having clause eg: key:'having_sum_0001' value:'having_sum_0001=SUM(OrderPrice)<2000' note: here ignore the logical relation value may be multi expression which split by ${SCRIPT_SPLIT_CHAR} update: change sql(move the function into select clause) to escape function in having clause - */ - private Map havingMap = new HashMap<>(16); - - /** - * computed column in select clause eg: key:'max_valid_user_fail_host_cnt' value:'max_valid_user_fail_host_cnt=max(base_line_invalid_user_fail_host_cnt)' note: 1) value may be multi expression which split by ${SCRIPT_SPLIT_CHAR} 2) computed column can't be same - */ - private Map selectMap = new HashMap<>(16); - /** * SQL中group by的字段,使用;拼接,如"name;age" */ protected String groupByFieldName; - + protected boolean supportRollup;//Support rollup group calculation + protected List rollupGroupByFieldNames; /** * 意义同blink中,允许最晚的消息到达时间,单位是分钟 */ protected int waterMarkMinute = 0; - /** * size or step of window, unit: minute */ @@ -128,69 +106,50 @@ public abstract class AbstractWindow extends BasedConfigurable implements IAfter * the variable name of window size which can be got from message */ protected String sizeVariable; - /** * the coefficient to adjust window size for use minute as unit */ protected Integer sizeAdjust; - /** * the coefficient to adjust window slide for use minute as unit */ protected Integer slideAdjust; - /** * the variable name of window slide which it's value can be got from message */ protected String slideVariable; - /** * 默认为空,窗口的触发类似flink,在测试模式下,因为消息有界,期望当消息发送完成后能触发,可以设置两条消息的最大间隔,超过这个间隔,将直接触发消息 */ protected Long msgMaxGapSecond = 60 * 3L; - protected String havingExpression; - protected Long emitBeforeValue;//output frequency before window fire protected Long emitAfterValue;// output frequency after window fire protected Long maxDelay = 60 * 60L;//when emitAfterValue>0, window last delay time after window fired - - protected String contextMsgSinkName;//上下文消息保存sink /** * 是否支持过期数据的计算 过期:当前时间大于数据所在窗口的触发时间 */ protected int fireMode = 0;//0:普通触发,firetime后收到数据丢弃;1:多实例多次独立触发,在watermark时间内,同starttime,endtime创建多个实例,多次触发;2.单实例,多次独立触发,每次触发是最新值 - protected boolean isLocalStorageOnly = true;//是否只用本地存储,可以提高性能,但不保证可靠性 - protected String reduceSerializeValue;//用户自定义的operator的序列化字节数组,做了base64解码 - protected transient IReducer reducer; - protected String mapFunctionSerializeValue;//用户自定义的operator的序列化字节数组,做了base64解码 + protected boolean isOutputWindowInstanceInfo = false;//output default value :window_start,window_end time protected transient MapFunction> mapFunction; /** * the computed column and it's process of computing */ protected transient Map> columnExecuteMap = new HashMap<>(16); - /** * used in last part to filter and transfer field in case data lost during firing */ protected transient Map columnProjectMap = new HashMap<>(); - /** * 当前计算节点的PipeLine里的Window实例对象 */ protected transient ConcurrentHashMap windowInstanceMap = new ConcurrentHashMap<>(); - - /** - * 触发窗口后需要执行的逻辑 - */ - protected transient SectionPipeline fireReceiver; - /** * 全局名称 */ protected transient String WINDOW_NAME; - + protected transient boolean isEmptyGroupBy = false;//忽略window_start,window_end /** * 内部使用,定期检查窗口有没有触发 */ @@ -198,95 +157,84 @@ public abstract class AbstractWindow extends BasedConfigurable implements IAfter // protected transient ExecutorService deleteService = Executors.newSingleThreadExecutor(); - protected volatile transient WindowCache windowCache; protected transient WindowStorage storage; - protected transient WindowTrigger windowFireSource; protected transient SQLCache sqlCache; protected transient EventTimeManager eventTimeManager; - protected transient ISink contextMsgSink; - + @ConfigurableReference protected ISink contextMsgSink; + protected transient WindowTrigger windowTrigger; //create and save window instacne max partitionNum and window max eventTime protected transient IWindowMaxValueManager windowMaxValueManager; + protected Map nonStatisticalFieldNames;// Non statistical calculation field, using in rollup + protected long updateFlag = 0; + protected AtomicBoolean hasStart = new AtomicBoolean(false); + /** + * having column in having clause eg: key:'having_sum_0001' value:'having_sum_0001=SUM(OrderPrice)<2000' note: here ignore the logical relation value may be multi expression which split by ${SCRIPT_SPLIT_CHAR} update: change sql(move the function into select clause) to escape function in having clause + */ + private Map havingMap = new HashMap<>(16); + /** + * computed column in select clause eg: key:'max_valid_user_fail_host_cnt' value:'max_valid_user_fail_host_cnt=max(base_line_invalid_user_fail_host_cnt)' note: 1) value may be multi expression which split by ${SCRIPT_SPLIT_CHAR} 2) computed column can't be same + */ + private Map selectMap = new HashMap<>(16); public AbstractWindow() { setType(IWindow.TYPE); } @Override - protected boolean initConfigurable() { - boolean success = super.initConfigurable(); + public void start() { + if (hasStart.compareAndSet(false, true)) { + startWindow(); + } + } + + @Override protected boolean initConfigurable() { + initFunctionExecutor(); + if (StringUtil.isEmpty(groupByFieldName) && CollectionUtil.isEmpty(this.rollupGroupByFieldNames)) { + this.isEmptyGroupBy = true; + } /** * 如果没有db配置,不开启远程存储服务 */ if (!ORMUtil.hasConfigueDB()) { isLocalStorageOnly = true; } - sqlCache = new SQLCache(isLocalStorageOnly); - AbstractWindow window = this; - windowCache = new WindowCache() { - - @Override - protected String generateShuffleKey(IMessage message) { - return window.generateShuffleKey(message); - } - }; - windowCache.init(); - - WINDOW_NAME = MapKeyUtil.createKey(getNameSpace(), getConfigureName()); - //fireDelaySecond时间检查一次是否有窗口需要触发 - //fireWindowInstanceChecker.scheduleWithFixedDelay(this, 0, 5, TimeUnit.SECONDS); - initFunctionExecutor(); - //启动shuffle channel 实现消息shuffle以及接收shuffle消息并处理 - // FireManager.getInstance().startFireCheck(); - if (StringUtil.isNotEmpty(this.reduceSerializeValue)) { - byte[] bytes = Base64Utils.decode(this.reduceSerializeValue); - reducer = InstantiationUtil.deserializeObject(bytes); - } - if (StringUtil.isNotEmpty(this.mapFunctionSerializeValue)) { - byte[] bytes = Base64Utils.decode(this.mapFunctionSerializeValue); - this.mapFunction = InstantiationUtil.deserializeObject(bytes); - } - eventTimeManager = new EventTimeManager(); - windowMaxValueManager = new WindowMaxValueManager(this, sqlCache); - - return success; - } - /** - * 对于一条消息来说,window 首先需要检查是否有窗口实例,如果没有则创建。如果窗口实例已经超过最大的water mark,数据丢弃,否则进行消息积累 消息会先经历batchAdd 然后flush - * - * @param message - * @param context - * @return - */ - @Override - public AbstractContext doMessage(IMessage message, AbstractContext context) { - if (StringUtils.isNotEmpty(sizeVariable)) { - if (message.getMessageBody().containsKey(sizeVariable)) { - try { - this.sizeInterval = sizeAdjust * message.getMessageBody().getInteger(sizeVariable); - } catch (Exception e) { - LOG.error("failed in getting the size value, message = " + message.toString(), e); + WINDOW_NAME = MapKeyUtil.createKey(getNameSpace(), getName()); + if (StringUtil.isNotEmpty(this.groupByFieldName)) { + String[] fieldNames = groupByFieldName.split(";"); + boolean isEmpty = true; + for (String filedName : fieldNames) { + if (WINDOW_START.equals(filedName)) { + this.isOutputWindowInstanceInfo = true; + continue; } - } - } - if (StringUtils.isNotEmpty(slideVariable)) { - if (message.getMessageBody().containsKey(slideVariable)) { - try { - this.slideInterval = slideAdjust * message.getMessageBody().getInteger(slideVariable); - } catch (Exception e) { - LOG.error("failed in getting the slide value, message = " + message.toString(), e); + if (WINDOW_END.equals(filedName)) { + this.isOutputWindowInstanceInfo = true; + continue; } + if (WINDOW_TIME.equals(filedName)) { + this.isOutputWindowInstanceInfo = true; + continue; + } + isEmpty = false; + break; + } + if (isEmpty && CollectionUtil.isEmpty(this.rollupGroupByFieldNames)) { + isEmptyGroupBy = true; } } - JSONObject msg = message.getMessageBody(); - msg.put(MessageHeader.class.getSimpleName(), message.getHeader()); - msg.put(AbstractWindow.class.getSimpleName(), this); - eventTimeManager.setSource(message.getHeader().getSource()); - windowCache.batchAdd(message); - TraceUtil.debug(message.getHeader().getTraceId(), "origin message in"); - return context; + if (CollectionUtil.isNotEmpty(this.rollupGroupByFieldNames)) { + this.supportRollup = true; + } + return super.initConfigurable(); + } + + protected void startWindow() { + windowInstanceMap = new ConcurrentHashMap(); + sqlCache = new SQLCache(isLocalStorageOnly); + eventTimeManager = new EventTimeManager(); + windowMaxValueManager = new WindowMaxValueManager(this, sqlCache); } public WindowInstance createWindowInstance(String startTime, String endTime, String fireTime, String splitId) { @@ -298,7 +246,7 @@ public WindowInstance createWindowInstance(String startTime, String endTime, Str windowInstance.setGmtCreate(new Date()); windowInstance.setGmtModified(new Date()); windowInstance.setWindowInstanceName(createWindowInstanceName(startTime, endTime, fireTime)); - windowInstance.setWindowName(getConfigureName()); + windowInstance.setWindowName(getName()); windowInstance.setWindowNameSpace(getNameSpace()); String windowInstanceId = windowInstance.createWindowInstanceId(); String dbWindowInstanceId = StringUtil.createMD5Str(windowInstanceId); @@ -308,11 +256,16 @@ public WindowInstance createWindowInstance(String startTime, String endTime, Str } else { windowInstance.setCanClearResource(true); } - windowInstance.setWindowInstanceSplitName(StringUtil.createMD5Str(MapKeyUtil.createKey(getNameSpace(), getConfigureName(), splitId))); + windowInstance.setWindowInstanceSplitName(StringUtil.createMD5Str(MapKeyUtil.createKey(getNameSpace(), getName(), splitId))); windowInstance.setNewWindowInstance(true); return windowInstance; } + public AbstractWindow copy() { + byte[] bytes = ReflectUtil.serialize(this); + return (AbstractWindow) ReflectUtil.deserialize(bytes); + } + /** * 创建window instance name * @@ -322,7 +275,7 @@ public WindowInstance createWindowInstance(String startTime, String endTime, Str * @return */ public String createWindowInstanceName(String startTime, String endTime, String fireTime) { - return (fireMode == 0 || fireMode == 2) ? getConfigureName() : fireTime; + return (fireMode == 0 || fireMode == 2) ? getName() : fireTime; } /** @@ -339,7 +292,15 @@ public long incrementAndGetSplitNumber(WindowInstance instance, String shuffleId public abstract Class getWindowBaseValueClass(); - public abstract int fireWindowInstance(WindowInstance windowInstance, Map queueId2Offset); + /** + * 计算每条记录的group by值,对于groupby分组,里面任何字段不能为null值,如果为null值,这条记录会被忽略 + * + * @param message + * @return + */ + @Override public String generateShuffleKey(IMessage message) { + return generateGroupByValue(message).get(0).getValue(); + } /** * 计算每条记录的group by值,对于groupby分组,里面任何字段不能为null值,如果为null值,这条记录会被忽略 @@ -347,9 +308,36 @@ public long incrementAndGetSplitNumber(WindowInstance instance, String shuffleId * @param message * @return */ - @Override - public String generateShuffleKey(IMessage message) { - if (StringUtil.isEmpty(groupByFieldName)) { + public List> generateGroupByValue(IMessage message) { + if (isEmptyGroupBy) { + List> list = new ArrayList<>(); + list.add(new Pair("globle_window", "globle_window")); + return list; + } + List> groupValus = new ArrayList<>(); + List groupFieldNames = new ArrayList<>(); + if (supportRollup && this.rollupGroupByFieldNames != null) { + if (StringUtil.isEmpty(this.groupByFieldName)) { + groupFieldNames = this.rollupGroupByFieldNames; + } else { + for (String rollupGroupByFieldName : this.rollupGroupByFieldNames) { + groupFieldNames.add(this.groupByFieldName + ";" + rollupGroupByFieldName); + } + } + + } else { + groupFieldNames.add(this.groupByFieldName); + } + for (String groupFieldName : groupFieldNames) { + String groupValue = generateGroupByValue(message, groupFieldName); + groupValus.add(new Pair<>(groupFieldName, groupValue)); + } + return groupValus; + + } + + public String generateGroupByValue(IMessage message, String groupByFieldName) { + if (isEmptyGroupBy) { return "globle_window"; } JSONObject msg = message.getMessageBody(); @@ -358,6 +346,15 @@ public String generateShuffleKey(IMessage message) { boolean isFirst = true; int i = 0; for (String filedName : fieldNames) { + if (WINDOW_START.equals(filedName)) { + continue; + } + if (WINDOW_END.equals(filedName)) { + continue; + } + if (WINDOW_TIME.equals(filedName)) { + continue; + } if (isFirst) { isFirst = false; } @@ -368,15 +365,6 @@ public String generateShuffleKey(IMessage message) { return MapKeyUtil.createKey(values); } - public abstract void clearFireWindowInstance(WindowInstance windowInstance); - - public void clearFire(WindowInstance windowInstance) { - if (windowInstance == null) { - return; - } - clearFireWindowInstance(windowInstance); - } - /** * init the function executor TODO: 1) function executor may be parsed in parser module; */ @@ -385,15 +373,17 @@ protected void initFunctionExecutor() { columnExecuteMap.clear(); columnProjectMap.clear(); // + Map nonStatisticalFieldNames = new HashMap<>(); for (Entry entry : selectMap.entrySet()) { String computedColumn = entry.getKey(); columnProjectMap.put(computedColumn, computedColumn); String scriptString = entry.getValue(); if (StringUtil.isEmpty(computedColumn) || StringUtil.isEmpty(scriptString)) { - LOG.warn("computed column or it's expression can not be empty! column = " + computedColumn + " expression = " + scriptString); + LOGGER.warn("computed column or it's expression can not be empty! column = " + computedColumn + " expression = " + scriptString); continue; } if (computedColumn.equals(scriptString)) { + nonStatisticalFieldNames.put(computedColumn, computedColumn); continue; } // @@ -402,10 +392,11 @@ protected void initFunctionExecutor() { try { functionList = FunctionParser.getInstance().parse(scriptString); } catch (Exception e) { - LOG.error("failed in parsing script expression = " + scriptString + " window = " + WINDOW_NAME); + LOGGER.error("failed in parsing script expression = " + scriptString + " window = " + WINDOW_NAME); throw new RuntimeException("failed in parsing operator expression = " + scriptString); } if (CollectionUtil.isNotEmpty(functionList)) { + boolean hasAggregationScript = false; StringBuilder scriptBuilder = new StringBuilder(); for (IScriptExpression expression : functionList) { String functionName = expression.getFunctionName(); @@ -413,6 +404,7 @@ protected void initFunctionExecutor() { String theScript = expression.getExpressionDescription(); IAccumulator director = AggregationScript.getAggregationFunction(functionName); if (director != null) { + hasAggregationScript = true; if (scriptBuilder.length() != 0) { FunctionScript scalarEngine = new FunctionScript(scriptBuilder.toString()); scalarEngine.init(); @@ -432,15 +424,32 @@ protected void initFunctionExecutor() { scalarEngine.init(); scriptExecutorList.add(new FunctionExecutor(computedColumn + "_" + scriptExecutorList.size(), scalarEngine)); } + if (hasAggregationScript == false) { + FunctionScript functionScript = new FunctionScript(scriptString); + functionScript.init(); + if (functionScript.getScriptExpressions() != null && functionScript.getScriptExpressions().size() == 1) { + IScriptExpression scriptExpression = functionScript.getScriptExpressions().get(0); + if (StringUtil.isEmpty(scriptExpression.getFunctionName()) && scriptExpression.getScriptParamters() != null && scriptExpression.getScriptParamters().size() == 1) { + if (IScriptParamter.class.isInstance(scriptExpression.getScriptParamters().get(0))) { + IScriptParamter scriptParamter = (IScriptParamter) scriptExpression.getScriptParamters().get(0); + if (isAssignmentExpression(scriptParamter)) { + nonStatisticalFieldNames.put(scriptParamter.getScriptParameterStr(), computedColumn); + } + } + + } + } + } + this.nonStatisticalFieldNames = nonStatisticalFieldNames; columnExecuteMap.put(computedColumn, scriptExecutorList); } else { - LOG.error("parser's result is empty, script expression = " + scriptString + " window = " + WINDOW_NAME); + LOGGER.error("parser's result is empty, script expression = " + scriptString + " window = " + WINDOW_NAME); throw new RuntimeException("parser's result is empty, operator expression = " + scriptString); } } - if (LOG.isDebugEnabled()) { + if (LOGGER.isDebugEnabled()) { Iterator>> iterator = columnExecuteMap.entrySet().iterator(); - LOG.debug("window function execute split as follows:\t"); + LOGGER.debug("window function execute split as follows:\t"); while (iterator.hasNext()) { Entry> entry = iterator.next(); StringBuilder builder = new StringBuilder(); @@ -451,11 +460,24 @@ protected void initFunctionExecutor() { builder.append(((FunctionScript) executor.getExecutor()).getScript()).append("\t"); } } - LOG.debug(entry.getKey() + " -> " + builder.toString()); + LOGGER.debug(entry.getKey() + " -> " + builder.toString()); } } } + protected boolean isAssignmentExpression(IScriptParamter paramter) { + if (!ScriptParameter.class.isInstance(paramter)) { + return false; + } + ScriptParameter scriptParameter = (ScriptParameter) paramter; + String fieldName = null; + if (StringUtil.isEmpty(scriptParameter.getFunctionName()) && StringUtil.isEmpty(scriptParameter.getRigthVarName()) && StringUtil.isNotEmpty(scriptParameter.getLeftVarName())) { + return true; + } + + return false; + } + /** * 根据消息获取对应的window instance 列表 * @@ -467,7 +489,7 @@ public List queryOrCreateWindowInstance(IMessage message, String } public List queryOrCreateWindowInstanceOnly(IMessage message, String queueId) { - return WindowInstance.getOrCreateWindowInstance(this, WindowInstance.getOccurTime(this, message), timeUnitAdjust, queueId,true); + return WindowInstance.getOrCreateWindowInstance(this, WindowInstance.getOccurTime(this, message), timeUnitAdjust, queueId, true); } public WindowInstance registerWindowInstance(WindowInstance windowInstance) { @@ -517,87 +539,6 @@ public Long getMaxEventTime(String queueId) { return this.eventTimeManager.getMaxEventTime(queueId); } - /** - * 聚合后的数据,继续走规则引擎的规则 - * - * @param windowValueList - */ - public void sendFireMessage(List windowValueList, String queueId) { - int count = 0; - List msgs = new ArrayList<>(); - for (WindowValue windowValue : windowValueList) { - JSONObject message = new JSONObject(); - - if (JSONObject.class.isInstance(windowValue.getcomputedResult())) { - message = (JSONObject) windowValue.getcomputedResult(); - } else { - Iterator> it = windowValue.iteratorComputedColumnResult(); - while (it.hasNext()) { - Entry entry = it.next(); - message.put(entry.getKey(), entry.getValue()); - } - } - - if (StringUtil.isNotEmpty(havingExpression)) { - boolean isMatch = ExpressionBuilder.executeExecute("tmp", havingExpression, message); - if (!isMatch) { - continue; - } - } - Long fireTime = DateUtil.parseTime(windowValue.getFireTime()).getTime(); - long baseTime = 1577808000000L;//set base time from 2021-01-01 00:00:00 - int sameFireCount = 0; - if (fireMode != 0) { - Long endTime = DateUtil.parseTime(windowValue.getEndTime()).getTime(); - sameFireCount = (int) ((fireTime - endTime) / 1000) / sizeInterval * timeUnitAdjust; - if (sameFireCount >= 1) { - sameFireCount = 1; - } - } - //can keep offset in order - Long offset = ((fireTime - baseTime) / 1000 * 10 + sameFireCount) * 100000000 + windowValue.getPartitionNum(); - message.put("start_time", windowValue.getStartTime()); - message.put("end_time", windowValue.getEndTime()); - message.put("fire_time", windowValue.getFireTime()); - Message newMessage = windowFireSource.createMessage(message, queueId, offset + "", false); - newMessage.getHeader().setOffsetIsLong(true); - if (count == windowValueList.size() - 1) { - newMessage.getHeader().setNeedFlush(true); - } - - msgs.add(newMessage); - windowFireSource.executeMessage(newMessage); - - count++; - } - - if (DebugWriter.getDebugWriter(this.getConfigureName()).isOpenDebug()) { - DebugWriter.getDebugWriter(this.getConfigureName()).writeWindowFire(this, msgs, queueId); - } - } - - @Override public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - this.contextMsgSink=configurableService.queryConfigurable(ISink.TYPE,contextMsgSinkName); - } - - @Override - public ChainStage createStageChain(PipelineBuilder pipelineBuilder) { - pipelineBuilder.addConfigurables(this); - WindowChainStage windowChainStage = new WindowChainStage(); - windowChainStage.setWindow(this); - windowChainStage.setNameSpace(getNameSpace()); - return windowChainStage; - } - - @Override - public void addConfigurables(PipelineBuilder pipelineBuilder) { - - } - - public void setSizeVariable(String variableName) { - sizeVariable = variableName; - } - public String getWindowType() { return windowType; } @@ -662,19 +603,7 @@ private void setWindowInstanceMap(ConcurrentHashMap wind this.windowInstanceMap = windowInstanceMap; } - public SectionPipeline getFireReceiver() { - return fireReceiver; - } - - @Override - public void setFireReceiver(SectionPipeline fireReceiver) { - if (this.fireReceiver == null) { - this.fireReceiver = fireReceiver; - } - } - - @Override - public boolean isSynchronous() { + @Override public boolean isSynchronous() { return false; } @@ -698,6 +627,10 @@ public String getSizeVariable() { return sizeVariable; } + public void setSizeVariable(String variableName) { + sizeVariable = variableName; + } + public Integer getSizeAdjust() { return sizeAdjust; } @@ -722,24 +655,6 @@ public void setSlideVariable(String slideVariable) { this.slideVariable = slideVariable; } - public String getReduceSerializeValue() { - return reduceSerializeValue; - } - - public void setReduceSerializeValue(String reduceSerializeValue) { - this.reduceSerializeValue = reduceSerializeValue; - } - - public IReducer getReducer() { - return reducer; - } - - public void setReducer(IReducer reducer) { - this.reducer = reducer; - byte[] bytes = InstantiationUtil.serializeObject(reducer); - this.reduceSerializeValue = Base64Utils.encode(bytes); - } - public int getTimeUnitAdjust() { return timeUnitAdjust; } @@ -769,24 +684,10 @@ public void removeInstanceFromMap(WindowInstance windowInstance) { } - @Override - public void windowInit() { - - } - - @Override - public WindowCache getWindowCache() { - return windowCache; - } - public WindowStorage getStorage() { return storage; } - public WindowTrigger getWindowFireSource() { - return windowFireSource; - } - public IWindowMaxValueManager getWindowMaxValueManager() { return windowMaxValueManager; } @@ -851,39 +752,89 @@ public ISink getContextMsgSink() { return contextMsgSink; } - public String getContextMsgSinkName() { - return contextMsgSinkName; - } - - public void setContextMsgSinkName(String contextMsgSinkName) { - this.contextMsgSinkName = contextMsgSinkName; - } - - public String getMapFunctionSerializeValue() { - return mapFunctionSerializeValue; + public void setContextMsgSink(ISink contextMsgSink) { + this.contextMsgSink = contextMsgSink; } - public void setMapFunctionSerializeValue(String mapFunctionSerializeValue) { - this.mapFunctionSerializeValue = mapFunctionSerializeValue; - } - - public void saveMsgContext(String queueId,WindowInstance windowInstance, List messages) { - if(this.mapFunction!=null&&this.contextMsgSink!=null){ - if(messages!=null){ - for(IMessage message:messages){ - JSONObject msg=message.getMessageBody(); + public void saveMsgContext(String queueId, WindowInstance windowInstance, List messages) { + if (this.mapFunction != null && this.contextMsgSink != null) { + if (messages != null) { + for (IMessage message : messages) { + JSONObject msg = message.getMessageBody(); try { - msg=this.mapFunction.map(new Pair(windowInstance,msg)); - Message copyMsg=new Message(msg); + msg = this.mapFunction.map(new Pair(windowInstance, msg)); + Message copyMsg = new Message(msg); copyMsg.getHeader().setQueueId(queueId); copyMsg.getHeader().setOffset(message.getHeader().getOffset()); this.contextMsgSink.batchAdd(copyMsg); } catch (Exception e) { - throw new RuntimeException("save window context msg error ",e); + throw new RuntimeException("save window context msg error ", e); } } this.contextMsgSink.flush(); } } } + + @Override public void destroy() { + super.destroy(); + hasStart.set(false); + } + + public MapFunction> getMapFunction() { + return mapFunction; + } + + public void setMapFunction( + MapFunction> mapFunction) { + this.mapFunction = mapFunction; + } + + public boolean isOutputWindowInstanceInfo() { + return isOutputWindowInstanceInfo; + } + + public void setOutputWindowInstanceInfo(boolean outputWindowInstanceInfo) { + isOutputWindowInstanceInfo = outputWindowInstanceInfo; + } + + public boolean isEmptyGroupBy() { + return isEmptyGroupBy; + } + + public boolean isSupportRollup() { + return supportRollup; + } + + public void setSupportRollup(boolean supportRollup) { + this.supportRollup = supportRollup; + } + + public List getRollupGroupByFieldNames() { + return rollupGroupByFieldNames; + } + + public void setRollupGroupByFieldNames(List rollupGroupByFieldNames) { + this.rollupGroupByFieldNames = rollupGroupByFieldNames; + } + + public Map getNonStatisticalFieldNames() { + return nonStatisticalFieldNames; + } + + public void setNonStatisticalFieldNames(Map nonStatisticalFieldNames) { + this.nonStatisticalFieldNames = nonStatisticalFieldNames; + } + + public WindowTrigger getWindowTrigger() { + return windowTrigger; + } + + public void setWindowTrigger(WindowTrigger windowTrigger) { + this.windowTrigger = windowTrigger; + } + + public long getUpdateFlag() { + return updateFlag; + } } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/IShuffleCallback.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/IShuffleCallback.java new file mode 100644 index 00000000..aef8ef31 --- /dev/null +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/IShuffleCallback.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.window.operator; + +import java.util.List; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.topology.model.SectionPipeline; +import org.apache.rocketmq.streams.window.model.WindowInstance; +import org.apache.rocketmq.streams.window.state.impl.WindowValue; + +public interface IShuffleCallback { + + /** + * 当收到消息时,应该如何计算,如count算子,没收到一条消息就会加1 + * + * @param receiveMessages + * @param instance + */ + void accumulate(List receiveMessages, WindowInstance instance); + + /** + * 当窗口触发时,应当如何处理 + * + * @return + */ + int fire(WindowInstance instance); + + /** + * 窗口触发后清理数据 + * + * @param windowInstance + */ + void clearWindowInstance(WindowInstance windowInstance); + + /** + * 发送数据到下一个节点 + * + * @param windowValueList + * @param queueId + */ + void sendFireMessage(List windowValueList, String queueId); + + /** + * 窗口触发后,需要执行的逻辑 + * + * @param receiver + */ + void setFireReceiver(SectionPipeline receiver); + +} diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/OrderBy.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/OrderBy.java index 9e8b5236..c78a44ef 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/OrderBy.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/OrderBy.java @@ -20,12 +20,12 @@ public class OrderBy { protected String fieldName; - protected boolean isAsc=true; + protected boolean isAsc = true; protected DataType dataType; - public OrderBy(String fieldName,boolean isAsc){ - this.fieldName=fieldName; - this.isAsc=isAsc; + public OrderBy(String fieldName, boolean isAsc) { + this.fieldName = fieldName; + this.isAsc = isAsc; } public DataType getDataType() { diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/OverWindow.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/OverWindow.java index 7f0f982f..67a21ce0 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/OverWindow.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/OverWindow.java @@ -18,10 +18,13 @@ import java.util.Date; import java.util.List; -import java.util.Map; import org.apache.rocketmq.streams.common.cache.compress.impl.IntValueKV; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.topology.ISynchronousWindow; +import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; +import org.apache.rocketmq.streams.common.topology.model.AbstractChainStage; +import org.apache.rocketmq.streams.common.topology.stages.SynchronousWindowChainStage; import org.apache.rocketmq.streams.common.utils.DateUtil; import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.operator.AbstractWindow; @@ -29,7 +32,7 @@ /** * 只支持 时间去重的场景,日志是按系统时间顺序,所以不落盘。需要设置groupByFieldName和rowNumerName字段 */ -public class OverWindow extends AbstractWindow { +public class OverWindow extends AbstractWindow implements ISynchronousWindow { private static int MAX_SIZE = 1000000; protected transient IntValueKV intValueKV; @@ -38,7 +41,7 @@ public class OverWindow extends AbstractWindow { * 需要把生成的序列号返回设置到message,这个是序列号对应的名字 */ protected String rowNumerName; - protected boolean isReservedOne=false; + protected boolean isReservedOne = false; /** * 针对这个窗口实例完成计算,实际上是写入了缓存,在flush时完成真正的计算。写入缓存时把上下文(header,windowinstance,window)保存在消息中 @@ -47,7 +50,7 @@ public class OverWindow extends AbstractWindow { * @param context */ @Override - public AbstractContext doMessage(IMessage message, AbstractContext context) { + public void accumulateDirectly(IMessage message, AbstractContext context) { String key = generateShuffleKey(message); createWindowInstanceByDate(new Date()); Integer value = intValueKV.get(key); @@ -61,13 +64,13 @@ public AbstractContext doMessage(IMessage message, AbstractContext con } } - if(isReservedOne){ - if(value>1){ + if (isReservedOne) { + if (value > 1) { context.breakExecute(); - return context; + return; } } - if(rowNumerName!=null){ + if (rowNumerName != null) { message.getMessageBody().put(rowNumerName, value); } @@ -81,7 +84,6 @@ public AbstractContext doMessage(IMessage message, AbstractContext con } } } - return context; } /** @@ -150,16 +152,6 @@ public Class getWindowBaseValueClass() { return null; } - @Override - public int fireWindowInstance(WindowInstance windowInstance, Map queueId2Offsets) { - return 0; - } - - @Override - public void clearFireWindowInstance(WindowInstance windowInstance) { - - } - public boolean isReservedOne() { return isReservedOne; } @@ -176,4 +168,14 @@ public void setRowNumerName(String rowNumerName) { this.rowNumerName = rowNumerName; } + @Override public AbstractChainStage createStageChain(PipelineBuilder pipelineBuilder) { + SynchronousWindowChainStage windowChainStage = new SynchronousWindowChainStage(); + windowChainStage.setWindow(this); + windowChainStage.setNameSpace(getNameSpace()); + return windowChainStage; + } + + @Override public void addConfigurables(PipelineBuilder pipelineBuilder) { + pipelineBuilder.addConfigurables(this); + } } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/SessionOperator.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/SessionOperator.java index 8da03987..86da8e30 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/SessionOperator.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/SessionOperator.java @@ -30,8 +30,6 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import org.apache.commons.lang3.tuple.Pair; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.utils.Base64Utils; import org.apache.rocketmq.streams.common.utils.CollectionUtil; @@ -43,6 +41,8 @@ import org.apache.rocketmq.streams.window.state.WindowBaseValue; import org.apache.rocketmq.streams.window.state.impl.WindowValue; import org.apache.rocketmq.streams.window.storage.WindowStorage.WindowBaseValueIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * an implementation of session window to save extra memory for different group by window instances @@ -51,12 +51,9 @@ */ public class SessionOperator extends WindowOperator { - protected static final Log LOG = LogFactory.getLog(SessionOperator.class); - public static final String SESSION_WINDOW_BEGIN_TIME = "1970-01-01"; - public static final String SESSION_WINDOW_END_TIME = "9999-01-01"; - + private static final Logger LOGGER = LoggerFactory.getLogger(SessionOperator.class); private static final String SESSION_DATETIME_PATTERN = "yyyy-MM-dd HH:mm:ss"; private static final String ORDER_BY_FIRE_TIME_PREFIX = "_order_by_fire_time_"; @@ -76,6 +73,14 @@ public SessionOperator(Integer timeout) { this.sessionTimeOut = Optional.ofNullable(timeout).orElse(sessionTimeOut); } + private static String createPrefixKey(WindowValue windowValue, WindowInstance windowInstance, String queueId) { + return MapKeyUtil.createKey(getOrderBypPrefix() + queueId, windowInstance.createWindowInstanceId(), windowValue.getFireTime(), String.valueOf(windowValue.getPartitionNum()), windowValue.getGroupBy()); + } + + protected static String getOrderBypPrefix() { + return ORDER_BY_FIRE_TIME_PREFIX; + } + public int getSessionTimeOut() { return sessionTimeOut; } @@ -108,11 +113,11 @@ public List queryOrCreateWindowInstance(IMessage message, String //out of order data, normal fire mode considered only Long maxEventTime = getMaxEventTime(queueId); if (maxEventTime == null) { - LOG.warn("use current time as max event time!"); + LOGGER.warn("use current time as max event time!"); maxEventTime = System.currentTimeMillis(); } if (fireDate.getTime() <= maxEventTime) { - LOG.warn("message is discarded as out of date! fire time: " + fireDate.getTime() + " max event time: " + maxEventTime); + LOGGER.warn("message is discarded as out of date! fire time: " + fireDate.getTime() + " max event time: " + maxEventTime); return new ArrayList<>(); } instance.setFireTime(DateUtil.format(fireDate, SESSION_DATETIME_PATTERN)); @@ -128,7 +133,7 @@ public List queryOrCreateWindowInstance(IMessage message, String } @Override - public void shuffleCalculate(List messages, WindowInstance instance, String queueId) { + protected void shuffleCalculate(List messages, WindowInstance instance, String queueId, String groupFieldName) { /** * 1、消息分组:获得分组的groupBy值和对应的消息 * 2、获取已有所有分组的窗口计算结果:1)通过queueId、instance和groupBy计算存储的key;2)调用存储的获取接口; @@ -139,12 +144,12 @@ public void shuffleCalculate(List messages, WindowInstance instance, S synchronized (lock) { // List groupSortedByOffset = new ArrayList<>(); - Map> groupBy = groupByGroupName(messages, groupSortedByOffset); + Map> groupBy = groupByGroupName(messages, groupSortedByOffset, groupFieldName); int groupSize = groupSortedByOffset.size(); // Map value2StoreMap = new HashMap<>(groupSize); for (String groupValue : groupSortedByOffset) { - String storeKey = createStoreKey(queueId, groupValue, instance); + String storeKey = createStoreKey(queueId, groupValue, instance, groupFieldName); value2StoreMap.put(groupValue, storeKey); } Map> storeValueMap = storage.multiGetList(WindowValue.class, new ArrayList<>(value2StoreMap.values())); @@ -300,16 +305,12 @@ private void deletePrefixValue(List keyList) { storage.getLocalStorage().removeKeys(keyList); } - private static String createPrefixKey(WindowValue windowValue, WindowInstance windowInstance, String queueId) { - return MapKeyUtil.createKey(getOrderBypPrefix() + queueId, windowInstance.createWindowInstanceId(), windowValue.getFireTime(), String.valueOf(windowValue.getPartitionNum()), windowValue.getGroupBy()); - } - private Pair getSessionTime(IMessage message) { Long occurTime = System.currentTimeMillis(); try { occurTime = WindowInstance.getOccurTime(this, message); } catch (Exception e) { - LOG.error("failed in computing occur time from the message!", e); + LOGGER.error("failed in computing occur time from the message!", e); } Date occurDate = new Date(occurTime); Date endDate = DateUtil.addDate(TimeUnit.SECONDS, occurDate, sessionTimeOut); @@ -359,29 +360,24 @@ protected WindowValue createWindowValue(String queueId, String groupBy, WindowIn value.setGroupBy(groupBy); value.setMsgKey(StringUtil.createMD5Str(storeKey)); //FIXME shuffleId vs queueId TODO delete assert - String shuffleId = shuffleChannel.getChannelQueue(groupBy).getQueueId(); - assert shuffleId.equalsIgnoreCase(queueId); + // String shuffleId = shuffleChannel.getChannelQueue(groupBy).getQueueId(); + //assert shuffleId.equalsIgnoreCase(queueId); value.setPartitionNum(createPartitionNum(value, queueId, instance)); - value.setPartition(shuffleId); + value.setPartition(queueId); value.setWindowInstancePartitionId(instance.getWindowInstanceKey()); value.setWindowInstanceId(instance.getWindowInstanceKey()); return value; } - protected static String getOrderBypPrefix() { - return ORDER_BY_FIRE_TIME_PREFIX; - } - /** * update session's next fire time * * @param windowInstance * @param queueId - * @param queueId2Offset * @return */ @Override - public int fireWindowInstance(WindowInstance windowInstance, String queueId, Map queueId2Offset) { + public int fireWindowInstance(WindowInstance windowInstance, String queueId) { synchronized (lock) { //get iterator sorted by fire time WindowBaseValueIterator it = storage.loadWindowInstanceSplitData(getOrderBypPrefix(), queueId, windowInstance.createWindowInstanceId(), null, getWindowBaseValueClass()); @@ -412,9 +408,9 @@ public int fireWindowInstance(WindowInstance windowInstance, String queueId, Map WindowInstance existedWindowInstance = searchWindowInstance(instanceId); if (existedWindowInstance != null) { existedWindowInstance.setFireTime(DateUtil.format(new Date(nextFireTime))); - windowFireSource.registFireWindowInstanceIfNotExist(windowInstance, this); + getWindowTrigger().registFireWindowInstanceIfNotExist(windowInstance, this); } else { - LOG.error("window instance lost, queueId: " + queueId + " ,fire time" + windowInstance.getFireTime()); + LOGGER.error("window instance lost, queueId: " + queueId + " ,fire time" + windowInstance.getFireTime()); } } // @@ -460,7 +456,7 @@ protected void clearWindowValues(List deleteValueList, String queue Set valueIdSet = new HashSet<>(deleteValueList.size()); Set prefixKeySet = new HashSet<>(deleteValueList.size()); for (WindowValue windowValue : deleteValueList) { - String storeKey = createStoreKey(queueId, windowValue.getGroupBy(), instance); + String storeKey = createStoreKey(queueId, windowValue.getGroupBy(), instance, windowValue.getGroupByFieldName()); String prefixKey = createPrefixKey(windowValue, instance, queueId); Long valueId = windowValue.getPartitionNum(); storeKeySet.add(storeKey); diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/ShuffleOverWindow.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/ShuffleOverWindow.java index e3486764..975eca67 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/ShuffleOverWindow.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/ShuffleOverWindow.java @@ -20,12 +20,13 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.state.impl.WindowValue; -public class ShuffleOverWindow extends WindowOperator implements IAfterConfigurableRefreshListener { +public class ShuffleOverWindow extends WindowOperator { protected static String TOPN_KEY = "___TopN_"; protected transient List orderList; protected List orderFieldNames;//name contains 2 part:name;true/false @@ -33,14 +34,53 @@ public class ShuffleOverWindow extends WindowOperator implements IAfterConfigura /** * 需要把生成的序列号返回设置到message,这个是序列号对应的名字 */ - protected String rowNumerName; + protected String rowNumerName = "rowNum"; + + protected int startRowNum;//默认为0,当设置这个值时,小于这个值的行被丢弃 + + protected transient Long lastFireTime; + protected transient Long lastChangeTime; @Override protected boolean initConfigurable() { boolean success = super.initConfigurable(); - this.setSizeInterval(3600); - this.setSlideInterval(3600); + if (this.getSizeInterval() == 0) { + String windowSizeStr = getConfiguration().getProperty(ConfigurationKey.DIPPER_WINDOW_OVER_DEFAULT_INTERVAL_SIZE); + int windowSize = 3600; + if (StringUtil.isNotEmpty(windowSizeStr)) { + windowSize = Integer.valueOf(windowSizeStr); + } + this.setSizeInterval(windowSize); + this.setSlideInterval(windowSize); + } + if (this.getEmitBeforeValue() == null) { + String emitBeforeStr = getConfiguration().getProperty(ConfigurationKey.DIPPER_WINDOW_OVER_DEFAULT_EMIT_BEFORE_SECOND); + long emitBefore = 60L; + if (StringUtil.isNotEmpty(emitBeforeStr)) { + emitBefore = Integer.valueOf(emitBeforeStr); + } + + this.setEmitBeforeValue(emitBefore); + } this.setTimeUnitAdjust(1); - this.setEmitBeforeValue(5L); + + if (this.orderFieldNames == null) { + return true; + } + if (orderList == null) { + synchronized (this) { + if (orderList == null) { + List list = new ArrayList<>(); + for (String name : orderFieldNames) { + String[] values = name.split(";"); + String fieldName = values[0]; + Boolean isAsc = Boolean.valueOf(values[1]); + OrderBy orderBy = new OrderBy(fieldName, isAsc); + list.add(orderBy); + } + this.orderList = list; + } + } + } return success; } @@ -51,52 +91,45 @@ public class ShuffleOverWindow extends WindowOperator implements IAfterConfigura topNState = new TopNState(topN); } - topNState.addAndSortMsg(msg.getMessageBody(), orderList); + boolean isChanged = topNState.addAndSortMsg(msg.getMessageBody(), orderList); + if (isChanged) { + lastChangeTime = System.currentTimeMillis(); + } windowValue.putAggColumnResult(TOPN_KEY, topNState); } + @Override + public int fireWindowInstance(WindowInstance instance, String queueId) { + if (lastFireTime != null && lastChangeTime != null) { + if (lastFireTime - lastChangeTime > 0) { + return 0; + } + } + int result = super.fireWindowInstance(instance, queueId); + lastFireTime = System.currentTimeMillis(); + return result; + } + @Override public void sendFireMessage(List windowValueList, String queueId) { List windowValues = new ArrayList<>(); for (WindowValue windowValue : windowValueList) { TopNState topNState = (TopNState) windowValue.getAggColumnResultByKey(TOPN_KEY); - if (topNState.isChanged()) { - int i = 0; - for (Map msg : topNState.getOrderMsgs(this.rowNumerName, this.getSelectMap().keySet())) { - WindowValue copy = windowValue.clone(); - copy.setAggColumnMap(new HashMap<>()); - copy.setPartitionNum(copy.getPartitionNum() * topN + i); - copy.putComputedColumnResult(msg); - windowValues.add(copy); - i++; - } - topNState.setChanged(false); + int i = 0; + for (Map msg : topNState.getOrderMsgs(this.rowNumerName, this.getSelectMap().keySet(), this.startRowNum)) { + WindowValue copy = windowValue.clone(); + copy.setAggColumnMap(new HashMap<>()); + copy.setPartitionNum(copy.getPartitionNum() * topN + i); + copy.putComputedColumnResult(msg); + windowValues.add(copy); + i++; } } super.sendFireMessage(windowValues, queueId); } - @Override public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - super.windowInit(); - if (orderList == null) { - synchronized (this) { - if (orderList == null) { - List list = new ArrayList<>(); - for (String name : orderFieldNames) { - String[] values = name.split(";"); - String fieldName = values[0]; - Boolean isAsc = Boolean.valueOf(values[1]); - OrderBy orderBy = new OrderBy(fieldName, isAsc); - list.add(orderBy); - } - this.orderList = list; - } - } - } - } - public List getOrderFieldNames() { return orderFieldNames; } @@ -120,4 +153,12 @@ public String getRowNumerName() { public void setRowNumerName(String rowNumerName) { this.rowNumerName = rowNumerName; } + + public int getStartRowNum() { + return startRowNum; + } + + public void setStartRowNum(int startRowNum) { + this.startRowNum = startRowNum; + } } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/TopNState.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/TopNState.java index 71fafc80..1fa4557b 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/TopNState.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/TopNState.java @@ -31,123 +31,190 @@ import org.apache.rocketmq.streams.common.datatype.ListDataType; import org.apache.rocketmq.streams.common.datatype.MapDataType; import org.apache.rocketmq.streams.common.datatype.StringDataType; +import org.apache.rocketmq.streams.common.utils.CollectionUtil; import org.apache.rocketmq.streams.common.utils.DataTypeUtil; -import org.apache.rocketmq.streams.window.model.WindowCache; +import org.apache.rocketmq.streams.window.WindowConstants; import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.operator.AbstractWindow; -import org.apache.rocketmq.streams.window.shuffle.ShuffleChannel; public class TopNState implements IJsonable { - private static final String SIGN ="#@#%@" ; - private static final String NULL ="" ; - protected List sortValues=new ArrayList<>(); - protected Map orderByValue2Msgs=new HashMap<>(); - protected int topN=100; - protected boolean isChanged=false; + private static final String SIGN = "#@#%@"; + private static final String NULL = ""; + protected List sortValues = new ArrayList<>(); + protected Map orderByValue2Msgs = new HashMap<>(); + protected int topN = 100; + // protected boolean isChanged=false; protected transient ListDataType listDataType; protected transient MapDataType mapDataType; - public TopNState(int topN){ + + public TopNState(int topN) { this(); - this.topN=topN; + this.topN = topN; } - public TopNState(){ - listDataType=new ListDataType(); + + public TopNState() { + listDataType = new ListDataType(); listDataType.setParadigmType(new StringDataType()); - mapDataType=new MapDataType(); + mapDataType = new MapDataType(); mapDataType.setKeyParadigmType(new StringDataType()); mapDataType.setValueParadigmType(new StringDataType()); } + + public static void main(String[] args) { + List orders = new ArrayList<>(); + OrderBy orderBy = new OrderBy("name", false); + orderBy.setDataType(new StringDataType()); + orders.add(orderBy); + orderBy = new OrderBy("age", false); + orderBy.setDataType(new IntDataType()); + orders.add(orderBy); + TopNState topNState = new TopNState(3); + JSONObject msg1 = new JSONObject(); + msg1.put("name", "chris2"); + msg1.put("age", 18); + topNState.addAndSortMsg(msg1, orders); + JSONObject msg2 = new JSONObject(); + msg2.put("name", "chris2"); + msg2.put("age", 19); + topNState.addAndSortMsg(msg2, orders); + JSONObject msg3 = new JSONObject(); + msg3.put("name", "chris1"); + msg3.put("age", 18); + topNState.addAndSortMsg(msg3, orders); + + JSONObject msg4 = new JSONObject(); + msg4.put("name", "chris1"); + msg4.put("age", 19); + topNState.addAndSortMsg(msg4, orders); + System.out.println(topNState.sortValues.size()); + } + /** - * * @return msg order by orderbyFields */ - public List getOrderMsgs(String rowNumerName, Set fieldNames){ - List msgs=new ArrayList<>(); - for(int i=0;i getOrderMsgs(String rowNumerName, Set fieldNames, int startRowNum) { + List msgs = new ArrayList<>(); + for (int i = 0; i < sortValues.size(); i++) { + JSONObject jsonObject = JSONObject.parseObject(orderByValue2Msgs.get(sortValues.get(i))); + JSONObject msg = new JSONObject(jsonObject); + msg.remove(WindowConstants.ORIGIN_QUEUE_ID); + msg.remove(WindowConstants.SHUFFLE_KEY); + msg.remove(WindowConstants.ORIGIN_OFFSET); + msg.remove(WindowConstants.ORIGIN_QUEUE_IS_LONG); + msg.remove(WindowConstants.ORIGIN_MESSAGE_HEADER); + msg.remove(WindowConstants.ORIGIN_MESSAGE_TRACE_ID); + msg.remove(WindowConstants.ORIGIN_SOURCE_NAME); msg.remove(WindowInstance.class.getSimpleName()); msg.remove(AbstractWindow.class.getSimpleName()); msg.remove(MessageHeader.class.getSimpleName()); msg.remove("HIT_WINDOW_INSTANCE_ID"); - msg.remove(ShuffleChannel.SHUFFLE_OFFSET); + msg.remove(WindowConstants.SHUFFLE_OFFSET); msg.remove(AbstractWindow.WINDOW_START); msg.remove(AbstractWindow.WINDOW_END); - msg.put(rowNumerName,i+1); + int index = i + 1; + if (rowNumerName != null) { + msg.put(rowNumerName, index); + } + if (index <= startRowNum) { + continue; + } msgs.add(msg); + } return msgs; } /** * add msg and sort + * * @param message * @param orderByList * @return */ - public boolean addAndSortMsg(JSONObject message,List orderByList){ - String orderByValue=createOrderByValue(message,orderByList); + public boolean addAndSortMsg(JSONObject message, List orderByList) { + if (CollectionUtil.isEmpty(orderByList)) { + /** + * 无排序场景 + */ + String orderByValue = message.toJSONString(); + if (sortValues.size() < topN) { + putMsg(this.orderByValue2Msgs, orderByValue, orderByValue); + this.sortValues.add(orderByValue); + return true; + } else { + return false; + } + } + String orderByValue = createOrderByValue(message, orderByList); - if(sortValues.size()() { @Override public int compare(String o1, String o2) { - return compareElement(o1,o2,orderByList); + return compareElement(o1, o2, orderByList); } }); - while (sortValues.size()>topN){ - String sortValue=sortValues.remove(sortValues.size()-1); - this.orderByValue2Msgs.remove(sortValue); + while (sortValues.size() > topN) { + String sortValue = sortValues.remove(sortValues.size() - 1); + if (!sortValues.contains(sortValue)) { + this.orderByValue2Msgs.remove(orderByValue); + } } return true; } - private int compareElement(String left, String right,List orderByList) { - if(left.equals(right)){ + protected void removeMsg(Map> msgList, String orderByValue) { + List msgs = msgList.get(orderByValue); + if (msgs != null) { + msgs.remove(msgs.size() - 1); + } + if (msgs.size() == 0) { + msgList.remove(orderByValue); + } + } + + protected void putMsg(Map msgList, String orderByValue, String msg) { + msgList.put(orderByValue, msg); + } + + private int compareElement(String left, String right, List orderByList) { + if (left.equals(right)) { return 0; } - String[] leftValues=left.split(SIGN); - String[] rigthValues=right.split(SIGN); - int len=leftValues.length>right.length()?right.length():left.length(); - for(int i=0;i right.length() ? right.length() : left.length(); + for (int i = 0; i < len; i++) { + String leftElement = leftValues[i]; + String rigthElement = rigthValues[i]; + if (leftElement.equals(rigthElement)) { continue; } - OrderBy orderBy=orderByList.get(i); - DataType dataType=orderBy.getDataType(); - boolean isAsc=orderBy.isAsc; - if(DataTypeUtil.isNumber(dataType)){ - Double leftDouble=Double.valueOf(leftElement); - Double rigthDoubel=Double.valueOf(rigthElement); - if(isAsc){ - return leftDouble-rigthDoubel<0?-1:1; - }else { - return rigthDoubel-leftDouble>0?1:-1; + OrderBy orderBy = orderByList.get(i); + DataType dataType = orderBy.getDataType(); + boolean isAsc = orderBy.isAsc; + if (DataTypeUtil.isNumber(dataType)) { + Double leftDouble = Double.valueOf(leftElement); + Double rigthDoubel = Double.valueOf(rigthElement); + if (isAsc) { + return leftDouble - rigthDoubel < 0 ? -1 : 1; + } else { + return rigthDoubel - leftDouble > 0 ? 1 : -1; } - }else { - if(isAsc){ + } else { + if (isAsc) { return leftElement.compareTo(rigthElement); - }else { + } else { return rigthElement.compareTo(leftElement); } } @@ -156,54 +223,24 @@ private int compareElement(String left, String right,List orderByList) } - public static void main(String[] args) { - List orders=new ArrayList<>(); - OrderBy orderBy=new OrderBy("name",false); - orderBy.setDataType(new StringDataType()); - orders.add(orderBy); - orderBy=new OrderBy("age",false); - orderBy.setDataType(new IntDataType()); - orders.add(orderBy); - TopNState topNState=new TopNState(3); - JSONObject msg1=new JSONObject(); - msg1.put("name","chris2"); - msg1.put("age",18); - topNState.addAndSortMsg(msg1,orders); - JSONObject msg2=new JSONObject(); - msg2.put("name","chris2"); - msg2.put("age",19); - topNState.addAndSortMsg(msg2,orders); - JSONObject msg3=new JSONObject(); - msg3.put("name","chris1"); - msg3.put("age",18); - topNState.addAndSortMsg(msg3,orders); - - JSONObject msg4=new JSONObject(); - msg4.put("name","chris1"); - msg4.put("age",19); - topNState.addAndSortMsg(msg4,orders); - System.out.println(topNState.sortValues.size()); - } - - protected String createOrderByValue(JSONObject message, List list) { - StringBuilder stringBuilder=new StringBuilder(); - boolean isFirst=true; - for(OrderBy orderBy:list){ - Object object=message.get(orderBy.getFieldName()); - DataType dataType=orderBy.getDataType(); - if(dataType==null&&object!=null){ - dataType= DataTypeUtil.getDataTypeFromClass(object.getClass()); + StringBuilder stringBuilder = new StringBuilder(); + boolean isFirst = true; + for (OrderBy orderBy : list) { + Object object = message.get(orderBy.getFieldName()); + DataType dataType = orderBy.getDataType(); + if (dataType == null && object != null) { + dataType = DataTypeUtil.getDataTypeFromClass(object.getClass()); orderBy.setDataType(dataType); } - String value=""; - if(object!=null){ - value=dataType.toDataJson(object); + String value = ""; + if (object != null) { + value = dataType.toDataJson(object); } - if(isFirst){ + if (isFirst) { stringBuilder.append(value); - isFirst=false; - }else { + isFirst = false; + } else { stringBuilder.append(SIGN); stringBuilder.append(value); } @@ -235,28 +272,18 @@ public void setOrderByValue2Msgs(Map orderByValue2Msgs) { this.orderByValue2Msgs = orderByValue2Msgs; } - public boolean isChanged() { - return isChanged; - } - - public void setChanged(boolean changed) { - isChanged = changed; - } - @Override public String toJson() { - JSONObject jsonObject=new JSONObject(); - jsonObject.put("sortValues",listDataType.toDataJson(this.sortValues)); - jsonObject.put("topN",this.topN); - jsonObject.put("changed",this.isChanged?1:0); - jsonObject.put("orderByValue2Msgs",mapDataType.toDataJson(this.orderByValue2Msgs)); + JSONObject jsonObject = new JSONObject(); + jsonObject.put("sortValues", listDataType.toDataJson(this.sortValues)); + jsonObject.put("topN", this.topN); + jsonObject.put("orderByValue2Msgs", mapDataType.toDataJson(this.orderByValue2Msgs)); return jsonObject.toJSONString(); } @Override public void toObject(String jsonString) { - JSONObject jsonObject=JSONObject.parseObject(jsonString); - this.topN=jsonObject.getInteger("topN"); - this.isChanged=jsonObject.getInteger("changed")==1?true:false; - this.sortValues=listDataType.getData(jsonObject.getString("sortValues")); - this.orderByValue2Msgs=mapDataType.getData(jsonObject.getString("orderByValue2Msgs")); + JSONObject jsonObject = JSONObject.parseObject(jsonString); + this.topN = jsonObject.getInteger("topN"); + this.sortValues = listDataType.getData(jsonObject.getString("sortValues")); + this.orderByValue2Msgs = mapDataType.getData(jsonObject.getString("orderByValue2Msgs")); } } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/WindowOperator.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/WindowOperator.java index d2806081..1f9c227d 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/WindowOperator.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/WindowOperator.java @@ -25,18 +25,20 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import org.apache.rocketmq.streams.common.channel.sink.AbstractSink; import org.apache.rocketmq.streams.common.channel.split.ISplit; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.MessageOffset; +import org.apache.rocketmq.streams.common.model.NameCreator; +import org.apache.rocketmq.streams.common.topology.IWindow; import org.apache.rocketmq.streams.common.utils.CollectionUtil; import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.common.utils.IdUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.db.driver.batchloader.IRowOperator; import org.apache.rocketmq.streams.db.driver.orm.ORMUtil; -import org.apache.rocketmq.streams.script.operator.impl.AggregationScript; import org.apache.rocketmq.streams.window.debug.DebugWriter; -import org.apache.rocketmq.streams.window.model.FunctionExecutor; import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.operator.AbstractShuffleWindow; import org.apache.rocketmq.streams.window.operator.AbstractWindow; @@ -50,14 +52,19 @@ public class WindowOperator extends AbstractShuffleWindow { private static final String ORDER_BY_SPLIT_NUM = "_order_by_split_num_";//key=_order;queueid,windowinstanceid,partitionNum + protected transient boolean supportQuickStoreModel = false; + protected transient List schema = new ArrayList<>(); + protected transient AtomicInteger shuffleCount = new AtomicInteger(0); + protected transient AtomicInteger fireCountAccumulator = new AtomicInteger(0); + protected transient AtomicLong fireCost = new AtomicLong(0); + protected transient Map shuffleWindowInstanceId2MsgCount = new HashMap<>(); + protected transient int windowvaluecount = 0; + protected transient AtomicLong shuffleCost = new AtomicLong(0); public WindowOperator() { super(); } - protected transient boolean supportQuickStoreModel=false; - protected transient List schema=new ArrayList<>(); - @Deprecated public WindowOperator(String timeFieldName, int windowPeriodMinute) { super(); @@ -79,25 +86,87 @@ public WindowOperator(int sizeInterval, String groupByFieldName, Map windowValues) { + if (windowValues == null || storage == null) { + return; + } + synchronized (storage) { + List storeKeys = new ArrayList<>(); + Map windowValueMap = new HashMap<>(); + for (WindowValue windowValue : windowValues) { + String storeKey = createStoreKey(windowValue.getPartition(), windowValue.getGroupBy(), windowInstance, windowValue.getGroupByFieldName()); + storeKeys.add(storeKey); + windowValueMap.put(storeKey, windowValue); + String storeOrderKey = createStoreKey(windowValue.getPartition(), windowValue.getPartitionNum() + "", windowInstance, windowValue.getGroupByFieldName()); + windowValueMap.put(storeOrderKey, windowValue); + } + Map valueMap = storage.multiGet(WindowValue.class, storeKeys); + if (valueMap == null || valueMap.size() == 0) { + storage.multiPut(windowValueMap); + return; + } + Iterator> it = valueMap.entrySet().iterator(); + + while (it.hasNext()) { + Entry entry = it.next(); + String storeKey = entry.getKey(); + WindowBaseValue localValue = entry.getValue(); + WindowValue windowValue = windowValueMap.get(storeKey); + if (windowValue.getUpdateVersion() <= localValue.getUpdateVersion()) { + windowValueMap.remove(storeKey); + } + } + if (CollectionUtil.isNotEmpty(windowValueMap)) { + storage.multiPut(windowValueMap); + } + } + } - protected transient AtomicLong fireCost=new AtomicLong(0); @Override - public int fireWindowInstance(WindowInstance instance, String queueId, Map queueId2Offset) { + public int fireWindowInstance(WindowInstance instance, String queueId) { List windowValues = new ArrayList<>(); int fireCount = 0; //long startTime = System.currentTimeMillis(); //int sendCost = 0; - // int currentCount = 0; + // int currentCount = 0; //for(String queueId:currentQueueIds){ WindowBaseValueIterator it = storage.loadWindowInstanceSplitData(getOrderBypPrefix(), queueId, instance.createWindowInstanceId(), null, getWindowBaseValueClass()); - if (queueId2Offset != null) { - String offset = queueId2Offset.get(queueId); - if (StringUtil.isNotEmpty(offset)) { - it.setPartitionNum(Long.valueOf(offset)); - } - } +// if (queueId2Offset != null) { +// String offset = queueId2Offset.get(queueId); +// if (StringUtil.isNotEmpty(offset)) { +// it.setPartitionNum(Long.valueOf(offset)); +// } +// } while (it.hasNext()) { WindowBaseValue windowBaseValue = it.next(); if (windowBaseValue == null) { @@ -109,7 +178,7 @@ public int fireWindowInstance(WindowInstance instance, String queueId, Map= windowCache.getBatchSize()) { + if (windowValues.size() >= AbstractSink.DEFAULT_BATCH_SIZE) { long sendFireCost = System.currentTimeMillis(); sendFireMessage(windowValues, queueId); //sendCost += (System.currentTimeMillis() - sendFireCost); @@ -121,29 +190,66 @@ public int fireWindowInstance(WindowInstance instance, String queueId, Map 0) { long sendFireCost = System.currentTimeMillis(); sendFireMessage(windowValues, queueId); - // sendCost += (System.currentTimeMillis() - sendFireCost); + // sendCost += (System.currentTimeMillis() - sendFireCost); fireCount += windowValues.size(); } - clearFire(instance); + clearWindowInstance(instance); this.sqlCache.addCache(new FiredNotifySQLElement(queueId, instance.createWindowInstanceId())); //long cost= this.fireCost.addAndGet(System.currentTimeMillis()-startTime); - // System.out.println("fire cost is "+cost+" "+ DateUtil.getCurrentTimeString()); + // System.out.println("fire cost is "+cost+" "+ DateUtil.getCurrentTimeString()); return fireCount; } - protected transient Map shuffleWindowInstanceId2MsgCount = new HashMap<>(); - protected transient int windowvaluecount = 0; - protected transient AtomicLong shuffleCost=new AtomicLong(0); @Override - public void shuffleCalculate(List messages, WindowInstance instance, String queueId) { - Long startTime=System.currentTimeMillis(); - DebugWriter.getDebugWriter(getConfigureName()).writeShuffleCalcultateReceveMessage(instance, messages, queueId); + public void accumulate(List messages, WindowInstance instance) { + try { + String queueId = instance.getSplitId(); + List groupFieldNames = new ArrayList<>(); + if (supportRollup && this.rollupGroupByFieldNames != null) { + + if (StringUtil.isEmpty(this.groupByFieldName)) { + groupFieldNames = rollupGroupByFieldNames; + } else { + for (String rollupGroupByFieldName : this.rollupGroupByFieldNames) { + groupFieldNames.add(this.groupByFieldName + ";" + rollupGroupByFieldName); + } + + } + + } else { + if (StringUtil.isNotEmpty(this.groupByFieldName)) { + groupFieldNames.add(this.groupByFieldName); + } + } + + if (CollectionUtil.isEmpty(groupFieldNames)) { + this.shuffleCalculate(messages, instance, queueId, "globle_window"); + } else { + for (String groupFieldName : groupFieldNames) { + this.shuffleCalculate(messages, instance, queueId, groupFieldName); + } + } + } catch (Exception e) { + Long time = this.eventTimeManager.getMaxEventTime(instance.getSplitId()); + String maxEventTime = null; + if (time != null) { + maxEventTime = DateUtil.longToString(time); + } + LOGGER.error("[{}][{}] Window_Accumulate_Error_On({})_Window(startTime:{}-endTime:{}-fireTime:{}-maxEventTime:{})_ErrorMsg({})", IdUtil.instanceId(), NameCreator.getFirstPrefix(getName(), IWindow.TYPE), this.getClass().getName(), instance.getStartTime(), instance.getEndTime(), instance.getFireTime(), maxEventTime, e.getMessage(), e); + throw new RuntimeException(e); + } + + } + + protected void shuffleCalculate(List messages, WindowInstance instance, String queueId, String groupByFieldName) { + Long startTime = System.currentTimeMillis(); + DebugWriter.getDebugWriter(getName()).writeShuffleCalcultateReceveMessage(instance, messages, queueId); List sortKeys = new ArrayList<>(); - Map> groupBy = groupByGroupName(messages, sortKeys); + Map> groupBy = groupByGroupName(messages, sortKeys, groupByFieldName); Set groupByKeys = groupBy.keySet(); List storeKeys = new ArrayList<>(); for (String groupByKey : groupByKeys) { - String storeKey = createStoreKey(queueId, groupByKey, instance); + String storeKey = createStoreKey(queueId, groupByKey, instance, groupByFieldName); storeKeys.add(storeKey); } Map allWindowValues = new HashMap<>(); @@ -153,12 +259,11 @@ public void shuffleCalculate(List messages, WindowInstance instance, S for (String groupByKey : sortKeys) { List msgs = groupBy.get(groupByKey); - String storeKey = createStoreKey(queueId, groupByKey, instance); + String storeKey = createStoreKey(queueId, groupByKey, instance, groupByFieldName); WindowValue windowValue = (WindowValue) existWindowValues.get(storeKey); - ; if (windowValue == null) { windowvaluecount++; - windowValue = createWindowValue(queueId, groupByKey, instance); + windowValue = createWindowValue(queueId, groupByKey, instance, groupByFieldName); // windowValue.setOrigOffset(msgs.get(0).getHeader().getOffset()); } allWindowValues.put(storeKey, windowValue); @@ -178,13 +283,13 @@ public void shuffleCalculate(List messages, WindowInstance instance, S shuffleCount.addAndGet(-origValue); shuffleCount.addAndGet(currentValue); } - if (DebugWriter.getDebugWriter(this.getConfigureName()).isOpenDebug()) { - DebugWriter.getDebugWriter(this.getConfigureName()).writeWindowCalculate(this, new ArrayList(allWindowValues.values()), queueId); + if (DebugWriter.getDebugWriter(this.getName()).isOpenDebug()) { + DebugWriter.getDebugWriter(this.getName()).writeWindowCalculate(this, new ArrayList(allWindowValues.values()), queueId); } - saveStorage(allWindowValues, instance, queueId); - long cost= this.shuffleCost.addAndGet(System.currentTimeMillis()-startTime); - // System.out.println("shuffle cost is "+cost+" "+ DateUtil.getCurrentTimeString()); + saveStorage(allWindowValues, instance, queueId, groupByFieldName); + long cost = this.shuffleCost.addAndGet(System.currentTimeMillis() - startTime); + // System.out.println("shuffle cost is "+cost+" "+ DateUtil.getCurrentTimeString()); } private Integer getValue(WindowValue windowValue, String fieldName) { @@ -202,14 +307,14 @@ private Integer getValue(WindowValue windowValue, String fieldName) { } protected void saveStorage(Map allWindowValues, WindowInstance windowInstance, - String queueId) { + String queueId, String groupFieldName) { String windowInstanceId = windowInstance.createWindowInstanceId(); storage.multiPut(allWindowValues, windowInstanceId, queueId, sqlCache); Map partionNumOrders = new HashMap<>();//需要基于key前缀排序partitionnum for (WindowBaseValue windowBaseValue : allWindowValues.values()) { WindowValue windowValue = (WindowValue) windowBaseValue; - String partitionNumKey = createStoreKey(getOrderBypPrefix() + queueId, MapKeyUtil.createKey(getOrderBypFieldName(windowValue), windowValue.getGroupBy()), windowInstance); + String partitionNumKey = createStoreKey(getOrderBypPrefix() + queueId, MapKeyUtil.createKey(getOrderBypFieldName(windowValue), windowValue.getGroupBy()), windowInstance, groupFieldName); partionNumOrders.put(partitionNumKey, windowValue); } storage.getLocalStorage().multiPut(partionNumOrders); @@ -226,14 +331,14 @@ public Class getWindowBaseValueClass() { * @param messages * @return */ - protected Map> groupByGroupName(List messages, List sortKeys) { + protected Map> groupByGroupName(List messages, List sortKeys, String groupByFieldName) { if (messages == null || messages.size() == 0) { return new HashMap<>(); } Map> groupBy = new HashMap<>(); Map minOffsets = new HashMap<>(); for (IMessage message : messages) { - String groupByValue = generateShuffleKey(message); + String groupByValue = generateGroupByValue(message, groupByFieldName); if (StringUtil.isEmpty(groupByValue)) { groupByValue = ""; } @@ -242,7 +347,7 @@ protected Map> groupByGroupName(List messages, if (minOffset == null) { minOffset = message.getHeader().getMessageOffset(); } else { - if (minOffset.greateThan(message.getHeader().getOffset())) { + if (minOffset.greaterThan(message.getHeader().getOffset())) { minOffset = message.getHeader().getMessageOffset(); } @@ -255,7 +360,7 @@ protected Map> groupByGroupName(List messages, if (o1.getValue().equals(o2.getValue())) { return 0; } - boolean success = o1.getValue().greateThan(o2.getValue().getOffsetStr()); + boolean success = o1.getValue().greaterThan(o2.getValue().getOffsetStr()); if (success) { return 1; } else { @@ -290,54 +395,40 @@ protected void calculateWindowValue(WindowValue windowValue, IMessage msg) { * @param instance * @return */ - protected WindowValue createWindowValue(String queueId, String groupBy, WindowInstance instance) { + protected WindowValue createWindowValue(String queueId, String groupBy, WindowInstance instance, String groupByFieldName) { WindowValue windowValue = new WindowValue(); windowValue.setStartTime(instance.getStartTime()); windowValue.setEndTime(instance.getEndTime()); windowValue.setFireTime(instance.getFireTime()); windowValue.setGroupBy(groupBy == null ? "" : groupBy); windowValue.setMsgKey(StringUtil.createMD5Str(MapKeyUtil.createKey(queueId, instance.createWindowInstanceId(), groupBy))); - String shuffleId = shuffleChannel.getChannelQueue(groupBy).getQueueId(); + String shuffleId = shuffleSink.getChannelQueue(groupBy).getQueueId(); windowValue.setPartitionNum(createPartitionNum(windowValue, queueId, instance)); windowValue.setPartition(shuffleId); + windowValue.setGroupByFieldName(groupByFieldName); windowValue.setWindowInstancePartitionId(instance.getWindowInstanceKey()); windowValue.setWindowInstanceId(instance.getWindowInstanceKey()); + windowValue.setIgnoreGroupFieldByRollup2ComputerColumn(createIgnoreFieldByRollup(groupByFieldName)); return windowValue; } - protected long createPartitionNum(WindowValue windowValue, String shuffleId, WindowInstance instance) { - return incrementAndGetSplitNumber(instance, shuffleId); - } - - /** - * 创建存储key - * - * @param groupByKey - * @param windowInstance - * @return - */ - protected static String createStoreKey(String shuffleId, String groupByKey, WindowInstance windowInstance) { - return MapKeyUtil.createKey(shuffleId, windowInstance.createWindowInstanceId(), groupByKey); - } - - /** - * 需要排序的前缀 - * - * @return - */ - protected static String getOrderBypPrefix() { - return ORDER_BY_SPLIT_NUM; + protected Map createIgnoreFieldByRollup(String groupByFieldName) { + if (this.supportRollup && CollectionUtil.isNotEmpty(this.nonStatisticalFieldNames)) { + Map nonStatisticalFieldNamesInWindowValue = new HashMap<>(); + for (String field : this.nonStatisticalFieldNames.keySet()) { + if (groupByFieldName != null && !groupByFieldName.contains(field)) { + nonStatisticalFieldNamesInWindowValue.put(field, this.nonStatisticalFieldNames.get(field)); + } + } + return nonStatisticalFieldNamesInWindowValue.size() == 0 ? null : nonStatisticalFieldNamesInWindowValue; + } + return null; } - /** - * 需要排序的字段值 - * - * @return - */ - protected static String getOrderBypFieldName(WindowValue windowValue) { - return windowValue.getPartitionNum() + ""; + protected long createPartitionNum(WindowValue windowValue, String shuffleId, WindowInstance instance) { + return incrementAndGetSplitNumber(instance, shuffleId); } /** @@ -346,7 +437,7 @@ protected static String getOrderBypFieldName(WindowValue windowValue) { * @param windowInstance */ @Override - public void clearFireWindowInstance(WindowInstance windowInstance) { + public void clearWindowInstance(WindowInstance windowInstance) { String partitionNum = (getOrderBypPrefix() + windowInstance.getSplitId()); boolean canClear = windowInstance.isCanClearResource(); @@ -369,14 +460,14 @@ public void clearFireWindowInstance(WindowInstance windowInstance) { @Override public void clearCache(String queueId) { - getStorage().clearCache(shuffleChannel.getChannelQueue(queueId), getWindowBaseValueClass()); + getStorage().clearCache(shuffleSink.getChannelQueue(queueId), getWindowBaseValueClass()); getStorage().clearCache(getOrderByQueue(queueId, getOrderBypPrefix()), getWindowBaseValueClass()); ShufflePartitionManager.getInstance().clearSplit(queueId); } public ISplit getOrderByQueue(String key, String prefix) { - int index = shuffleChannel.hash(key); - ISplit targetQueue = shuffleChannel.getQueueList().get(index); + int index = shuffleSink.hash(key); + ISplit targetQueue = shuffleSink.getQueueList().get(index); return new ISplit() { @Override public String getQueueId() { @@ -405,43 +496,6 @@ public void toObject(String jsonString) { }; } - public static void compareAndSet(WindowInstance windowInstance, IWindowStorage storage, - List windowValues) { - if (windowValues == null || storage == null) { - return; - } - synchronized (storage) { - List storeKeys = new ArrayList<>(); - Map windowValueMap = new HashMap<>(); - for (WindowValue windowValue : windowValues) { - String storeKey = createStoreKey(windowValue.getPartition(), windowValue.getGroupBy(), windowInstance); - storeKeys.add(storeKey); - windowValueMap.put(storeKey, windowValue); - String storeOrderKey = createStoreKey(windowValue.getPartition(), windowValue.getPartitionNum() + "", windowInstance); - windowValueMap.put(storeOrderKey, windowValue); - } - Map valueMap = storage.multiGet(WindowValue.class, storeKeys); - if (valueMap == null || valueMap.size() == 0) { - storage.multiPut(windowValueMap); - return; - } - Iterator> it = valueMap.entrySet().iterator(); - - while (it.hasNext()) { - Entry entry = it.next(); - String storeKey = entry.getKey(); - WindowBaseValue localValue = entry.getValue(); - WindowValue windowValue = windowValueMap.get(storeKey); - if (windowValue.getUpdateVersion() <= localValue.getUpdateVersion()) { - windowValueMap.remove(storeKey); - } - } - if (CollectionUtil.isNotEmpty(windowValueMap)) { - storage.multiPut(windowValueMap); - } - } - } - public static class WindowRowOperator implements IRowOperator { protected WindowInstance windowInstance; @@ -458,9 +512,9 @@ public WindowRowOperator(WindowInstance windowInstance, String spiltId, Abstract public synchronized void doProcess(Map row) { WindowValue windowValue = ORMUtil.convert(row, WindowValue.class); List keys = new ArrayList<>(); - String storeKey = createStoreKey(spiltId, windowValue.getGroupBy(), windowInstance); + String storeKey = createStoreKey(spiltId, windowValue.getGroupBy(), windowInstance, windowValue.getGroupByFieldName()); keys.add(storeKey); - String storeOrderKey = createStoreKey(getOrderBypPrefix() + windowValue.getPartition(), MapKeyUtil.createKey(getOrderBypFieldName(windowValue), windowValue.getGroupBy()), windowInstance); + String storeOrderKey = createStoreKey(getOrderBypPrefix() + windowValue.getPartition(), MapKeyUtil.createKey(getOrderBypFieldName(windowValue), windowValue.getGroupBy()), windowInstance, windowValue.getGroupByFieldName()); Map valueMap = window.getStorage().getLocalStorage().multiGet(WindowValue.class, keys); if (CollectionUtil.isEmpty(valueMap)) { Map map = new HashMap<>(4); diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/DBOperator.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/DBOperator.java index e6923eea..0cd834f4 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/DBOperator.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/DBOperator.java @@ -25,8 +25,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.Message; import org.apache.rocketmq.streams.common.context.MessageHeader; @@ -35,10 +33,70 @@ import org.apache.rocketmq.streams.window.state.impl.JoinLeftState; import org.apache.rocketmq.streams.window.state.impl.JoinRightState; import org.apache.rocketmq.streams.window.state.impl.JoinState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class DBOperator implements Operator { - private static final Log LOG = LogFactory.getLog(DBOperator.class); + private static final Logger LOGGER = LoggerFactory.getLogger(DBOperator.class); + + /** + * 根据join条件生成消息比对key值 + * + * @param messageBody + * @param joinLabel + * @param leftJoinFieldNames + * @param rightJoinFieldNames + * @return + */ + public static String generateKey(JSONObject messageBody, String joinLabel, List leftJoinFieldNames, + List rightJoinFieldNames) { + StringBuffer buffer = new StringBuffer(); + if ("left".equalsIgnoreCase(joinLabel)) { + for (String field : leftJoinFieldNames) { + String value = messageBody.getString(field); + buffer.append(value).append("_"); + } + } else { + for (String field : rightJoinFieldNames) { + String[] rightFields = field.split("\\."); + if (rightFields.length > 1) { + field = rightFields[1]; + } + String value = messageBody.getString(field); + buffer.append(value).append("_"); + } + } + + return MD5(buffer.toString()); + } + + public static String MD5(String s) { + char hexDigits[] = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'A', 'B', 'C', 'D', 'E', 'F'}; + + try { + byte[] btInput = s.getBytes(); + // 获得MD5摘要算法的 MessageDigest 对象 + MessageDigest mdInst = MessageDigest.getInstance("MD5"); + // 使用指定的字节更新摘要 + mdInst.update(btInput); + // 获得密文 + byte[] md = mdInst.digest(); + // 把密文转换成十六进制的字符串形式 + int j = md.length; + char str[] = new char[j * 2]; + int k = 0; + for (int i = 0; i < j; i++) { + byte byte0 = md[i]; + str[k++] = hexDigits[byte0 >>> 4 & 0xf]; + str[k++] = hexDigits[byte0 & 0xf]; + } + return new String(str); + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException(e.getMessage(), e); + } + } /** * 根据join流对原始数据进行处理并入库 @@ -142,64 +200,6 @@ private List toJavaList(JSONArray jsonArray) { return list; } - /** - * 根据join条件生成消息比对key值 - * - * @param messageBody - * @param joinLabel - * @param leftJoinFieldNames - * @param rightJoinFieldNames - * @return - */ - public static String generateKey(JSONObject messageBody, String joinLabel, List leftJoinFieldNames, - List rightJoinFieldNames) { - StringBuffer buffer = new StringBuffer(); - if ("left".equalsIgnoreCase(joinLabel)) { - for (String field : leftJoinFieldNames) { - String value = messageBody.getString(field); - buffer.append(value).append("_"); - } - } else { - for (String field : rightJoinFieldNames) { - String[] rightFields = field.split("\\."); - if (rightFields.length > 1) { - field = rightFields[1]; - } - String value = messageBody.getString(field); - buffer.append(value).append("_"); - } - } - - return MD5(buffer.toString()); - } - - public static String MD5(String s) { - char hexDigits[] = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'A', 'B', 'C', 'D', 'E', 'F'}; - - try { - byte[] btInput = s.getBytes(); - // 获得MD5摘要算法的 MessageDigest 对象 - MessageDigest mdInst = MessageDigest.getInstance("MD5"); - // 使用指定的字节更新摘要 - mdInst.update(btInput); - // 获得密文 - byte[] md = mdInst.digest(); - // 把密文转换成十六进制的字符串形式 - int j = md.length; - char str[] = new char[j * 2]; - int k = 0; - for (int i = 0; i < j; i++) { - byte byte0 = md[i]; - str[k++] = hexDigits[byte0 >>> 4 & 0xf]; - str[k++] = hexDigits[byte0 & 0xf]; - } - return new String(str); - } catch (Exception e) { - e.printStackTrace(); - throw new RuntimeException(e.getMessage(), e); - } - } - public List> dealJoin(IMessage message) { List> result = new ArrayList<>(); JSONObject messageBody = message.getMessageBody(); @@ -363,7 +363,7 @@ public List getJoinData(String tableName, String messageKey, String try { bodys.add(Message.parseObject(tmp.getMessageBody())); } catch (Exception e) { - LOG.error("json parase error:", e); + LOGGER.error("json parase error:", e); } } @@ -392,8 +392,8 @@ public void cleanMessage(String windowNameSpace, String windowName, int retainWi params.put("startTime", start); params.put("windowNameSpace", windowNameSpace); params.put("windowName", windowName); - if (LOG.isDebugEnabled()) { - LOG.debug("dboperata delete param is " + JSONObject.toJSONString(params)); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("dboperata delete param is " + JSONObject.toJSONString(params)); } List joinLeftStates = ORMUtil.queryForList("select id from join_left_state where window_name_space = #{windowNameSpace} and " + diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/JoinWindow.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/JoinWindow.java index 5eb3b784..feadc9ab 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/JoinWindow.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/JoinWindow.java @@ -20,22 +20,27 @@ import java.util.ArrayList; import java.util.Date; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; import org.apache.rocketmq.streams.common.component.ComponentCreator; 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.interfaces.INeedAalignWaiting; +import org.apache.rocketmq.streams.common.optimization.MessageTrace; import org.apache.rocketmq.streams.common.utils.DateUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.common.utils.TraceUtil; import org.apache.rocketmq.streams.dim.model.AbstractDim; -import org.apache.rocketmq.streams.window.model.WindowCache; +import org.apache.rocketmq.streams.window.WindowConstants; import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.operator.AbstractShuffleWindow; import org.apache.rocketmq.streams.window.state.WindowBaseValue; @@ -43,17 +48,16 @@ import org.apache.rocketmq.streams.window.state.impl.JoinRightState; import org.apache.rocketmq.streams.window.state.impl.JoinState; import org.apache.rocketmq.streams.window.storage.ShufflePartitionManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import static org.apache.rocketmq.streams.window.shuffle.ShuffleChannel.SHUFFLE_OFFSET; - -public class JoinWindow extends AbstractShuffleWindow { +public class JoinWindow extends AbstractShuffleWindow implements INeedAalignWaiting { + public static final String SHUFFLE_OFFSET = "SHUFFLE_OFFSET"; public static final String JOIN_KEY = "JOIN_KEY"; - public static final String LABEL_LEFT = "left"; - public static final String LABEL_RIGHT = "right"; - + private static final Logger LOGGER = LoggerFactory.getLogger(JoinWindow.class); //保存多少个周期的数据。比如window的滚动周期是5分钟,join需要1个小时数据,则retainWindowCount=12 protected int retainWindowCount = 4; protected List leftJoinFieldNames;//join等值条件中,左流的字段列表 @@ -64,36 +68,60 @@ public class JoinWindow extends AbstractShuffleWindow { protected String expression;//条件表达式。在存在非等值比较时使用 protected transient DBOperator joinOperator = new DBOperator(); protected String rightDependentTableName; - // @Override - // protected void addPropertyToMessage(IMessage oriMessage, JSONObject oriJson){ - // oriJson.put("AbstractWindow", this); - // - // } - - @Override - protected int fireWindowInstance(WindowInstance instance, String shuffleId, Map queueId2Offsets) { - clearFire(instance); + protected transient AtomicInteger count = new AtomicInteger(0); + protected Map> alignWaitingMap = new HashMap<>(); + + /** + * 根据join条件生成消息比对key值 + * + * @param messageBody + * @param joinLabel + * @param leftJoinFieldNames + * @param rightJoinFieldNames + * @return + */ + public static String generateKey(JSONObject messageBody, String joinLabel, List leftJoinFieldNames, List rightJoinFieldNames) { + StringBuffer buffer = new StringBuffer(); + if ("left".equalsIgnoreCase(joinLabel)) { + for (String field : leftJoinFieldNames) { + String value = messageBody.getString(field); + buffer.append(value).append("_"); + } + } else { + for (String field : rightJoinFieldNames) { + String[] rightFields = field.split("\\."); + if (rightFields.length > 1) { + field = rightFields[1]; + } + String value = messageBody.getString(field); + buffer.append(value).append("_"); + } + } + + return StringUtil.createMD5Str(buffer.toString()); + } + + @Override protected int fireWindowInstance(WindowInstance instance, String shuffleId) { + clearWindowInstance(instance); return 0; } - @Override - public void clearCache(String queueId) { - getStorage().clearCache(shuffleChannel.getChannelQueue(queueId), getWindowBaseValueClass()); + @Override public void clearCache(String queueId) { + getStorage().clearCache(shuffleSink.getChannelQueue(queueId), getWindowBaseValueClass()); ShufflePartitionManager.getInstance().clearSplit(queueId); } - @Override - public void shuffleCalculate(List messages, WindowInstance instance, String queueId) { - + @Override public void accumulate(List messages, WindowInstance instance) { + String queueId = instance.getSplitId(); for (IMessage msg : messages) { Map joinLeftStates = new HashMap<>(); Map joinRightStates = new HashMap<>(); - MessageHeader header = JSONObject.parseObject(msg.getMessageBody().getString(WindowCache.ORIGIN_MESSAGE_HEADER), MessageHeader.class); + MessageHeader header = JSONObject.parseObject(msg.getMessageBody().getString(WindowConstants.ORIGIN_MESSAGE_HEADER), MessageHeader.class); msg.setHeader(header); String routeLabel = null; - String lable = msg.getHeader().getMsgRouteFromLable(); - if (lable != null) { - if (lable.equals(rightDependentTableName)) { + String label = msg.getHeader().getMsgRouteFromLable(); + if (label != null) { + if (label.equals(rightDependentTableName)) { routeLabel = MessageHeader.JOIN_RIGHT; } else { routeLabel = MessageHeader.JOIN_LEFT; @@ -146,8 +174,7 @@ public void shuffleCalculate(List messages, WindowInstance instance, S } - private Iterator getMessageIterator(String queueId, WindowInstance instance, IMessage msg, - String keyPrefix, Class clazz) { + private Iterator getMessageIterator(String queueId, WindowInstance instance, IMessage msg, String keyPrefix, Class clazz) { List instances = new ArrayList<>(); for (Map.Entry entry : this.windowInstanceMap.entrySet()) { @@ -159,31 +186,26 @@ private Iterator getMessageIterator(String queueId, WindowInsta return new Iterator() { private Iterator iterator = null; - @Override - public boolean hasNext() { + @Override public boolean hasNext() { if (iterator != null && iterator.hasNext()) { return true; } if (windowInstanceIter.hasNext()) { WindowInstance instance = windowInstanceIter.next(); iterator = storage.loadWindowInstanceSplitData(null, null, instance.createWindowInstanceId(), keyPrefix, clazz); - if (iterator != null && iterator.hasNext()) { - return true; - } + return iterator != null && iterator.hasNext(); } return false; } - @Override - public WindowBaseValue next() { + @Override public WindowBaseValue next() { return iterator.next(); } }; } - private Iterator getIterator(String queueId, String keyPrefix, WindowInstance instance, - Class clazz) { + private Iterator getIterator(String queueId, String keyPrefix, WindowInstance instance, Class clazz) { List instances = new ArrayList<>(); for (Map.Entry entry : this.windowInstanceMap.entrySet()) { @@ -193,37 +215,30 @@ private Iterator getIterator(String queueId, String keyPrefix, return new Iterator() { private Iterator iterator = null; - @Override - public boolean hasNext() { + @Override public boolean hasNext() { if (iterator != null && iterator.hasNext()) { return true; } if (windowInstanceIter.hasNext()) { WindowInstance instance = windowInstanceIter.next(); iterator = storage.loadWindowInstanceSplitData(null, queueId, instance.createWindowInstanceId(), keyPrefix, clazz); - if (iterator != null && iterator.hasNext()) { - return true; - } else { - return false; - } + return iterator != null && iterator.hasNext(); } return false; } - @Override - public WindowBaseValue next() { + @Override public WindowBaseValue next() { return iterator.next(); } }; } - public List connectJoin(IMessage message, List> rows, String joinType, - String rightAsName) { + public List connectJoin(IMessage message, List> rows, String joinType, String rightAsName) { List result = new ArrayList<>(); if ("inner".equalsIgnoreCase(joinType)) { - if (rows.size() <= 0) { + if (rows.size() == 0) { return result; } result = connectInnerJoin(message, rows, rightAsName); @@ -242,20 +257,20 @@ private List connectLeftJoin(IMessage message, List 0) { for (Map raw : rows) { - JSONObject object = (JSONObject) messageBody.clone(); + JSONObject object = messageBody.clone(); object.fluentPutAll(addAsName(raw, rightAsName)); object.put(TraceUtil.TRACE_ID_FLAG, traceId + "-" + index); index++; result.add(object); } } else if (LABEL_LEFT.equalsIgnoreCase(routeLabel) && rows.size() <= 0) { - JSONObject object = (JSONObject) messageBody.clone(); + JSONObject object = messageBody.clone(); object.put(TraceUtil.TRACE_ID_FLAG, traceId + "-" + index); result.add(object); } else if (LABEL_RIGHT.equalsIgnoreCase(routeLabel) && rows.size() > 0) { messageBody = addAsName(messageBody, rightAsName); for (Map raw : rows) { - JSONObject object = (JSONObject) messageBody.clone(); + JSONObject object = messageBody.clone(); object.fluentPutAll(raw); object.put(TraceUtil.TRACE_ID_FLAG, traceId + "-" + index); index++; @@ -280,7 +295,7 @@ public List connectInnerJoin(IMessage message, List raw : rows) { - JSONObject object = (JSONObject) messageBody.clone(); + JSONObject object = messageBody.clone(); object.fluentPutAll(addAsName(raw, rightAsName)); object.put(TraceUtil.TRACE_ID_FLAG, traceId + "-" + index); index++; @@ -290,7 +305,7 @@ public List connectInnerJoin(IMessage message, List raw : rows) { - JSONObject object = (JSONObject) messageBody.clone(); + JSONObject object = messageBody.clone(); object.fluentPutAll(raw); object.put(TraceUtil.TRACE_ID_FLAG, traceId + "-" + index); index++; @@ -322,17 +337,15 @@ private JSONObject addAsName(Map raw, String rightAsName) { * @return */ protected String createStoreKey(IMessage message, String routeLabel, WindowInstance windowInstance) { - String shuffleKey = message.getMessageBody().getString(WindowCache.SHUFFLE_KEY); - String orginQueueId = message.getMessageBody().getString(WindowCache.ORIGIN_QUEUE_ID); - String originOffset = message.getMessageBody().getString(WindowCache.ORIGIN_OFFSET); - String storeKey = MapKeyUtil.createKey(windowInstance.createWindowInstanceId(), shuffleKey, routeLabel, orginQueueId, originOffset); - return storeKey; + String shuffleKey = message.getMessageBody().getString(WindowConstants.SHUFFLE_KEY); + String orginQueueId = message.getMessageBody().getString(WindowConstants.ORIGIN_QUEUE_ID); + String originOffset = message.getMessageBody().getString(WindowConstants.ORIGIN_OFFSET); + return MapKeyUtil.createKey(windowInstance.createWindowInstanceId(), shuffleKey, routeLabel, orginQueueId, originOffset); } protected String createStoreKeyPrefix(IMessage message, String routeLabel, WindowInstance windowInstance) { - String shuffleKey = message.getMessageBody().getString(WindowCache.SHUFFLE_KEY); - String storeKey = MapKeyUtil.createKey(shuffleKey, routeLabel); - return storeKey; + String shuffleKey = message.getMessageBody().getString(WindowConstants.SHUFFLE_KEY); + return MapKeyUtil.createKey(shuffleKey, routeLabel); } /** @@ -355,13 +368,13 @@ private JoinState createJoinState(IMessage message, WindowInstance instance, Str offset = header.getOffset(); } - String messageId = this.getNameSpace() + "_" + this.getConfigureName() + "_" + queueId + "_" + offset; + String messageId = this.getNameSpace() + "_" + this.getName() + "_" + queueId + "_" + offset; String messageKey = generateKey(message.getMessageBody(), routeLabel, this.leftJoinFieldNames, this.rightJoinFieldNames); - JSONObject messageBody = (JSONObject) message.getMessageBody().clone(); + JSONObject messageBody = message.getMessageBody().clone(); messageBody.remove("WindowInstance"); messageBody.remove("AbstractWindow"); - messageBody.remove(WindowCache.ORIGIN_MESSAGE_HEADER); + messageBody.remove(WindowConstants.ORIGIN_MESSAGE_HEADER); messageBody.remove("MessageHeader"); JoinState state = null; @@ -373,15 +386,14 @@ private JoinState createJoinState(IMessage message, WindowInstance instance, Str state.setGmtCreate(new Date()); state.setGmtModified(new Date()); - state.setWindowName(this.getConfigureName()); + state.setWindowName(this.getName()); state.setWindowNameSpace(this.getNameSpace()); state.setMessageId(messageId); state.setMessageKey(messageKey); state.setMessageTime(new Date()); state.setMessageBody(messageBody.toJSONString()); state.setMsgKey(createStoreKey(message, routeLabel, instance)); - String shuffleKey = message.getMessageBody().getString(WindowCache.SHUFFLE_KEY); - String shuffleId = shuffleChannel.getChannelQueue(shuffleKey).getQueueId(); + String shuffleId = instance.getSplitId(); state.setPartition(shuffleId); state.setWindowInstanceId(instance.getWindowInstanceKey()); state.setPartitionNum(incrementAndGetSplitNumber(instance, shuffleId)); @@ -390,40 +402,8 @@ private JoinState createJoinState(IMessage message, WindowInstance instance, Str return state; } - /** - * 根据join条件生成消息比对key值 - * - * @param messageBody - * @param joinLabel - * @param leftJoinFieldNames - * @param rightJoinFieldNames - * @return - */ - public static String generateKey(JSONObject messageBody, String joinLabel, List leftJoinFieldNames, - List rightJoinFieldNames) { - StringBuffer buffer = new StringBuffer(); - if ("left".equalsIgnoreCase(joinLabel)) { - for (String field : leftJoinFieldNames) { - String value = messageBody.getString(field); - buffer.append(value).append("_"); - } - } else { - for (String field : rightJoinFieldNames) { - String[] rightFields = field.split("\\."); - if (rightFields.length > 1) { - field = rightFields[1]; - } - String value = messageBody.getString(field); - buffer.append(value).append("_"); - } - } - - return StringUtil.createMD5Str(buffer.toString()); - } - - @Override - public String generateShuffleKey(IMessage message) { - String routeLabel =null; + @Override public String generateShuffleKey(IMessage message) { + String routeLabel = null; String lable = message.getHeader().getMsgRouteFromLable(); if (lable != null) { if (lable.equals(rightDependentTableName)) { @@ -434,12 +414,10 @@ public String generateShuffleKey(IMessage message) { } else { throw new RuntimeException("can not dipatch message, need route label " + toJson()); } - String messageKey = generateKey(message.getMessageBody(), routeLabel, leftJoinFieldNames, rightJoinFieldNames); - return messageKey; + return generateKey(message.getMessageBody(), routeLabel, leftJoinFieldNames, rightJoinFieldNames); } - @Override - public Class getWindowBaseValueClass() { + @Override public Class getWindowBaseValueClass() { return JoinState.class; } @@ -448,8 +426,7 @@ public Class getWindowBaseValueClass() { * * @param windowInstance */ - @Override - public synchronized void clearFireWindowInstance(WindowInstance windowInstance) { + @Override public synchronized void clearWindowInstance(WindowInstance windowInstance) { List removeInstances = new ArrayList<>(); Date clearTime = DateUtil.addSecond(DateUtil.parse(windowInstance.getStartTime()), -sizeInterval * (retainWindowCount - 1) * 60); @@ -485,13 +462,11 @@ private List> converToMapFromList(List rows List> joinMessages = new ArrayList<>(); for (WindowBaseValue value : rows) { JSONObject obj = Message.parseObject(((JoinState) value).getMessageBody()); - joinMessages.add((Map) obj); + joinMessages.add(obj); } return joinMessages; } - protected transient AtomicInteger count = new AtomicInteger(0); - /** * 把触发的数据,发送到下一个节点 * @@ -507,8 +482,9 @@ protected void sendMessage(JSONObject message, boolean needFlush) { AbstractContext context = new Context(nextMessage); boolean isWindowTest = ComponentCreator.getPropertyBooleanValue("window.fire.isTest"); if (isWindowTest) { - System.out.println(getConfigureName() + " result send count is " + count.incrementAndGet()); + LOGGER.info("[{}] result send count is {}", getName(), count.incrementAndGet()); } + MessageTrace.joinMessage(nextMessage);//关联全局监控器 this.getFireReceiver().doMessage(nextMessage, context); } @@ -537,20 +513,19 @@ protected void cleanMessage(Message msg) { JSONObject messageBody = msg.getMessageBody(); messageBody.remove("WindowInstance"); messageBody.remove("AbstractWindow"); - messageBody.remove(WindowCache.ORIGIN_MESSAGE_HEADER); + messageBody.remove(WindowConstants.ORIGIN_MESSAGE_HEADER); messageBody.remove("MessageHeader"); messageBody.remove(SHUFFLE_OFFSET); messageBody.remove("HIT_WINDOW_INSTANCE_ID"); messageBody.remove(TraceUtil.TRACE_ID_FLAG); - messageBody.remove(WindowCache.ORIGIN_QUEUE_ID); - messageBody.remove(WindowCache.SHUFFLE_KEY); - messageBody.remove(WindowCache.ORIGIN_MESSAGE_TRACE_ID); - messageBody.remove(WindowCache.ORIGIN_OFFSET); - messageBody.remove(WindowCache.ORIGIN_QUEUE_IS_LONG); + messageBody.remove(WindowConstants.ORIGIN_QUEUE_ID); + messageBody.remove(WindowConstants.SHUFFLE_KEY); + messageBody.remove(WindowConstants.ORIGIN_MESSAGE_TRACE_ID); + messageBody.remove(WindowConstants.ORIGIN_OFFSET); + messageBody.remove(WindowConstants.ORIGIN_QUEUE_IS_LONG); } - @Override - public void removeInstanceFromMap(WindowInstance windowInstance) { + @Override public void removeInstanceFromMap(WindowInstance windowInstance) { String begin = DateUtil.getBeforeMinutesTime(windowInstance.getStartTime(), (this.retainWindowCount - 1) * this.sizeInterval); String deletePrefix = MapKeyUtil.createKey(windowInstance.getWindowNameSpace(), windowInstance.getWindowName(), begin); for (Map.Entry tmp : windowInstanceMap.entrySet()) { @@ -560,8 +535,7 @@ public void removeInstanceFromMap(WindowInstance windowInstance) { } } - @Override - protected Long queryWindowInstanceMaxSplitNum(WindowInstance instance) { + @Override protected Long queryWindowInstanceMaxSplitNum(WindowInstance instance) { Long leftMaxSplitNum = storage.getMaxSplitNum(instance, JoinLeftState.class); Long rigthMaxSplitNum = storage.getMaxSplitNum(instance, JoinRightState.class); if (leftMaxSplitNum == null) { @@ -579,9 +553,8 @@ protected Long queryWindowInstanceMaxSplitNum(WindowInstance instance) { return null; } - @Override - public boolean supportBatchMsgFinish() { - return false; + @Override public boolean supportBatchMsgFinish() { + return true; } public int getRetainWindowCount() { @@ -640,4 +613,31 @@ public void setRightDependentTableName(String rightDependentTableName) { this.rightDependentTableName = rightDependentTableName; } + @Override public synchronized boolean alignWaiting(IMessage oriMessage) { + String routeLabel = null; + String label = oriMessage.getHeader().getMsgRouteFromLable(); + if (label != null) { + if (label.equals(rightDependentTableName)) { + routeLabel = MessageHeader.JOIN_RIGHT; + } else { + routeLabel = MessageHeader.JOIN_LEFT; + } + oriMessage.getHeader().setMsgRouteFromLable(routeLabel); + } + BatchFinishMessage batchFinishMessage = (BatchFinishMessage) oriMessage.getSystemMessage(); + Set set = alignWaitingMap.get(batchFinishMessage.getMessageId()); + if (set == null) { + set = new HashSet<>(); + alignWaitingMap.put(batchFinishMessage.getMessageId(), set); + } + set.add(routeLabel); + if (set.size() == 2) { + if (set.contains(MessageHeader.JOIN_RIGHT) && set.contains(MessageHeader.JOIN_LEFT)) { + alignWaitingMap.remove(batchFinishMessage.getMessageId()); + return true; + } + } + + return false; + } } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/AbstractSystemChannel.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/AbstractSystemChannel.java deleted file mode 100644 index d0e499f2..00000000 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/AbstractSystemChannel.java +++ /dev/null @@ -1,358 +0,0 @@ -/* - * 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.window.shuffle; - -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder; -import org.apache.rocketmq.streams.common.channel.builder.IShuffleChannelBuilder; -import org.apache.rocketmq.streams.common.channel.impl.memory.MemoryCache; -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.AbstractSupportShuffleSink; -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.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configurable.IConfigurableIdentification; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; -import org.apache.rocketmq.streams.common.interfaces.ISystemMessageProcessor; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; -import org.apache.rocketmq.streams.common.utils.ReflectUtil; -import org.apache.rocketmq.streams.common.utils.StringUtil; -import org.apache.rocketmq.streams.serviceloader.ServiceLoaderComponent; -import org.apache.rocketmq.streams.window.operator.AbstractWindow; - -public abstract class AbstractSystemChannel implements IConfigurableIdentification, ISystemMessageProcessor, IStreamOperator { - - protected static final Log LOG = LogFactory.getLog(AbstractSystemChannel.class); - - protected static final String CHANNEL_PROPERTY_KEY_PREFIX = "CHANNEL_PROPERTY_KEY_PREFIX"; - protected static final String CHANNEL_TYPE = "CHANNEL_TYPE"; - - protected ISource consumer; - protected AbstractSupportShuffleSink producer; - protected Map channelConfig = new HashMap<>(); - protected volatile boolean hasCreateShuffleChannel = false; - - protected transient AtomicBoolean hasStart = new AtomicBoolean(false); - public void startChannel() { - if (consumer == null) { - return; - } - if (hasStart.compareAndSet(false, true)) { - consumer.start(this); - } - - } - - - /** - * init shuffle channel - */ - public void init(AbstractWindow window) { - this.consumer = createSource(window.getNameSpace(), window.getConfigureName()); - this.producer = createSink(window.getNameSpace(), window.getConfigureName()); - if (this.consumer == null || this.producer == null) { - autoCreateShuffleChannel(window.getFireReceiver().getPipeline()); - } - if (this.consumer == null) { - return; - } - if (this.consumer instanceof AbstractSource) { - ((AbstractSource) this.consumer).setJsonData(true); - } - if(producer!=null){ - this.producer.init(); - } - - } - /** - * 如果用户未配置shuffle channel,根据pipeline数据源动态创建 - * - * @param pipeline - */ - public void autoCreateShuffleChannel(ChainPipeline pipeline) { - if (!hasCreateShuffleChannel) { - synchronized (this) { - if (!hasCreateShuffleChannel) { - ISource pipelineSource = pipeline.getSource(); - ServiceLoaderComponent serviceLoaderComponent = ComponentCreator.getComponent(IChannelBuilder.class.getName(), ServiceLoaderComponent.class); - - IChannelBuilder builder = (IChannelBuilder) serviceLoaderComponent.loadService(pipelineSource.getClass().getSimpleName()); - if (builder == null) { - throw new RuntimeException("can not create shuffle channel, not find channel builder " + pipelineSource.toJson()); - } - if (!(builder instanceof IShuffleChannelBuilder)) { - throw new RuntimeException("can not create shuffle channel, builder not impl IShuffleChannelBuilder " + pipelineSource.toJson()); - } - IShuffleChannelBuilder shuffleChannelBuilder = (IShuffleChannelBuilder) builder; - ISink sink = shuffleChannelBuilder.createBySource(pipelineSource); - sink.init(); - if (!(sink instanceof MemoryChannel) && !(sink instanceof AbstractSupportShuffleSink)) { - throw new RuntimeException("can not create shuffle channel, sink not extends AbstractSupportShuffleSink " + pipelineSource.toJson()); - } - ISource source = null; - if (sink instanceof MemoryChannel) { - MemoryCache memoryCache = new MemoryCache(); - memoryCache.setNameSpace(createShuffleChannelNameSpace(pipeline)); - memoryCache.setConfigureName(createShuffleChannelName(pipeline)); - - sink = new MemorySink(); - source = new MemorySource(); - ((MemorySink) sink).setMemoryCache(memoryCache); - ((MemorySource) source).setMemoryCache(memoryCache); - memoryCache.init(); - } - - Properties properties = new Properties(); - putDynamicPropertyValue(new HashSet<>(), properties); - - AbstractSupportShuffleSink shuffleSink = (AbstractSupportShuffleSink) sink; - //todo 为什么这里的分区数量要和源头topic的分区数量一直? - shuffleSink.setSplitNum(getShuffleSplitCount(shuffleSink)); - shuffleSink.setNameSpace(createShuffleChannelNameSpace(pipeline)); - shuffleSink.setConfigureName(createShuffleChannelName(pipeline)); - String topicFiledName = shuffleSink.getShuffleTopicFieldName(); - String shuffleTopic = null; - //内存模式,是无topic的 - if (StringUtil.isNotEmpty(topicFiledName)) { - String topic = ReflectUtil.getDeclaredField(shuffleSink, topicFiledName); - shuffleTopic = createShuffleTopic(topic, pipeline); - ReflectUtil.setBeanFieldValue(shuffleSink, topicFiledName, shuffleTopic); - } - - //修改和window有关的属性,如groupname,tags - List fields = ReflectUtil.getDeclaredFieldsContainsParentClass(sink.getClass()); - for (Field field : fields) { - String fieldName = field.getName(); - String value = properties.getProperty(fieldName); - if (StringUtil.isNotEmpty(value)) { - ReflectUtil.setBeanFieldValue(sink, fieldName, value); - } - } - shuffleSink.setHasInit(false); - shuffleSink.init();//在这里完成shuffle channel的创建 - if (source == null) { - source = shuffleChannelBuilder.copy(pipelineSource); - } - - //修改和window有关的属性,如groupname,tags - - fields = ReflectUtil.getDeclaredFieldsContainsParentClass(source.getClass()); - for (Field field : fields) { - String fieldName = field.getName(); - String value = properties.getProperty(fieldName); - if (StringUtil.isNotEmpty(value)) { - ReflectUtil.setBeanFieldValue(source, fieldName, value); - } - } - - source.setNameSpace(sink.getNameSpace()); - source.setConfigureName(sink.getConfigureName()); - //修改主题 - if (shuffleTopic != null && topicFiledName != null) { - ReflectUtil.setBeanFieldValue(source, topicFiledName, shuffleTopic); - } - if (source instanceof AbstractSource) { - AbstractSource abstractSource = (AbstractSource) source; - abstractSource.setHasInit(false); - } - source.init(); - - this.producer = shuffleSink; - this.consumer = source; - } - } - } - } - - /** - * 根据数据源的名字,给shufflechannel取主题名 - * - * @param topic - * @param message - * @return - */ - protected abstract String createShuffleTopic(String topic, ChainPipeline message); - - protected abstract int getShuffleSplitCount(AbstractSupportShuffleSink shuffleSink); - - /** - * shuffle channel的名字 - * - * @param message - * @return - */ - protected abstract String createShuffleChannelName(ChainPipeline message); - - /** - * shuffle channel的名字 - * - * @param message - * @return - */ - protected abstract String createShuffleChannelNameSpace(ChainPipeline message); - - protected Map getChannelConfig() { - return channelConfig; - } - - protected abstract String getDynamicPropertyValue(); - - /** - * 创建channel,根据配置文件配置channel的连接信息 - * - * @return - */ - protected ISource createSource(String namespace, String name) { - IChannelBuilder builder = createBuilder(); - if (builder == null) { - return null; - } - Properties properties = createChannelProperties(namespace); - ISource source = builder.createSource(namespace, name, properties, null); - if (source instanceof MemorySource) { - MemorySource memorySource = (MemorySource) source; - MemoryCache memoryCache = new MemoryCache(); - memorySource.setMemoryCache(memoryCache); - memoryCache.init(); - } - source.init(); - return source; - } - - /** - * 创建channel,根据配置文件配置channel的连接信息 - * - * @return - */ - protected AbstractSupportShuffleSink createSink(String namespace, String name) { - - IChannelBuilder builder = createBuilder(); - if (builder == null) { - return null; - } - Properties properties = createChannelProperties(namespace); - - ISink sink = builder.createSink(namespace, name, properties, null); - if (!(sink instanceof AbstractSupportShuffleSink)) { - throw new RuntimeException("can not support shuffle " + sink.toJson()); - } - if (sink instanceof MemorySink) { - MemorySink memorySink = (MemorySink) sink; - if (!(this.consumer instanceof MemorySource)) { - throw new RuntimeException("shuffle cosumer need memory, real is " + this.consumer); - } - MemorySource memorySource = (MemorySource) this.consumer; - MemoryCache memoryCache = memorySource.getMemoryCache(); - memorySink.setMemoryCache(memoryCache); - } - - sink.init(); - return (AbstractSupportShuffleSink) sink; - } - - /** - * create channel builder - * - * @return - */ - protected IChannelBuilder createBuilder() { - String type = ComponentCreator.getProperties().getProperty(getChannelConfig().get(CHANNEL_TYPE)); - if (StringUtil.isEmpty(type)) { - return null; - } - ServiceLoaderComponent serviceLoaderComponent = ComponentCreator.getComponent(IChannelBuilder.class.getName(), ServiceLoaderComponent.class); - return (IChannelBuilder) serviceLoaderComponent.loadService(type); - } - - /** - * 根据属性文件配置 - * - * @return 资源文件 - */ - protected Properties createChannelProperties(String namespace) { - Properties properties = new Properties(); - for (Map.Entry entry : ComponentCreator.getProperties().entrySet()) { - String key = (String) entry.getKey(); - String value = (String) entry.getValue(); - if (key.startsWith(getChannelConfig().get(CHANNEL_PROPERTY_KEY_PREFIX))) { - String channelKey = key.replace(getChannelConfig().get(CHANNEL_PROPERTY_KEY_PREFIX), ""); - if (channelKey.startsWith(namespace)) {//支持基于namespace 做shuffle window共享 - channelKey = channelKey.replace(namespace, ""); - properties.put(channelKey, value); - } else { - if (!properties.containsKey(channelKey)) { - properties.put(channelKey, value); - } - } - } - } - Set multiPropertySet = new HashSet<>(); - String dynamicProperty = properties.getProperty("dynamic.property"); - if (dynamicProperty != null) { - - String dynamicPropertyValue = getDynamicPropertyValue(); - String[] mutilPropertys = dynamicProperty.split(","); - - for (String properyKey : mutilPropertys) { - properties.put(properyKey, dynamicPropertyValue); - multiPropertySet.add(properyKey); - } - - } - putDynamicPropertyValue(multiPropertySet, properties); - return properties; - } - - /** - * 如果需要额外的动态属性,可以在子类添加 - * - * @param dynamicPropertySet 资源集 - */ - protected void putDynamicPropertyValue(Set dynamicPropertySet, Properties properties) { - - } - - public ISource getConsumer() { - return consumer; - } - - public ISink getProducer() { - return producer; - } - - public void sendMessage(IMessage message) { - List msgs = new ArrayList<>(); - msgs.add(message); - producer.batchSave(msgs); - producer.flush(); - } - -} diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleCache.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleCache.java index 054a418d..248dc0d4 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleCache.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleCache.java @@ -26,19 +26,27 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.rocketmq.streams.common.channel.sink.AbstractSink; import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.context.MessageHeader; import org.apache.rocketmq.streams.db.driver.orm.ORMUtil; +import org.apache.rocketmq.streams.script.function.impl.window.WindowFunction; +import org.apache.rocketmq.streams.window.WindowConstants; import org.apache.rocketmq.streams.window.debug.DebugWriter; -import org.apache.rocketmq.streams.window.model.WindowCache; import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.offset.WindowMaxValue; import org.apache.rocketmq.streams.window.operator.AbstractShuffleWindow; import org.apache.rocketmq.streams.window.sqlcache.impl.SplitSQLElement; +import org.apache.rocketmq.streams.window.util.ShuffleUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * save receiver messages into cachefilter when checkpoint/autoflush/flush, process cachefilter message */ public class ShuffleCache extends AbstractSink { + private static final Logger LOGGER = LoggerFactory.getLogger(ShuffleCache.class); + protected AbstractShuffleWindow window; + protected transient AtomicLong count = new AtomicLong(0); public ShuffleCache(AbstractShuffleWindow window) { this.window = window; @@ -46,6 +54,7 @@ public ShuffleCache(AbstractShuffleWindow window) { @Override protected boolean batchInsert(List messageList) { + LOGGER.debug("[{}] the count is {}", getName(), count.addAndGet(messageList.size())); Map, List> instance2Messages = new HashMap<>(); Map windowInstanceMap = new HashMap<>(); groupByWindowInstanceAndQueueId(messageList, instance2Messages, windowInstanceMap); @@ -56,12 +65,20 @@ protected boolean batchInsert(List messageList) { String windowInstanceId = queueIdAndInstanceKey.getRight(); List messages = instance2Messages.get(queueIdAndInstanceKey); WindowInstance windowInstance = windowInstanceMap.get(windowInstanceId); - DebugWriter.getDebugWriter(window.getConfigureName()).writeShuffleReceive(window, messages, windowInstance); - window.shuffleCalculate(messages, windowInstance, queueId); + DebugWriter.getDebugWriter(window.getName()).writeShuffleReceive(window, messages, windowInstance); + + /** + * TUMBLE_START(ts, INTERVAL '1' MINUTE) 语句使用 + */ + for (IMessage message : messages) { + message.getMessageBody().put(WindowFunction.WINDOW_START, windowInstance.getStartTime()); + message.getMessageBody().put(WindowFunction.WINDOW_END, windowInstance.getEndTime()); + } + window.accumulate(messages, windowInstance); saveSplitProgress(queueId, messages); - window.saveMsgContext(queueId,windowInstance,messages); + window.saveMsgContext(queueId, windowInstance, messages); } - return true; + return true; } /** @@ -74,17 +91,16 @@ protected void saveSplitProgress(String queueId, List messages) { Map queueId2OrigOffset = new HashMap<>(); Boolean isLong = false; for (IMessage message : messages) { - isLong = message.getMessageBody().getBoolean(WindowCache.ORIGIN_QUEUE_IS_LONG); - String oriQueueId = message.getMessageBody().getString(WindowCache.ORIGIN_QUEUE_ID); - String oriOffset = message.getMessageBody().getString(WindowCache.ORIGIN_OFFSET); + isLong = message.getMessageBody().getBoolean(WindowConstants.ORIGIN_QUEUE_IS_LONG); + String oriQueueId = message.getMessageBody().getString(WindowConstants.ORIGIN_QUEUE_ID); + String oriOffset = message.getMessageBody().getString(WindowConstants.ORIGIN_OFFSET); queueId2OrigOffset.put(oriQueueId, oriOffset); } - Map windowMaxValueMap = window.getWindowMaxValueManager().saveMaxOffset(isLong, window.getConfigureName(), queueId, queueId2OrigOffset); - window.getSqlCache().addCache(new SplitSQLElement(queueId, ORMUtil.createBatchReplacetSQL(new ArrayList<>(windowMaxValueMap.values())))); + Map windowMaxValueMap = window.getWindowMaxValueManager().saveMaxOffset(isLong, window.getName(), queueId, queueId2OrigOffset); + window.getSqlCache().addCache(new SplitSQLElement(queueId, ORMUtil.createBatchReplaceSQL(new ArrayList<>(windowMaxValueMap.values())))); } - /** * 根据message,把message分组到不同的group,分别处理 * @@ -97,9 +113,11 @@ protected void groupByWindowInstanceAndQueueId(List messageList, for (IMessage message : messageList) { //the queueId will be replace below, so get first here! String queueId = message.getHeader().getQueueId(); - String oriQueueId = message.getMessageBody().getString(WindowCache.ORIGIN_QUEUE_ID); - String oriOffset = message.getMessageBody().getString(WindowCache.ORIGIN_OFFSET); - Boolean isLong = message.getMessageBody().getBoolean(WindowCache.ORIGIN_QUEUE_IS_LONG); + String oriQueueId = message.getMessageBody().getString(WindowConstants.ORIGIN_QUEUE_ID); + String oriOffset = message.getMessageBody().getString(WindowConstants.ORIGIN_OFFSET); + Boolean isLong = message.getMessageBody().getBoolean(WindowConstants.ORIGIN_QUEUE_IS_LONG); + MessageHeader header = ShuffleUtil.getMessageHeader(message.getMessageBody()); + message.setHeader(header); message.getHeader().setQueueId(oriQueueId); message.getHeader().setOffset(oriOffset); message.getHeader().setOffsetIsLong(isLong); diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleChannel.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleChannel.java deleted file mode 100644 index 9280a2e4..00000000 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleChannel.java +++ /dev/null @@ -1,536 +0,0 @@ -/* - * 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.window.shuffle; - -import com.alibaba.fastjson.JSONArray; -import com.alibaba.fastjson.JSONObject; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.lang3.tuple.MutablePair; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; -import org.apache.rocketmq.streams.common.channel.sink.AbstractSupportShuffleSink; -import org.apache.rocketmq.streams.common.channel.source.AbstractSource; -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.channel.split.ISplit; -import org.apache.rocketmq.streams.common.checkpoint.CheckPointManager; -import org.apache.rocketmq.streams.common.checkpoint.CheckPointMessage; -import org.apache.rocketmq.streams.common.checkpoint.CheckPointState; -import org.apache.rocketmq.streams.common.checkpoint.SourceState; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configurable.IConfigurableIdentification; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; -import org.apache.rocketmq.streams.common.context.AbstractContext; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.context.Message; -import org.apache.rocketmq.streams.common.context.MessageOffset; -import org.apache.rocketmq.streams.common.interfaces.ISystemMessage; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; -import org.apache.rocketmq.streams.common.topology.model.Pipeline; -import org.apache.rocketmq.streams.common.utils.CollectionUtil; -import org.apache.rocketmq.streams.common.utils.CompressUtil; -import org.apache.rocketmq.streams.common.utils.DateUtil; -import org.apache.rocketmq.streams.common.utils.MapKeyUtil; -import org.apache.rocketmq.streams.common.utils.StringUtil; -import org.apache.rocketmq.streams.common.utils.TraceUtil; -import org.apache.rocketmq.streams.db.driver.orm.ORMUtil; -import org.apache.rocketmq.streams.window.debug.DebugWriter; -import org.apache.rocketmq.streams.window.minibatch.MiniBatchMsgCache; -import org.apache.rocketmq.streams.window.model.WindowCache; -import org.apache.rocketmq.streams.window.model.WindowInstance; -import org.apache.rocketmq.streams.window.operator.AbstractShuffleWindow; -import org.apache.rocketmq.streams.window.operator.AbstractWindow; -import org.apache.rocketmq.streams.window.operator.impl.WindowOperator.WindowRowOperator; -import org.apache.rocketmq.streams.window.sqlcache.impl.SQLElement; -import org.apache.rocketmq.streams.window.storage.ShufflePartitionManager; - -import static org.apache.rocketmq.streams.window.model.WindowCache.ORIGIN_MESSAGE_TRACE_ID; - -/** - * 负责处理分片 - */ -public class ShuffleChannel extends AbstractSystemChannel { - - protected static final Log LOG = LogFactory.getLog(ShuffleChannel.class); - - protected static final String SHUFFLE_QUEUE_ID = "SHUFFLE_QUEUE_ID"; - public static final String SHUFFLE_OFFSET = "SHUFFLE_OFFSET"; - protected static final String SHUFFLE_MESSAGES = "SHUFFLE_MESSAGES"; - /** - * 消息所属的window - */ - protected String MSG_OWNER = "MSG_OWNER"; - - private static final String SHUFFLE_TRACE_ID = "SHUFFLE_TRACE_ID"; - - protected ShuffleCache shuffleCache; - - protected Map> queueMap = new ConcurrentHashMap<>(); - /** - * 所有的分片 - */ - protected List> queueList; - - protected AbstractShuffleWindow window; - /** - * 当前管理的分片 - */ - private Set currentQueueIds; - - protected transient boolean isWindowTest = false; - - /** - * 每个分片,已经确定处理的最大offset - */ - protected transient Map split2MaxOffsets = new HashMap<>(); - - public ShuffleChannel(AbstractShuffleWindow window) { - this.window = window; - channelConfig = new HashMap<>(); - channelConfig.put(CHANNEL_PROPERTY_KEY_PREFIX, ConfigureFileKey.WINDOW_SHUFFLE_CHANNEL_PROPERTY_PREFIX); - channelConfig.put(CHANNEL_TYPE, ConfigureFileKey.WINDOW_SHUFFLE_CHANNEL_TYPE); - - this.shuffleCache = new ShuffleCache(window); - this.shuffleCache.init(); - this.shuffleCache.openAutoFlush(); - - } - - - /** - * init shuffle channel - */ - public void init() { - init(this.window); - if (producer != null && (queueList == null || queueList.size() == 0)) { - queueList = producer.getSplitList(); - Map> tmp = new ConcurrentHashMap<>(); - for (ISplit queue : queueList) { - tmp.put(queue.getQueueId(), queue); - } - this.queueMap = tmp; - } - isWindowTest = ComponentCreator.getPropertyBooleanValue("window.fire.isTest"); - } - - /** - * 接收到分片信息,如果是系统消息,做缓存刷新,否则把消息放入缓存,同时计算存储的有效性 - * - * @param oriMessage - * @param context - * @return - */ - - protected transient AtomicLong COUNT = new AtomicLong(0); - - @Override - public Object doMessage(IMessage oriMessage, AbstractContext context) { - if (oriMessage.getHeader().isSystemMessage()) { - doSystemMessage(oriMessage, context); - return null; - - } - if (oriMessage.getMessageBody().getBooleanValue(WindowCache.IS_COMPRESSION_MSG)) { - byte[] bytes = oriMessage.getMessageBody().getBytes(WindowCache.COMPRESSION_MSG_DATA); - String msgStr = CompressUtil.unGzip(bytes); - oriMessage.setMessageBody(JSONObject.parseObject(msgStr)); - } - /** - * 过滤不是这个window的消息,一个shuffle通道,可能多个window共享,这里过滤掉非本window的消息 - */ - boolean isFilter = filterNotOwnerMessage(oriMessage); - if (isFilter) { - return null; - } - String queueId = oriMessage.getHeader().getQueueId(); - JSONArray messages = oriMessage.getMessageBody().getJSONArray(SHUFFLE_MESSAGES); - if (messages == null) { - return null; - } - - String traceId = oriMessage.getMessageBody().getString(SHUFFLE_TRACE_ID); - if (!StringUtil.isEmpty(traceId)) { - TraceUtil.debug(traceId, "shuffle message in", "received message size:" + messages.size()); - } - for (Object obj : messages) { - IMessage message = new Message((JSONObject) obj); - message.getHeader().setQueueId(queueId); - message.getMessageBody().put(SHUFFLE_OFFSET, oriMessage.getHeader().getOffset()); - window.updateMaxEventTime(message); - if (isRepeateMessage(message, queueId)) { - continue; - } - List windowInstances = window.queryOrCreateWindowInstance(message, queueId); - if (windowInstances == null || windowInstances.size() == 0) { - LOG.warn("the message is out of window instance, the message is discard"); - continue; - } - for (WindowInstance windowInstance : windowInstances) { - String windowInstanceId = windowInstance.createWindowInstanceId(); - //new instance, not need load data from remote - if (windowInstance.isNewWindowInstance()) { - window.getSqlCache().addCache(new SQLElement(windowInstance.getSplitId(), windowInstanceId, ORMUtil.createBatchReplacetSQL(windowInstance))); - windowInstance.setNewWindowInstance(false); - ShufflePartitionManager.getInstance().setWindowInstanceFinished(windowInstance.createWindowInstanceId()); - } - } - - message.getMessageBody().put(WindowInstance.class.getSimpleName(), windowInstances); - message.getMessageBody().put(AbstractWindow.class.getSimpleName(), window); - - if (DebugWriter.getDebugWriter(window.getConfigureName()).isOpenDebug()) { - List msgs = new ArrayList<>(); - msgs.add(message); - DebugWriter.getDebugWriter(window.getConfigureName()).writeShuffleReceiveBeforeCache(window, msgs, queueId); - } - - for (WindowInstance windowInstance : windowInstances) { - window.getWindowFireSource().updateWindowInstanceLastUpdateTime(windowInstance); - } - shuffleCache.batchAdd(message); - } - if (isWindowTest) { - long count = COUNT.addAndGet(messages.size()); - System.out.println(window.getConfigureName() + " receive shuffle msg count is " + count); - } - - return null; - } - - @Override - public void addNewSplit(IMessage message, AbstractContext context, NewSplitMessage newSplitMessage) { - this.currentQueueIds = newSplitMessage.getCurrentSplitIds(); - loadSplitProgress(newSplitMessage); - - List allWindowInstances = WindowInstance.queryAllWindowInstance(DateUtil.getCurrentTimeString(), window, newSplitMessage.getSplitIds()); - if (CollectionUtil.isNotEmpty(allWindowInstances)) { - Map> queueId2WindowInstances = new HashMap<>(); - for (WindowInstance windowInstance : allWindowInstances) { - windowInstance.setNewWindowInstance(false); - window.registerWindowInstance(windowInstance); - window.getWindowFireSource().registFireWindowInstanceIfNotExist(windowInstance, window); - String queueId = windowInstance.getSplitId(); - window.getStorage().loadSplitData2Local(queueId, windowInstance.createWindowInstanceId(), window.getWindowBaseValueClass(), new WindowRowOperator(windowInstance, queueId, window)); - window.initWindowInstanceMaxSplitNum(windowInstance); - } - - } else { - for (String queueId : newSplitMessage.getSplitIds()) { - ShufflePartitionManager.getInstance().setSplitFinished(queueId); - } - } - window.getFireReceiver().doMessage(message, context); - } - - /** - * load ori split consume offset - * - * @param newSplitMessage - */ - protected void loadSplitProgress(NewSplitMessage newSplitMessage) { - for (String queueId : newSplitMessage.getSplitIds()) { - Map result = window.getWindowMaxValueManager().loadOffsets(window.getConfigureName(), queueId); - if (result != null) { - this.split2MaxOffsets.putAll(result); - } - } - } - - @Override - public void removeSplit(IMessage message, AbstractContext context, RemoveSplitMessage removeSplitMessage) { - this.currentQueueIds = removeSplitMessage.getCurrentSplitIds(); - Set queueIds = removeSplitMessage.getSplitIds(); - if (queueIds != null) { - for (String queueId : queueIds) { - ShufflePartitionManager.getInstance().setSplitInValidate(queueId); - window.clearCache(queueId); - - } - window.getWindowMaxValueManager().removeKeyPrefixFromLocalCache(queueIds); - //window.getWindowFireSource().removeSplit(queueIds); - } - window.getFireReceiver().doMessage(message, context); - } - - @Override - public void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage) { - if (message.getHeader().isNeedFlush()) { - this.flush(message.getHeader().getCheckpointQueueIds()); - window.getSqlCache().flush(message.getHeader().getCheckpointQueueIds()); - } - CheckPointState checkPointState = new CheckPointState(); - checkPointState.setQueueIdAndOffset(this.shuffleCache.getFinishedQueueIdAndOffsets(checkPointMessage)); - checkPointMessage.reply(checkPointState); - } - - /** - * do system message - * - * @param oriMessage - * @param context - */ - protected void doSystemMessage(IMessage oriMessage, AbstractContext context) { - ISystemMessage systemMessage = oriMessage.getSystemMessage(); - if (systemMessage instanceof CheckPointMessage) { - this.checkpoint(oriMessage, context, (CheckPointMessage) systemMessage); - } else if (systemMessage instanceof NewSplitMessage) { - this.addNewSplit(oriMessage, context, (NewSplitMessage) systemMessage); - } else if (systemMessage instanceof RemoveSplitMessage) { - this.removeSplit(oriMessage, context, (RemoveSplitMessage) systemMessage); - } else if (systemMessage instanceof BatchFinishMessage) { - // if(COUNT.get()!=88121){ - // throw new RuntimeException("fired before receiver"); - // } - - this.batchMessageFinish(oriMessage, context, (BatchFinishMessage) systemMessage); - } else { - throw new RuntimeException("can not support this system message " + systemMessage.getClass().getName()); - } - afterFlushCallback(oriMessage, context); - } - - /** - * if the message offset is old filter the repeate message - * - * @param message - * @param queueId - * @return - */ - protected boolean isRepeateMessage(IMessage message, String queueId) { - boolean isOrigOffsetLong = message.getMessageBody().getBoolean(WindowCache.ORIGIN_QUEUE_IS_LONG); - String oriQueueId = message.getMessageBody().getString(WindowCache.ORIGIN_QUEUE_ID); - String oriOffset = message.getMessageBody().getString(WindowCache.ORIGIN_OFFSET); - String key = MapKeyUtil.createKey(window.getConfigureName(), queueId, oriQueueId); - String offset = this.split2MaxOffsets.get(key); - if (offset != null) { - MessageOffset messageOffset = new MessageOffset(oriOffset, isOrigOffsetLong); - if (!messageOffset.greateThan(offset)) { - System.out.println("the message offset is old, the message is discard "); - return true; - } - } - return false; - } - - @Override - protected Map getChannelConfig() { - return channelConfig; - } - - @Override - protected void putDynamicPropertyValue(Set dynamicPropertySet, Properties properties) { - String groupName = "groupName"; - if (!dynamicPropertySet.contains(groupName)) { - properties.put(groupName, getDynamicPropertyValue()); - } - if (!dynamicPropertySet.contains("tags")) { - properties.put("tags", getDynamicPropertyValue()); - } - } - - /** - * 1个pipeline一个 shuffle topic - * - * @param topic - * @param pipeline - * @return - */ - @Override - protected String createShuffleTopic(String topic, ChainPipeline pipeline) { - return "shuffle_" + topic + "_" + pipeline.getSource().getNameSpace().replaceAll("\\.", "_") + "_" + pipeline - .getConfigureName().replaceAll("\\.", "_").replaceAll(";", "_"); - } - - /** - * 一个window 一个channel - * - * @param pipeline - * @return - */ - @Override - protected String createShuffleChannelName(ChainPipeline pipeline) { - return getDynamicPropertyValue(); - } - - /** - * 和pipeline namespace 相同 - * - * @param pipeline - * @return - */ - @Override - protected String createShuffleChannelNameSpace(ChainPipeline pipeline) { - return pipeline.getSource().getNameSpace(); - } - - @Override - public String getConfigureName() { - return window.getConfigureName() + "_shuffle"; - } - - @Override - public String getNameSpace() { - return window.getNameSpace(); - } - - @Override - public String getType() { - return Pipeline.TYPE; - } - - public ISplit getSplit(Integer index) { - return queueList.get(index); - } - - public JSONObject createMsg(JSONArray messages, ISplit split) { - JSONObject msg = new JSONObject(); - //分片id - msg.put(SHUFFLE_QUEUE_ID, split.getQueueId()); - //合并的消息 - msg.put(SHUFFLE_MESSAGES, messages); - //消息owner - msg.put(MSG_OWNER, getDynamicPropertyValue()); - // - try { - List traceList = new ArrayList<>(); - List groupByList = new ArrayList<>(); - for (int i = 0; i < messages.size(); i++) { - JSONObject object = messages.getJSONObject(i); - groupByList.add(object.getString("SHUFFLE_KEY")); - traceList.add(object.getString(ORIGIN_MESSAGE_TRACE_ID)); - } - String traceInfo = StringUtils.join(traceList); - String groupInfo = StringUtils.join(groupByList); - msg.put(SHUFFLE_TRACE_ID, StringUtils.join(traceList)); - TraceUtil.debug(traceInfo, "origin message out", split.getQueueId(), groupInfo, getConfigureName()); - } catch (Exception e) { - //do nothing - } - return msg; - } - - public JSONArray getMsgs(JSONObject msg) { - return msg.getJSONArray(SHUFFLE_MESSAGES); - } - - public ISplit getChannelQueue(String key) { - int index = hash(key); - ISplit targetQueue = queueList.get(index); - return targetQueue; - } - - public int hash(Object key) { - int mValue = queueList.size(); - int h = 0; - if (key != null) { - h = key.hashCode() ^ (h >>> 16); - if (h < 0) { - h = -h; - } - } - return h % mValue; - } - - public void flush(Set checkpointQueueIds) { - shuffleCache.flush(checkpointQueueIds); - } - - /** - * 每次checkpoint的回调函数,默认是空实现,子类可以扩展实现 - * - * @param oriMessage - * @param context - */ - protected void afterFlushCallback(IMessage oriMessage, AbstractContext context) { - } - - /** - * shuffle 获取数据,插入缓存前的回调函数,默认空实现,可以子类覆盖扩展 - * - * @param oriMessage - * @param message - */ - protected void beforeBatchAdd(IMessage oriMessage, IMessage message) { - } - - /** - * 过滤掉不是这个window的消息 - * - * @param oriMessage - * @return - */ - protected boolean filterNotOwnerMessage(IMessage oriMessage) { - String owner = oriMessage.getMessageBody().getString(MSG_OWNER); - if (owner != null && owner.equals(getDynamicPropertyValue())) { - return false; - } - return true; - } - - @Override - protected String getDynamicPropertyValue() { - String dynamicPropertyValue = MapKeyUtil.createKey(window.getNameSpace(), window.getConfigureName(), window.getUpdateFlag() + ""); - dynamicPropertyValue = dynamicPropertyValue.replaceAll("\\.", "_").replaceAll(";", "_"); - return dynamicPropertyValue; - } - - @Override - protected int getShuffleSplitCount(AbstractSupportShuffleSink shuffleSink) { - int splitNum = shuffleSink.getSplitNum(); - return splitNum > 0 ? splitNum : 32; - } - - public Set getCurrentQueueIds() { - return currentQueueIds; - } - - public List> getQueueList() { - return queueList; - } - - public AbstractShuffleWindow getWindow() { - return window; - } - - @Override - public void batchMessageFinish(IMessage message, AbstractContext context, BatchFinishMessage batchFinishMessage) { - if (window.supportBatchMsgFinish()) { - // System.out.println("start fire window by fininsh flag "); - long startTime=System.currentTimeMillis(); - Set splitIds=new HashSet<>(); - splitIds.add(message.getHeader().getQueueId()); - shuffleCache.flush(splitIds); - window.getSqlCache().flush(splitIds); - window.getWindowFireSource().fireWindowInstance(message.getHeader().getQueueId()); - IMessage cpMsg = batchFinishMessage.getMsg().copy(); - window.getFireReceiver().doMessage(cpMsg, context); - System.out.println("batch message finish cost is "+(System.currentTimeMillis()-startTime)); - } - - } -} \ No newline at end of file diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleContext.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleContext.java new file mode 100644 index 00000000..a5c1d2c9 --- /dev/null +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleContext.java @@ -0,0 +1,29 @@ +/* + * 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.window.shuffle; + +public class ShuffleContext extends ThreadLocal { + private static ShuffleContext shuffleContext = new ShuffleContext(); + + private ShuffleContext() { + + } + + public static ShuffleContext getInstance() { + return shuffleContext; + } +} diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleManager.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleManager.java new file mode 100644 index 00000000..4e0f5a9e --- /dev/null +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleManager.java @@ -0,0 +1,315 @@ +/* + * 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.window.shuffle; + +import java.lang.reflect.Field; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder; +import org.apache.rocketmq.streams.common.channel.builder.IShuffleChannelBuilder; +import org.apache.rocketmq.streams.common.channel.impl.memory.MemoryCache; +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.AbstractSupportShuffleSink; +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.component.ComponentCreator; +import org.apache.rocketmq.streams.common.configuration.ConfigurationKey; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; +import org.apache.rocketmq.streams.common.utils.ReflectUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.serviceloader.ServiceLoaderComponent; +import org.apache.rocketmq.streams.window.operator.AbstractWindow; + +/** + * 创建shuffle source and sink + */ +public class ShuffleManager { + + protected static Map, AbstractSupportShuffleSink>> cache = new HashMap<>(); + + /** + * 创建shuffle source and sink,先检查配置文件,如果没有配置,则自动创建 + * + * @param pipelineSource 自动创建时,参考分片数量 + * @param window 需要shuffle的窗口 + * @return + */ + public static synchronized Pair, AbstractSupportShuffleSink> createOrGetShuffleSourceAndSink(ISource pipelineSource, AbstractWindow window) { + if (cache.get(window) != null) { + return cache.get(window); + } + String sourceType = getSourceTypeFromConfigue(window); + if (StringUtil.isNotEmpty(sourceType)) { + Pair, AbstractSupportShuffleSink> sourceISinkPair = createShuffleChannelByConfigure(window, sourceType); + if (sourceISinkPair != null && sourceISinkPair.getLeft() != null && sourceISinkPair.getRight() != null) { + cache.put(window, sourceISinkPair); + return sourceISinkPair; + } + } + Pair, AbstractSupportShuffleSink> sourceISinkPair = autoCreateShuffleChannel(pipelineSource, window); + if(sourceISinkPair.getLeft() instanceof AbstractSource){ + ((AbstractSource) sourceISinkPair.getLeft()).setJsonData(true); + ((AbstractSource) sourceISinkPair.getLeft()).setMsgIsJsonArray(false); + } + cache.put(window, sourceISinkPair); + return sourceISinkPair; + } + + protected static String getSourceTypeFromConfigue(AbstractWindow window) { + String type = ShuffleContext.getInstance().get(); + if (StringUtil.isNotEmpty(type)) { + return type; + } + return window.getConfiguration().getProperty(ConfigurationKey.WINDOW_SHUFFLE_CHANNEL_TYPE); + } + + public static Pair, AbstractSupportShuffleSink> createShuffleChannelByConfigure(AbstractWindow window, String sourceType) { + + ISource source = createSource(window, sourceType); + AbstractSupportShuffleSink shuffleSink = createSink(window, sourceType, source); + return Pair.of(source, shuffleSink); + } + + /** + * 如果用户未配置shuffle channel,根据pipeline数据源动态创建 + */ + public static Pair, AbstractSupportShuffleSink> autoCreateShuffleChannel(ISource pipelineSource, AbstractWindow window) { + synchronized (window) { + ServiceLoaderComponent serviceLoaderComponent = ComponentCreator.getComponent(IChannelBuilder.class.getName(), ServiceLoaderComponent.class); + + IChannelBuilder builder = (IChannelBuilder) serviceLoaderComponent.loadService(pipelineSource.getClass().getSimpleName()); + if (builder == null) { + throw new RuntimeException("can not create shuffle channel, not find channel builder " + pipelineSource.toJson()); + } + if (!(builder instanceof IShuffleChannelBuilder)) { + throw new RuntimeException("can not create shuffle channel, builder not impl IShuffleChannelBuilder " + pipelineSource.toJson()); + } + IShuffleChannelBuilder shuffleChannelBuilder = (IShuffleChannelBuilder) builder; + ISink sink = shuffleChannelBuilder.createBySource(pipelineSource); + if (sink == null) { + return null; + } + sink.init(); + if (!(sink instanceof MemorySink) && !(sink instanceof AbstractSupportShuffleSink)) { + throw new RuntimeException("can not create shuffle channel, sink not extends AbstractSupportShuffleSink " + pipelineSource.toJson()); + } + AbstractSupportShuffleSink abstractSupportShuffleSink = (AbstractSupportShuffleSink) sink; + abstractSupportShuffleSink.setSplitNum(10); + ISource source = null; + if (sink instanceof MemorySink) { + MemoryCache memoryCache = new MemoryCache(); + memoryCache.setNameSpace(pipelineSource.getNameSpace()); + memoryCache.setName(createNameByWindow(window)); + + sink = new MemorySink(); + source = new MemorySource(); + ((MemorySink) sink).setMemoryCache(memoryCache); + ((MemorySource) source).setMemoryCache(memoryCache); + memoryCache.init(); + } + + Properties properties = new Properties(); + properties.put("groupName", createNameByWindow(window)); + properties.put("tags", createNameByWindow(window)); + + AbstractSupportShuffleSink shuffleSink = (AbstractSupportShuffleSink) sink; + //todo 为什么这里的分区数量要和源头topic的分区数量一直? + shuffleSink.setSplitNum(getShuffleSplitCount(shuffleSink)); + shuffleSink.setNameSpace(pipelineSource.getNameSpace()); + shuffleSink.setName(createNameByWindow(window)); + String topicFiledName = shuffleSink.getShuffleTopicFieldName(); + String shuffleTopic = null; + //内存模式,是无topic的 + if (StringUtil.isNotEmpty(topicFiledName)) { + String topic = ReflectUtil.getDeclaredField(shuffleSink, topicFiledName); + shuffleTopic = createShuffleTopic(topic, pipelineSource); + ReflectUtil.setBeanFieldValue(shuffleSink, topicFiledName, shuffleTopic); + } + + //修改和window有关的属性,如groupname,tags + List fields = ReflectUtil.getDeclaredFieldsContainsParentClass(sink.getClass()); + for (Field field : fields) { + String fieldName = field.getName(); + String value = properties.getProperty(fieldName); + if (StringUtil.isNotEmpty(value)) { + ReflectUtil.setBeanFieldValue(sink, fieldName, value); + } + } + shuffleSink.setHasInit(false); + shuffleSink.init();//在这里完成shuffle channel的创建 + if (source == null) { + source = shuffleChannelBuilder.copy(pipelineSource); + } + + //修改和window有关的属性,如groupname,tags + + fields = ReflectUtil.getDeclaredFieldsContainsParentClass(source.getClass()); + for (Field field : fields) { + String fieldName = field.getName(); + String value = properties.getProperty(fieldName); + if (StringUtil.isNotEmpty(value)) { + ReflectUtil.setBeanFieldValue(source, fieldName, value); + } + } + + source.setNameSpace(sink.getNameSpace()); + source.setName(sink.getName()); + //修改主题 + if (shuffleTopic != null && topicFiledName != null) { + ReflectUtil.setBeanFieldValue(source, topicFiledName, shuffleTopic); + } + if (source instanceof AbstractSource) { + AbstractSource abstractSource = (AbstractSource) source; + abstractSource.setHasInit(false); + } + source.init(); + return Pair.of(source, shuffleSink); + + } + } + + protected static ISource createSource(AbstractWindow window, String sourceType) { + IChannelBuilder builder = createChannelBuilder(sourceType); + if (builder == null) { + return null; + } + Properties properties = createChannelProperties(window); + ISource source = builder.createSource(window.getNameSpace(), window.getName(), properties, null); + if (source instanceof MemorySource) { + MemorySource memorySource = (MemorySource) source; + MemoryCache memoryCache = new MemoryCache(); + memorySource.setMemoryCache(memoryCache); + memorySource.setNameSpace(window.getNameSpace()); + memorySource.setName(window.getName()); + memoryCache.init(); + } + source.init(); + return source; + } + + /** + * 创建channel,根据配置文件配置channel的连接信息 + * + * @return + */ + protected static AbstractSupportShuffleSink createSink(AbstractWindow window, String sourceType, ISource source) { + + IChannelBuilder builder = createChannelBuilder(sourceType); + if (builder == null) { + return null; + } + Properties properties = createChannelProperties(window); + + ISink sink = builder.createSink(window.getNameSpace(), window.getName(), properties, null); + if (!(sink instanceof AbstractSupportShuffleSink)) { + throw new RuntimeException("can not support shuffle " + sink.toJson()); + } + if (sink instanceof MemorySink) { + MemorySink memorySink = (MemorySink) sink; + if (!(source instanceof MemorySource)) { + throw new RuntimeException("shuffle cosumer need memory, real is " + source); + } + MemorySource memorySource = (MemorySource) source; + MemoryCache memoryCache = memorySource.getMemoryCache(); + memorySink.setMemoryCache(memoryCache); + memorySink.setNameSpace(window.getNameSpace()); + memorySink.setName(window.getName()); + } + + sink.init(); + return (AbstractSupportShuffleSink) sink; + } + + /** + * 自动创建shuffle source的分片数,等于shuffle sink的,shufflesink=pipeline source的分片 + * + * @param shuffleSink + * @return + */ + protected static int getShuffleSplitCount(AbstractSupportShuffleSink shuffleSink) { + int splitNum = shuffleSink.getSplitNum(); + return splitNum > 0 ? splitNum : 32; + } + + /** + * 根据window 信息生成名字,用于source,sink,group等名字 + * + * @param window + * @return + */ + protected static String createNameByWindow(AbstractWindow window) { + String dynamicPropertyValue = MapKeyUtil.createKey(window.getNameSpace(), window.getName(), String.valueOf(window.getUpdateFlag())); + dynamicPropertyValue = dynamicPropertyValue.replaceAll("\\.", "_").replaceAll(";", "_"); + return dynamicPropertyValue; + } + + /** + * 创建topic + * + * @param topic + * @param pipelineSource + * @return + */ + protected static String createShuffleTopic(String topic, ISource pipelineSource) { + return "shuffle_" + topic + "_" + pipelineSource.getNameSpace().replaceAll("\\.", "_") + "_" + pipelineSource.getName().replaceAll("\\.", "_").replaceAll(";", "_"); + } + + /** + * 加载source和sink的builder + * + * @param type + * @return + */ + protected static IChannelBuilder createChannelBuilder(String type) { + ServiceLoaderComponent serviceLoaderComponent = ComponentCreator.getComponent(IChannelBuilder.class.getName(), ServiceLoaderComponent.class); + return (IChannelBuilder) serviceLoaderComponent.loadService(type); + } + + /** + * 根据属性文件配置 + * + * @return 资源文件 + */ + protected static Properties createChannelProperties(AbstractWindow window) { + Properties properties = new Properties(); + for (Map.Entry entry : window.getConfiguration().entrySet()) { + String key = (String) entry.getKey(); + String value = (String) entry.getValue(); + if (key.startsWith(ConfigurationKey.WINDOW_SHUFFLE_CHANNEL_PROPERTY_PREFIX)) { + String channelKey = key.replace(ConfigurationKey.WINDOW_SHUFFLE_CHANNEL_PROPERTY_PREFIX, ""); + if (channelKey.startsWith(window.getNameSpace())) {//支持基于namespace 做shuffle window共享 + channelKey = channelKey.replace(window.getNameSpace(), ""); + properties.put(channelKey, value); + } else { + if (!properties.containsKey(channelKey)) { + properties.put(channelKey, value); + } + } + } + } + //TODO 其实只有rocketmq作为shuffle队列的时候,才需要设置这个参数, 这里后续需要修改 + properties.put("groupName", createNameByWindow(window)); + properties.put("tags", createNameByWindow(window)); + return properties; + } + +} diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleSink.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleSink.java new file mode 100644 index 00000000..94a36b05 --- /dev/null +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleSink.java @@ -0,0 +1,266 @@ +/* + * 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.window.shuffle; + +import com.alibaba.fastjson.JSONArray; +import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.MutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; +import org.apache.rocketmq.streams.common.channel.sink.AbstractSink; +import org.apache.rocketmq.streams.common.channel.sink.AbstractSupportShuffleSink; +import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageFlushCallBack; +import org.apache.rocketmq.streams.common.channel.source.systemmsg.WaterMarkNotifyMessage; +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.configurable.annotation.ConfigurableReference; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.context.Message; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; +import org.apache.rocketmq.streams.common.utils.TraceUtil; +import org.apache.rocketmq.streams.window.WindowConstants; +import org.apache.rocketmq.streams.window.minibatch.MiniBatchMsgCache; +import org.apache.rocketmq.streams.window.operator.AbstractShuffleWindow; +import org.apache.rocketmq.streams.window.util.ShuffleUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ShuffleSink extends + AbstractSink { + + public static final String ORIGIN_QUEUE_ID = "origin_queue_id"; + public static final String ORIGIN_MESSAGE_TRACE_ID = "origin_request_id"; + public static final String SHUFFLE_OFFSET = "SHUFFLE_OFFSET"; + protected static final String SHUFFLE_QUEUE_ID = "SHUFFLE_QUEUE_ID"; + protected static final String SHUFFLE_MESSAGES = "SHUFFLE_MESSAGES"; + private static final Logger LOGGER = LoggerFactory.getLogger(ShuffleSink.class); + private static final String SHUFFLE_TRACE_ID = "SHUFFLE_TRACE_ID"; + protected transient boolean isWindowTest = false; + protected transient AtomicLong COUNT = new AtomicLong(0); + /** + * 分片转发channel + */ + @ConfigurableReference protected AbstractSupportShuffleSink sink; + @ConfigurableReference protected AbstractShuffleWindow window; + /** + * 所有的分片 + */ + protected transient List> queueList; + protected transient Map> queueMap = new ConcurrentHashMap<>(); + protected transient MiniBatchMsgCache shuffleMsgCache; + /** + * 消息所属的window + */ + protected String MSG_OWNER = "MSG_OWNER"; + + @Override + protected boolean initConfigurable() { + super.initConfigurable(); + isWindowTest = ComponentCreator.getPropertyBooleanValue("window.fire.isTest"); + queueList = sink.getSplitList(); + Map> tmp = new ConcurrentHashMap<>(); + for (ISplit queue : queueList) { + tmp.put(queue.getQueueId(), queue); + } + this.queueMap = tmp; + initMiniBatch(); + return true; + + } + + public void initMiniBatch() { + shuffleMsgCache = new MiniBatchMsgCache(new ShuffleSink.MultiMsgMergerAndCompressFlushCallBack(), window, window); + shuffleMsgCache.openAutoFlush(); + } + + @Override + protected boolean batchInsert(List messageList) { + for (IMessage msg : messageList) { + String shuffleKey = window.generateShuffleKey(msg); + IMessage message = ShuffleUtil.createShuffleMsg(msg, shuffleKey); + if (message == null) { + continue; + } + addPropertyToMessage(msg, message.getMessageBody()); + Integer index = hash(shuffleKey); + ISplit split = getSplit(index); + shuffleMsgCache.addCache(new MutablePair<>(split, message)); + } + if (isWindowTest) { + long count = COUNT.addAndGet(messageList.size()); + System.out.println(window.getName() + " send shuffle msg count is " + count); + shuffleMsgCache.flush(); + } + return true; + } + + public void finishBatchMsg(BatchFinishMessage batchFinishMessage) { + if (sink != null) { + this.flush(); + shuffleMsgCache.flush(); + for (ISplit split : queueList) { + IMessage message = batchFinishMessage.getMsg().deepCopy(); + message.getMessageBody().put(ORIGIN_QUEUE_ID, message.getHeader().getQueueId()); + sink.batchAdd(message, split); + } + sink.flush(); + } + } + + /** + * @param msg + */ + public void sendNotifyToShuffleSource(String sourceSplitId, JSONObject msg) { + for (ISplit split : this.queueList) { + WaterMarkNotifyMessage waterMarkNotifyMessage = new WaterMarkNotifyMessage(); + waterMarkNotifyMessage.setSourceSplitId(sourceSplitId); + Message message = new Message(msg); + message.getHeader().setSystemMessage(true); + message.getMessageBody().put(ORIGIN_QUEUE_ID, sourceSplitId); + message.setSystemMessage(waterMarkNotifyMessage); + this.shuffleMsgCache.addCache(new MutablePair(split, message)); + } + this.shuffleMsgCache.flush(); + + } + + /** + * 根据message生成shuffle key + */ + // protected abstract String generateShuffleKey(IMessage message); + @Override + public boolean checkpoint(Set queueIds) { + this.flush(queueIds); + if (this.shuffleMsgCache != null) { + this.shuffleMsgCache.flush(queueIds); + } + return true; + } + + public ISplit getChannelQueue(String key) { + int index = hash(key); + return queueList.get(index); + } + + public int hash(Object key) { + int mValue = queueList.size(); + int h = 0; + if (key != null) { + h = key.hashCode() ^ (h >>> 16); + if (h < 0) { + h = -h; + } + } + return h % mValue; + } + + /** + * 如果需要额外的字段附加到shuffle前的message,通过实现这个子类增加 + */ + protected void addPropertyToMessage(IMessage oriMessage, JSONObject oriJson) { + + } + + public JSONObject createMsg(JSONArray messages, ISplit split) { + JSONObject msg = new JSONObject(); + //分片id + msg.put(SHUFFLE_QUEUE_ID, split.getQueueId()); + //合并的消息 + msg.put(SHUFFLE_MESSAGES, messages); + //消息owner + msg.put(MSG_OWNER, getDynamicPropertyValue()); + // + try { + List traceList = new ArrayList<>(); + List groupByList = new ArrayList<>(); + for (int i = 0; i < messages.size(); i++) { + JSONObject object = messages.getJSONObject(i); + groupByList.add(object.getString("SHUFFLE_KEY")); + traceList.add(object.getString(ORIGIN_MESSAGE_TRACE_ID)); + } + String traceInfo = StringUtils.join(traceList); + String groupInfo = StringUtils.join(groupByList); + msg.put(SHUFFLE_TRACE_ID, StringUtils.join(traceList)); + TraceUtil.debug(traceInfo, "origin message out", split.getQueueId(), groupInfo, getName()); + } catch (Exception e) { + LOGGER.error("[{}] create message error", getName(), e); + } + return msg; + } + + protected String getDynamicPropertyValue() { + String dynamicPropertyValue = MapKeyUtil.createKey(window.getNameSpace(), window.getName(), window.getUpdateFlag() + ""); + dynamicPropertyValue = dynamicPropertyValue.replaceAll("\\.", "_").replaceAll(";", "_"); + return dynamicPropertyValue; + } + + public MiniBatchMsgCache getShuffleMsgCache() { + return this.shuffleMsgCache; + } + + public ISplit getSplit(Integer index) { + return queueList.get(index); + } + + public AbstractShuffleWindow getWindow() { + return window; + } + + public void setWindow(AbstractShuffleWindow window) { + this.window = window; + } + + public void setSink(AbstractSupportShuffleSink sink) { + this.sink = sink; + } + + public List> getQueueList() { + return queueList; + } + + protected class MultiMsgMergerAndCompressFlushCallBack implements IMessageFlushCallBack, IMessage>> { + + @Override + public boolean flushMessage(List, IMessage>> messages) { + if (messages == null || messages.isEmpty()) { + return true; + } + ISplit split = messages.get(0).getLeft(); + JSONArray allMsgs = new JSONArray(); + for (Pair, IMessage> pair : messages) { + IMessage message = pair.getRight(); + if (message.getHeader().isSystemMessage()) { + message.getMessageBody().put(WindowConstants.IS_SYSTEME_MSG, true); + message.getMessageBody().put(WindowConstants.SYSTEME_MSG, message.getSystemMessage()); + message.getMessageBody().put(WindowConstants.SYSTEME_MSG_CLASS, message.getSystemMessage().getClass().getName()); + } + allMsgs.add(message.getMessageBody()); + } + JSONObject jsonObject = createMsg(allMsgs, split); + sink.batchAdd(new Message(jsonObject), split); + sink.flush(split.getQueueId()); + return true; + } + } +} diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/source/WindowShuffleSource.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/source/WindowShuffleSource.java deleted file mode 100644 index ef8f934a..00000000 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/source/WindowShuffleSource.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * 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.window.source; - -import org.apache.rocketmq.streams.common.channel.source.AbstractSource; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; -import org.apache.rocketmq.streams.common.topology.ChainPipeline; -import org.apache.rocketmq.streams.common.topology.SectionPipeline; -import org.apache.rocketmq.streams.common.topology.model.AbstractStage; -import org.apache.rocketmq.streams.common.topology.model.IWindow; -import org.apache.rocketmq.streams.common.topology.model.Pipeline; -import org.apache.rocketmq.streams.common.utils.MapKeyUtil; -import org.apache.rocketmq.streams.window.operator.AbstractWindow; -import org.apache.rocketmq.streams.window.util.WindowChannellUtil; - -public class WindowShuffleSource extends AbstractSource implements IAfterConfigurableRefreshListener { - - protected String windowName; - protected String pipelineName; - protected String windowStageLableName; - - protected transient AbstractSource source; - protected transient AbstractWindow window; - protected transient IStreamOperator pipelineAfterWindow; - - @Override protected boolean startSource() { - source.start(pipelineAfterWindow); - return true; - } - - public String getWindowName() { - return windowName; - } - - public void setWindowName(String windowName) { - this.windowName = windowName; - } - - public void setWindow(AbstractWindow window) { - this.window = window; - this.windowName = window.getConfigureName(); - } - - @Override public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - window = configurableService.queryConfigurable(IWindow.TYPE, windowName); - ChainPipeline pipeline = configurableService.queryConfigurable(Pipeline.TYPE, pipelineName); - AbstractStage windowStage = (AbstractStage) pipeline.getStageMap().get(windowStageLableName); - pipelineAfterWindow = new SectionPipeline(pipeline, windowStage); - String connector = ComponentCreator.getProperties().getProperty(WindowChannellUtil.WINDOW_SHUFFLE_CHANNEL_TYPE); - source = (AbstractSource) WindowChannellUtil.createSource(window.getNameSpace(), window.getConfigureName(), connector, ComponentCreator.getProperties(), WindowChannellUtil.WINDOW_SHUFFLE_CHANNEL_PROPERTY_PREFIX, getDynamicPropertyValue()); - if (source == null) { - - } - } - - protected String getDynamicPropertyValue() { - String dynamicPropertyValue = MapKeyUtil.createKey(window.getNameSpace(), window.getConfigureName()); - dynamicPropertyValue = dynamicPropertyValue.replaceAll("\\.", "_").replaceAll(";", "_"); - return dynamicPropertyValue; - } - - @Override public boolean supportRemoveSplitFind() { - throw new RuntimeException("can not support this method"); - } - - @Override protected boolean isNotDataSplit(String queueId) { - throw new RuntimeException("can not support this method"); - } - -} diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/SQLCache.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/SQLCache.java index bfe7a2b1..e18c96c6 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/SQLCache.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/SQLCache.java @@ -28,17 +28,24 @@ import org.apache.rocketmq.streams.common.channel.sinkcache.impl.AbstractMultiSplitMessageCache; import org.apache.rocketmq.streams.db.driver.DriverBuilder; import org.apache.rocketmq.streams.db.driver.JDBCDriver; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * cache sql, async and batch commit */ public class SQLCache extends AbstractMultiSplitMessageCache { + + private static final Logger LOGGER = LoggerFactory.getLogger(SQLCache.class); + protected Boolean isOpenCache = true;//if false,then execute sql when receive sql protected Set firedWindowInstances = new HashSet<>();//fired window instance ,if the owned sqls have not commit, can cancel the sqls protected Map windowInstance2Index = new HashMap<>();//set index to ISQLElement group by window instance protected boolean isLocalOnly; + protected AtomicInteger executeSQLCount = new AtomicInteger(0); + protected AtomicInteger cancelQLCount = new AtomicInteger(0); public SQLCache(boolean isLocalOnly) { super(null); @@ -55,7 +62,7 @@ public int addCache(ISQLElement isqlElement) { if (isLocalOnly) { return 0; } - if (isOpenCache == false) { + if (!isOpenCache) { DriverBuilder.createDriver().execute(isqlElement.getSQL()); return 1; } @@ -79,9 +86,6 @@ protected String createSplitId(ISQLElement msg) { return msg.getQueueId(); } - protected AtomicInteger executeSQLCount = new AtomicInteger(0); - protected AtomicInteger cancelQLCount = new AtomicInteger(0); - protected class SQLCacheCallback implements IMessageFlushCallBack { @Override @@ -105,15 +109,13 @@ public boolean flushMessage(List messages) { JDBCDriver dataSource = DriverBuilder.createDriver(); try { executeSQLCount.addAndGet(sqls.size()); - dataSource.executSqls(sqls); - System.out.println("execute sql count is " + executeSQLCount.get() + "; cancel sql count is " + cancelQLCount.get()); + dataSource.executeSqls(sqls); + LOGGER.debug("execute sql count is {}; cancel sql count is {}", executeSQLCount.get(), cancelQLCount.get()); } catch (Exception e) { e.printStackTrace(); throw new RuntimeException(e); } finally { - if (dataSource != null) { - dataSource.destroy(); - } + dataSource.destroy(); } return true; } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/FiredNotifySQLElement.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/FiredNotifySQLElement.java index b06b691c..5298d93b 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/FiredNotifySQLElement.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/FiredNotifySQLElement.java @@ -22,9 +22,10 @@ public class FiredNotifySQLElement implements ISQLElement { protected String queueId; protected String windowInstanceId; - public FiredNotifySQLElement(String splitId,String windowInstanceId){ - this.queueId=splitId; - this.windowInstanceId=windowInstanceId; + + public FiredNotifySQLElement(String splitId, String windowInstanceId) { + this.queueId = splitId; + this.windowInstanceId = windowInstanceId; } @Override public boolean isWindowInstanceSQL() { diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/SQLElement.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/SQLElement.java index 94abbf95..29ec7ffa 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/SQLElement.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/SQLElement.java @@ -24,10 +24,11 @@ public class SQLElement implements ISQLElement { protected String windowInstanceId; protected String sql; protected Integer index; - public SQLElement(String splitId,String windowInstanceId,String sql){ - this.queueId=splitId; - this.windowInstanceId=windowInstanceId; - this.sql=sql; + + public SQLElement(String splitId, String windowInstanceId, String sql) { + this.queueId = splitId; + this.windowInstanceId = windowInstanceId; + this.sql = sql; } @Override public boolean isWindowInstanceSQL() { @@ -66,5 +67,4 @@ public void setIndex(int index) { this.index = index; } - } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/SplitSQLElement.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/SplitSQLElement.java index ddf5e2f7..470aefc0 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/SplitSQLElement.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/SplitSQLElement.java @@ -22,11 +22,11 @@ public class SplitSQLElement implements ISQLElement { protected String queueId; protected String sql; - public SplitSQLElement(String splitId,String sql){ - this.queueId=splitId; - this.sql=sql; - } + public SplitSQLElement(String splitId, String sql) { + this.queueId = splitId; + this.sql = sql; + } @Override public boolean isWindowInstanceSQL() { return false; diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/AbstractMapState.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/AbstractMapState.java index 5bcc3d23..a0b6dd1a 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/AbstractMapState.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/AbstractMapState.java @@ -31,17 +31,16 @@ public AbstractMapState(String namespace, String backendName) { } public static void main(String[] args) { - TreeMap treeMa=new TreeMap<>(); - treeMa.put("2",2); - treeMa.put("3",3); - treeMa.put("1",1); + TreeMap treeMa = new TreeMap<>(); + treeMa.put("2", 2); + treeMa.put("3", 3); + treeMa.put("1", 1); Iterator> it = treeMa.entrySet().iterator(); - while (it.hasNext()){ - Map.Entry entry=it.next(); + while (it.hasNext()) { + Map.Entry entry = it.next(); System.out.println(entry.getKey()); } } - } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/WindowBaseValue.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/WindowBaseValue.java index 01295538..45b6afc9 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/WindowBaseValue.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/WindowBaseValue.java @@ -18,17 +18,12 @@ import java.io.Serializable; import java.util.Date; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.model.Entity; import org.apache.rocketmq.streams.common.utils.DateUtil; import org.apache.rocketmq.streams.common.utils.SerializeUtil; public class WindowBaseValue extends Entity implements Serializable { - private static final Log LOG = LogFactory.getLog(WindowBaseValue.class); - private static final long serialVersionUID = -4985883726971532986L; /** @@ -71,15 +66,17 @@ public class WindowBaseValue extends Entity implements Serializable { */ protected String fireTime; - protected AtomicLong updateVersion = new AtomicLong(0); + protected Long updateVersion = new Long(0); public WindowBaseValue() { setGmtCreate(DateUtil.getCurrentTime()); setGmtModified(DateUtil.getCurrentTime()); } - public long incrementUpdateVersion() { - return updateVersion.incrementAndGet(); + public synchronized long incrementUpdateVersion() { + + updateVersion = updateVersion + 1; + return updateVersion; } @Override @@ -159,11 +156,11 @@ public void setFireTime(String fireTime) { } public long getUpdateVersion() { - return updateVersion.get(); + return updateVersion; } public void setUpdateVersion(long updateVersion) { - this.updateVersion.set(updateVersion); + this.updateVersion = (updateVersion); } public String getWindowInstancePartitionId() { diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/WindowValue.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/WindowValue.java index e45e96f1..9948ff6b 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/WindowValue.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/WindowValue.java @@ -28,22 +28,21 @@ import java.util.Map.Entry; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.Message; import org.apache.rocketmq.streams.common.datatype.DataType; import org.apache.rocketmq.streams.common.datatype.NotSupportDataType; import org.apache.rocketmq.streams.common.interfaces.ISerialize; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; +import org.apache.rocketmq.streams.common.model.NameCreator; +import org.apache.rocketmq.streams.common.model.ThreadContext; import org.apache.rocketmq.streams.common.utils.Base64Utils; import org.apache.rocketmq.streams.common.utils.DataTypeUtil; import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.common.utils.IdUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.common.utils.ReflectUtil; -import org.apache.rocketmq.streams.common.utils.SerializeUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.common.utils.TraceUtil; import org.apache.rocketmq.streams.db.driver.orm.ORMUtil; @@ -55,35 +54,35 @@ import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.operator.AbstractWindow; import org.apache.rocketmq.streams.window.state.WindowBaseValue; -import org.nustaq.serialization.FSTConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class WindowValue extends WindowBaseValue implements Serializable ,ISerialize{ +public class WindowValue extends WindowBaseValue implements Serializable, ISerialize { private static final long serialVersionUID = 1083444850264401338L; - - private static final Log LOG = LogFactory.getLog(WindowValue.class); - + private static final Logger LOGGER = LoggerFactory.getLogger(WindowValue.class); + protected static AtomicInteger SUM = new AtomicInteger(0); /** * 如果做分组,设置分组 */ protected String groupBy = "0"; - + protected String groupByFieldName; /** * split id和max offset的映射关系 */ protected Map maxOffset = new ConcurrentHashMap<>(16); - - /** - * the result of aggregation column - */ - private Map aggColumnResult = new ConcurrentHashMap<>(16); - /** * the result of select column */ protected Map computedColumnResult = new HashMap<>(16); + protected transient Map ignoreGroupFieldByRollup2ComputerColumn;//In rollup, fields that are not in the prefix field will be ignored because of the prefix field. Ignoring is only limited to non udaf functions + protected transient Long lastUpdateTime;//used in session window,set last update time + /** + * the result of aggregation column + */ + private Map aggColumnResult = new ConcurrentHashMap<>(16); public WindowValue() { setGmtCreate(DateUtil.getCurrentTime()); @@ -97,6 +96,142 @@ public WindowValue(WindowValue theValue) { this.groupBy = theValue.getGroupBy(); } + /** + * merge different window values into one window value which have the same group by value + * + * @param window the window definition + * @param windowInstances all window instance which belong to same window and have different group by value + * @return + */ + public static List mergeWindowValues(AbstractWindow window, List windowInstances) { + if (windowInstances == null || windowInstances.size() == 0) { + return new ArrayList<>(); + } + StringBuilder sb = new StringBuilder(); + boolean isFirst = true; + String name = MapKeyUtil.createKey(window.getNameSpace(), window.getName()); + for (WindowInstance windowInstance : windowInstances) { + if (isFirst) { + isFirst = false; + } else { + sb.append(","); + } + sb.append("('" + name + "','" + windowInstance.getStartTime() + "','" + windowInstance.getEndTime() + "')"); + } + String inSQL = sb.toString(); + /* + * 分批,内存撑暴 todo + */ + String sql = "select * from " + ORMUtil.getTableName(WindowValue.class) + " where status > 0 && (name, start_time, end_time) in (" + inSQL + ")"; + Map paras = new HashMap<>(4); + List windowValueList = ORMUtil.queryForList(sql, paras, WindowValue.class); + return queryMergeWindowValues(window, windowValueList); + } + + public static List queryMergeWindowValues(AbstractWindow window, List windowValueList) { + Map> groupWindowMap = new HashMap<>(64); + for (WindowValue value : windowValueList) { + String key = MapKeyUtil.createKeyBySign(value.getStartTime(), value.getEndTime(), + value.getGroupBy()); + if (groupWindowMap.containsKey(key)) { + groupWindowMap.get(key).add(value); + } else { + groupWindowMap.put(key, new ArrayList() {{ + add(value); + }}); + } + } + List mergedValueList = new ArrayList<>(); + for (Entry> entry : groupWindowMap.entrySet()) { + mergedValueList.add(mergeWindowValue(window, entry.getValue())); + } + return mergedValueList; + } + + /** + * merge the group which has the same group by value and different split id + */ + public static WindowValue mergeWindowValue(AbstractWindow window, List valueList) { + WindowValue lastWindowValue = new WindowValue(valueList.get(0)); + lastWindowValue.setComputedColumnResult(valueList.get(0).getComputedColumnResult()); + // + for (Entry> entry : window.getColumnExecuteMap().entrySet()) { + String computedColumn = entry.getKey(); + IMessage message = new Message(new JSONObject()); + FunctionContext context = new FunctionContext(message); + List executorList = entry.getValue(); + //column outside of the aggregation function should be calculated again! + boolean needMergeComputation = false; + for (FunctionExecutor info : executorList) { + String column = info.getColumn(); + IStreamOperator> engine = info.getExecutor(); + if (engine instanceof AggregationScript) { + AggregationScript origin = (AggregationScript) engine; + AggregationScript operator = origin.clone(); + if (needMergeComputation) { + message.getMessageBody().put(AggregationScript.INNER_AGGREGATION_COMPUTE_KEY, AggregationScript.INNER_AGGREGATION_COMPUTE_SINGLE); + operator.setAccumulator(operator.getDirector().createAccumulator()); + operator.doMessage(message, context); + } else { + message.getMessageBody().put(AggregationScript.INNER_AGGREGATION_COMPUTE_KEY, AggregationScript.INNER_AGGREGATION_COMPUTE_MULTI); + List actors = valueList.stream().map( + windowValue -> { + Object accumulator = null; + if (windowValue.aggColumnResult.containsKey(column)) { + accumulator = windowValue.aggColumnResult.get(column); + } else { + IAccumulator director = AggregationScript.getAggregationFunction(operator.getFunctionName()); + accumulator = director.createAccumulator(); + windowValue.aggColumnResult.put(column, accumulator); + } + return accumulator; + }).collect( + Collectors.toList()); + operator.setAccumulator(operator.getDirector().createAccumulator()); + operator.setAccumulators(actors); + lastWindowValue.aggColumnResult.put(column, operator.getAccumulator()); + operator.doMessage(message, context); + needMergeComputation = true; + } + } else if (engine instanceof FunctionScript) { + FunctionScript theScript = (FunctionScript) engine; + String[] parameters = theScript.getDependentParameters(); + for (String parameter : parameters) { + if (!message.getMessageBody().containsKey(parameter) && lastWindowValue.computedColumnResult.containsKey(parameter)) { + message.getMessageBody().put(parameter, lastWindowValue.computedColumnResult.get(parameter)); + } + } + if (needMergeComputation) { + engine.doMessage(message, context); + } + } + } + if (message.getMessageBody().containsKey(computedColumn)) { + lastWindowValue.computedColumnResult.put(computedColumn, message.getMessageBody().get(computedColumn)); + } else if (!needMergeComputation) { + lastWindowValue.computedColumnResult.put(computedColumn, valueList.get(0).computedColumnResult.get(computedColumn)); + } + } + List traceList = new ArrayList<>(); + for (WindowValue value : valueList) { + if (value.computedColumnResult.containsKey(TraceUtil.TRACE_ID_FLAG)) { + String traceIds = String.valueOf(value.computedColumnResult.get(TraceUtil.TRACE_ID_FLAG)); + traceList.addAll(Arrays.asList(traceIds.split(","))); + } + } + if (!traceList.isEmpty()) { + StringBuffer buffer = new StringBuffer(); + for (int i = 0; i < traceList.size(); i++) { + buffer.append(traceList.get(i)); + if (i != traceList.size() - 1) { + buffer.append(","); + } + } + lastWindowValue.computedColumnResult.put(TraceUtil.TRACE_ID_FLAG, buffer.toString()); + } + return lastWindowValue; + } + /** * 计算结果序列化成json * @@ -105,12 +240,10 @@ public WindowValue(WindowValue theValue) { public String getAggColumnResult() { JSONArray jsonArray = new JSONArray(); - Iterator> it = aggColumnResult.entrySet().iterator(); - while (it.hasNext()) { + for (Entry objectEntry : aggColumnResult.entrySet()) { JSONObject jsonObject = new JSONObject(); - Entry entry = it.next(); - String functionName = entry.getKey(); - Object value = entry.getValue(); + String functionName = objectEntry.getKey(); + Object value = objectEntry.getValue(); jsonObject.put("function", functionName); if (value == null) { continue; @@ -118,12 +251,11 @@ public String getAggColumnResult() { DataType dataType = DataTypeUtil.getDataTypeFromClass(value.getClass()); boolean isBasicType = false; String jsonValue = null; - if (!NotSupportDataType.class.isInstance(dataType)) { + if (!(dataType instanceof NotSupportDataType)) { isBasicType = true; jsonValue = dataType.toDataJson(value); jsonObject.put("datatype", dataType.getDataTypeName()); } else { - isBasicType = false; jsonValue = ReflectUtil.serializeObject(value).toJSONString(); } @@ -167,18 +299,13 @@ public void setAggColumnMap(Map aggColumnResult) { this.aggColumnResult = aggColumnResult; } - public void setComputedColumnResult(String computedColumnResult) { - computedColumnResult = decodeSQLContent(computedColumnResult); - this.computedColumnResult = Message.parseObject(computedColumnResult); - } - public void putComputedColumnResult(Map msg) { this.computedColumnResult.putAll(msg); } public String getComputedColumnResult() { JSONObject object = null; - if (JSONObject.class.isInstance(computedColumnResult)) { + if (computedColumnResult instanceof JSONObject) { object = (JSONObject) computedColumnResult; } else { object = new JSONObject(computedColumnResult); @@ -186,6 +313,17 @@ public String getComputedColumnResult() { return encodeSQLContent(object.toJSONString()); } + public void setComputedColumnResult(String computedColumnResult) { + computedColumnResult = decodeSQLContent(computedColumnResult); + this.computedColumnResult = Message.parseObject(computedColumnResult); + } + + public String getMaxOffset() { + JSONObject object = new JSONObject(); + object.putAll(maxOffset); + return object.toJSONString(); + } + public void setMaxOffset(String theOffset) { JSONObject object = JSONObject.parseObject(theOffset); Iterator iterator = object.keySet().iterator(); @@ -195,12 +333,6 @@ public void setMaxOffset(String theOffset) { } } - public String getMaxOffset() { - JSONObject object = new JSONObject(); - object.putAll(maxOffset); - return object.toJSONString(); - } - public Iterator> iteratorComputedColumnResult() { return computedColumnResult.entrySet().iterator(); } @@ -230,8 +362,6 @@ public Map getcomputedResult() { * @param message the consumed data */ public boolean calculate(AbstractWindow window, IMessage message) { - message.getMessageBody().put(AbstractWindow.WINDOW_START, startTime); - message.getMessageBody().put(AbstractWindow.WINDOW_END, endTime); //每个计算节点对应一个consume split,如果多于一个queue的话, String queueId = message.getHeader().getQueueId(); String offset = message.getHeader().getOffset(); @@ -246,27 +376,35 @@ public boolean calculate(AbstractWindow window, IMessage message) { if (message.getHeader().greateThan(maxOffsetOfQueue)) { this.maxOffset.put(queueId, offset); } else { - //如果比最大的offset 小或等于,则直接丢弃掉消息 - System.out.println("!!!!!!!!!!!!!!!!!!! has outOfOrder data " + maxOffsetOfQueue + " " + message.getHeader().getOffset()); - return false; + //如果是内存存储,不需要支持精确计算一次,能容忍乱序 + if (window.isLocalStorageOnly() == false) { + //如果比最大的offset 小或等于,则直接丢弃掉消息 + Long time = window.getEventTimeManager().getMaxEventTime(this.partition); + String maxEventTime = null; + if (time != null) { + maxEventTime = DateUtil.longToString(time); + } + LOGGER.warn("[{}][{}] Window_OutOfOrder_Discard_Offset({}-{})_(startTime:{}-endTime:{}-fireTime:{}-maxEventTime:{})", IdUtil.instanceId(), NameCreator.getFirstPrefix(window.getName()), message.getHeader().getOffset(), maxOffsetOfQueue, this.startTime, this.endTime, this.fireTime, maxEventTime); + return false; + } } } try { this.lastUpdateTime = WindowInstance.getOccurTime(window, message); - if (window.getReduceSerializeValue() != null) { - JSONObject accumulator = null; - if (computedColumnResult != null && JSONObject.class.isInstance(computedColumnResult)) { - accumulator = (JSONObject) computedColumnResult; - } - - JSONObject result = window.getReducer().reduce(accumulator, message.getMessageBody()); - computedColumnResult = result; - return true; - } +// if (window.getReduceSerializeValue() != null) { +// JSONObject accumulator = null; +// if (computedColumnResult != null && JSONObject.class.isInstance(computedColumnResult)) { +// accumulator = (JSONObject) computedColumnResult; +// } +// +// JSONObject result = window.getReducer().reduce(accumulator, message.getMessageBody()); +// computedColumnResult = result; +// return true; +// } calFunctionColumn(window, message); } catch (Exception e) { - throw new RuntimeException("failed in window value calculating",e); + throw new RuntimeException("failed in window value calculating", e); } //there is no need writing back to message @@ -274,22 +412,26 @@ public boolean calculate(AbstractWindow window, IMessage message) { return true; } - protected static AtomicInteger SUM=new AtomicInteger(0); - protected void calFunctionColumn(AbstractWindow window, IMessage message) { - String introduction = (String)message.getMessageBody().getOrDefault(AggregationScript.INNER_AGGREGATION_COMPUTE_KEY, ""); + if (window.isOutputWindowInstanceInfo()) { + message.getMessageBody().put(AbstractWindow.WINDOW_START, this.startTime); + message.getMessageBody().put(AbstractWindow.WINDOW_END, this.endTime); + message.getMessageBody().put(AbstractWindow.WINDOW_TIME, DateUtil.parse(this.endTime).getTime() - 1); + } + + String introduction = (String) message.getMessageBody().getOrDefault(AggregationScript.INNER_AGGREGATION_COMPUTE_KEY, ""); boolean isMultiAccumulate = AggregationScript.INNER_AGGREGATION_COMPUTE_MULTI.equals(introduction); - if(isMultiAccumulate){ - WindowValue windowValue=message.getMessageBody().getObject(WindowValue.class.getName(),WindowValue.class); + if (isMultiAccumulate) { + WindowValue windowValue = message.getMessageBody().getObject(WindowValue.class.getName(), WindowValue.class); try { - // windowValue= SerializeUtil.deserialize(windowValueJson,WindowValue.class); - }catch (Exception e){ - throw new RuntimeException("window value deserializeObject error",e); + // windowValue= SerializeUtil.deserialize(windowValueJson,WindowValue.class); + } catch (Exception e) { + throw new RuntimeException("window value deserializeObject error", e); } - List windowValues=new ArrayList<>(); + List windowValues = new ArrayList<>(); windowValues.add(this); windowValues.add(windowValue); - WindowValue mergerWindowValue= WindowValue.mergeWindowValue(window,windowValues); + WindowValue mergerWindowValue = WindowValue.mergeWindowValue(window, windowValues); this.computedColumnResult.putAll(mergerWindowValue.computedColumnResult); this.aggColumnResult.putAll(mergerWindowValue.aggColumnResult); return; @@ -313,181 +455,52 @@ protected void calFunctionColumn(AbstractWindow window, IMessage message) { aggColumnResult.put(executorName, accumulator); } windowAccScript.setAccumulator(accumulator); - if(!isMultiAccumulate){ + if (!isMultiAccumulate) { message.getMessageBody().put(AggregationScript.INNER_AGGREGATION_COMPUTE_KEY, AggregationScript.INNER_AGGREGATION_COMPUTE_SINGLE); } FunctionContext context = new FunctionContext(message); + ThreadContext threadContext = ThreadContext.getInstance(); + context.put(IMessage.class.getName(), message); + context.put("start_time", getStartTime()); + context.put("end_time", getEndTime()); + threadContext.set(context); windowAccScript.doMessage(message, context); } else if (executor instanceof FunctionScript) { + FunctionScript functionScript = (FunctionScript) executor; FunctionContext context = new FunctionContext(message); - ((FunctionScript) executor).doMessage(message, context); + functionScript.doMessage(message, context); + } } // - computedColumnResult.put(computedColumn, message.getMessageBody().get(computedColumn)); + if (this.ignoreGroupFieldByRollup2ComputerColumn != null && this.ignoreGroupFieldByRollup2ComputerColumn.values().contains(computedColumn)) { + // in rollup场景 + } else { + computedColumnResult.put(computedColumn, message.getMessageBody().get(computedColumn)); + } + } calProjectColumn(window, message); } protected void calProjectColumn(AbstractWindow window, IMessage message) { + Map constMap = window.getColumnProjectMap(); for (Entry entry : constMap.entrySet()) { String computedColumn = entry.getKey(); String originColumn = entry.getValue(); + if (this.ignoreGroupFieldByRollup2ComputerColumn != null && this.ignoreGroupFieldByRollup2ComputerColumn.values().contains(originColumn)) { + continue; + } if (message.getMessageBody().containsKey(originColumn)) { computedColumnResult.put(computedColumn, message.getMessageBody().get(originColumn)); } else { - if (LOG.isWarnEnabled()) { - LOG.warn("field:\t " + originColumn + " lost!"); - } - } - } - } - - /** - * merge different window values into one window value which have the same group by value - * - * @param window the window definition - * @param windowInstances all window instance which belong to same window and have different group by value - * @return - */ - public static List mergeWindowValues(AbstractWindow window, List windowInstances) { - if (windowInstances == null || windowInstances.size() == 0) { - return new ArrayList<>(); - } - StringBuilder sb = new StringBuilder(); - boolean isFirst = true; - String name = MapKeyUtil.createKey(window.getNameSpace(), window.getConfigureName()); - for (WindowInstance windowInstance : windowInstances) { - if (isFirst) { - isFirst = false; - } else { - sb.append(","); - } - sb.append("('" + name + "','" + windowInstance.getStartTime() + "','" + windowInstance.getEndTime() + "')"); - } - String inSQL = sb.toString(); - /** - * 分批,内存撑暴 todo - */ - String sql = "select * from " + ORMUtil - .getTableName(WindowValue.class) + " where status > 0 && (name, start_time, end_time) in (" + inSQL + ")"; - Map paras = new HashMap<>(4); - List windowValueList = ORMUtil.queryForList(sql, paras, WindowValue.class); - return queryMergeWindowValues(window, windowValueList); - } - - public static List queryMergeWindowValues(AbstractWindow window, List windowValueList) { - Map> groupWindowMap = new HashMap<>(64); - for (WindowValue value : windowValueList) { - String key = MapKeyUtil.createKeyBySign(value.getStartTime(), value.getEndTime(), - value.getGroupBy()); - if (groupWindowMap.containsKey(key)) { - groupWindowMap.get(key).add(value); - } else { - groupWindowMap.put(key, new ArrayList() {{ - add(value); - }}); - } - } - List mergedValueList = new ArrayList<>(); - for (Entry> entry : groupWindowMap.entrySet()) { - mergedValueList.add(mergeWindowValue(window, entry.getValue())); - } - return mergedValueList; - } - - /** - * merge the group which has the same group by value and different split id - */ - public static WindowValue mergeWindowValue(AbstractWindow window, List valueList) { - WindowValue lastWindowValue = new WindowValue(valueList.get(0)); - lastWindowValue.setComputedColumnResult(valueList.get(0).getComputedColumnResult()); - // - for (Entry> entry : window.getColumnExecuteMap().entrySet()) { - String computedColumn = entry.getKey(); - IMessage message = new Message(new JSONObject()); - FunctionContext context = new FunctionContext(message); - List executorList = entry.getValue(); - //column outside of the aggregation function should be calculated again! - boolean needMergeComputation = false; - for (FunctionExecutor info : executorList) { - String column = info.getColumn(); - IStreamOperator> engine = info.getExecutor(); - if (engine instanceof AggregationScript) { - AggregationScript origin = (AggregationScript) engine; - AggregationScript operator = origin.clone(); - if (needMergeComputation) { - message.getMessageBody().put(AggregationScript.INNER_AGGREGATION_COMPUTE_KEY, - AggregationScript.INNER_AGGREGATION_COMPUTE_SINGLE); - operator.setAccumulator(operator.getDirector().createAccumulator()); - operator.doMessage(message, context); - } else { - message.getMessageBody().put(AggregationScript.INNER_AGGREGATION_COMPUTE_KEY, - AggregationScript.INNER_AGGREGATION_COMPUTE_MULTI); - List actors = valueList.stream().map( - windowValue -> { - Object accumulator = null; - if (windowValue.aggColumnResult.containsKey(column)) { - accumulator = windowValue.aggColumnResult.get(column); - } else { - IAccumulator director = AggregationScript.getAggregationFunction( - operator.getFunctionName()); - accumulator = director.createAccumulator(); - windowValue.aggColumnResult.put(column, accumulator); - } - return accumulator; - }).collect( - Collectors.toList()); - operator.setAccumulator(operator.getDirector().createAccumulator()); - operator.setAccumulators(actors); - lastWindowValue.aggColumnResult.put(column,operator.getAccumulator()); - operator.doMessage(message, context); - needMergeComputation = true; - } - } else if (engine instanceof FunctionScript) { - FunctionScript theScript = (FunctionScript) engine; - String[] parameters = theScript.getDependentParameters(); - for (String parameter : parameters) { - if (!message.getMessageBody().containsKey(parameter) && lastWindowValue.computedColumnResult - .containsKey(parameter)) { - message.getMessageBody().put(parameter, - lastWindowValue.computedColumnResult.get(parameter)); - } - } - if (needMergeComputation) { - engine.doMessage(message, context); - } + if (LOGGER.isWarnEnabled()) { + LOGGER.warn(window.getName() + " field:\t " + originColumn + " lost!"); } } - if (message.getMessageBody().containsKey(computedColumn)) { - lastWindowValue.computedColumnResult.put(computedColumn, message.getMessageBody().get(computedColumn)); - } else if (!needMergeComputation) { - lastWindowValue.computedColumnResult.put(computedColumn, - valueList.get(0).computedColumnResult.get(computedColumn)); - } } - // valueList.stream().map(value -> lastWindowValue.count += value.getCount()); - // - List traceList = new ArrayList<>(); - for (WindowValue value : valueList) { - if (value.computedColumnResult.containsKey(TraceUtil.TRACE_ID_FLAG)) { - String traceIds = String.valueOf(value.computedColumnResult.get(TraceUtil.TRACE_ID_FLAG)); - traceList.addAll(Arrays.asList(traceIds.split(","))); - } - } - if (!traceList.isEmpty()) { - StringBuffer buffer = new StringBuffer(); - for (int i = 0; i < traceList.size(); i++) { - buffer.append(traceList.get(i)); - if (i != traceList.size() - 1) { - buffer.append(","); - } - } - lastWindowValue.computedColumnResult.put(TraceUtil.TRACE_ID_FLAG, buffer.toString()); - } - return lastWindowValue; } public Object getAccumulatorByColumn(String column) { @@ -553,5 +566,21 @@ protected String decodeSQLContent(String sqlContent) { } } + public String getGroupByFieldName() { + return groupByFieldName; + } + + public void setGroupByFieldName(String groupByFieldName) { + this.groupByFieldName = groupByFieldName; + } + + public Map getIgnoreGroupFieldByRollup2ComputerColumn() { + return ignoreGroupFieldByRollup2ComputerColumn; + } + + public void setIgnoreGroupFieldByRollup2ComputerColumn( + Map ignoreGroupFieldByRollup2ComputerColumn) { + this.ignoreGroupFieldByRollup2ComputerColumn = ignoreGroupFieldByRollup2ComputerColumn; + } } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/AbstractWindowStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/AbstractWindowStorage.java index 12cffbf3..c255ac57 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/AbstractWindowStorage.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/AbstractWindowStorage.java @@ -17,12 +17,12 @@ package org.apache.rocketmq.streams.window.storage; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import org.apache.rocketmq.streams.common.threadpool.ThreadPoolFactory; import org.apache.rocketmq.streams.common.utils.StringUtil; @@ -33,11 +33,9 @@ public abstract class AbstractWindowStorage implements IWindowStorage { protected boolean isLocalStorageOnly = false; -// protected transient ExecutorService dataLoaderExecutor = new ThreadPoolExecutor(10, 10, -// 0L, TimeUnit.MILLISECONDS, -// new LinkedBlockingQueue(), new ThreadPoolFactory.DipperThreadFactory("AbstractWindowStorage-")); - protected transient ExecutorService dataLoaderExecutor = ThreadPoolFactory.createThreadPool(10); + protected transient ExecutorService dataLoaderExecutor = ThreadPoolFactory.createThreadPool(10, 10, 0L, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(), "AbstractWindowStorage"); ; @Override @@ -56,18 +54,11 @@ public void loadSplitData2Local(String queueId, String windowInstanceId, Class { + long start = System.currentTimeMillis(); + BatchRowLoader batchRowLoader = new BatchRowLoader("partition_num", "select * from " + ORMUtil.getTableName(clazz) + " where window_instance_partition_id ='" + windowInstancePartitionId + "'", processor); + batchRowLoader.startLoadData(); + ShufflePartitionManager.getInstance().setWindowInstanceFinished(windowInstanceId); }); } @@ -90,10 +81,7 @@ public T get(Class clazz, String key) { @Override public Map multiGet(Class clazz, String... keys) { - List keyList = new ArrayList<>(); - for (String key : keys) { - keyList.add(key); - } + List keyList = new ArrayList<>(Arrays.asList(keys)); return multiGet(clazz, keyList); } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IRemoteStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IRemoteStorage.java index f7e93622..18d21935 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IRemoteStorage.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IRemoteStorage.java @@ -27,6 +27,7 @@ public interface IRemoteStorage extends ICommonStorage { /** * generate sql sentence for session window + * * @param infoMap * @return */ diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IStorage.java index 074770d2..efd87348 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IStorage.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IStorage.java @@ -13,7 +13,8 @@ * 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.window.storage; + */ +package org.apache.rocketmq.streams.window.storage; import java.util.Collection; import java.util.Iterator; @@ -23,9 +24,9 @@ public interface IStorage { - /** * 支持单个key value的存储 + * * @param key * @param value */ @@ -38,18 +39,21 @@ public interface IStorage { T get(Class clazz, IKeyGenerator keyGenerator, String key); //获取多个key的值 - Map mutilGet(Class clazz, IKeyGenerator keyGenerator, String... keyValues); + Map mutilGet(Class clazz, IKeyGenerator keyGenerator, String... keyValues); + //获取多个key的值 - Map mutilGet(Class clazz, IKeyGenerator keyGenerator, List keys); + Map mutilGet(Class clazz, IKeyGenerator keyGenerator, List keys); /** * remove keys + * * @param keys */ void removeKeys(IKeyGenerator keyGenerator, Collection keys); /** * remove keys by prefix + * * @param keyPrefix */ void removeKeyPrefix(IKeyGenerator keyGenerator, String keyPrefix); @@ -59,20 +63,15 @@ public interface IStorage { */ Iterator iterateByPrefix(IKeyGenerator keyGenerator, String keyPrefix, Class clazz); - T putIfAbsent(T t, Class clazz); - int count(IKeyGenerator keyGenerator, String key); int incrementAndGet(IKeyGenerator keyGenerator, String key); - Iterator queryByPrefixBetweenOrderByValue(IKeyGenerator keyGenerator, String keyPrefix, Object startIndexValue, Object endIndexValue, Class clazz); - - void loadByPrefixBetweenOrderByValue(IKeyGenerator keyGenerator, String keyPrefix, Object startIndexValue, Object endIndexValue, IRowOperator rowOperator, Class clazz); diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/ShufflePartitionManager.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/ShufflePartitionManager.java index aa71a81c..95cb8781 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/ShufflePartitionManager.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/ShufflePartitionManager.java @@ -18,10 +18,6 @@ import java.util.HashMap; import java.util.Map; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; public class ShufflePartitionManager { private static ShufflePartitionManager instance = new ShufflePartitionManager(); @@ -77,7 +73,7 @@ public synchronized void clearWindowInstance(String windowInstanceId) { public synchronized void clearSplit(String queueId) { splitId2AllWindowInstanceFinishInit.remove(queueId); - Map map=new HashMap<>(this.windowInstanceId2FinishInit); + Map map = new HashMap<>(this.windowInstanceId2FinishInit); for (String windowInstanceId : map.keySet()) { if (windowInstanceId.startsWith(queueId)) { this.windowInstanceId2FinishInit.remove(windowInstanceId); diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowStorage.java index 82fef5c5..bdb42f8f 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowStorage.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowStorage.java @@ -42,17 +42,27 @@ public class WindowStorage extends AbstractWindowStorage { protected transient ShufflePartitionManager shufflePartitionManager = ShufflePartitionManager.getInstance(); - protected IWindowStorage localStorage; - protected IWindowStorage remoteStorage; + protected IWindowStorage localStorage; + protected IWindowStorage remoteStorage; - public WindowStorage(boolean isLoaclStorageOnly) { + public WindowStorage(boolean isLocalStorageOnly) { this(); - this.isLocalStorageOnly = isLoaclStorageOnly; + this.isLocalStorageOnly = isLocalStorageOnly; } public WindowStorage() { - localStorage = new RocksdbStorage(); - remoteStorage = new DBStorage(); + localStorage = new RocksdbStorage<>(); + remoteStorage = new DBStorage<>(); + } + + /** + * refer to: WindowMessageProcessor.createStoreKey + */ + public static String createWindowInstanceId(String msgKey) { + String[] values = MapKeyUtil.splitKey(msgKey); + String[] lastValues = Arrays.copyOfRange(values, 1, values.length - 2); + //values[4]: endTime or fireTime + return MapKeyUtil.createKey(lastValues); } @Override @@ -205,16 +215,6 @@ public void removeKeys(Collection keys) { localStorage.removeKeys(keys); } - /** - * refer to: WindowMessageProcessor.createStoreKey - */ - public static String createWindowInstanceId(String msgKey) { - String[] values = MapKeyUtil.splitKey(msgKey); - String[] lastValues = Arrays.copyOfRange(values, 1, values.length - 2); - //values[4]: endTime or fireTime - return MapKeyUtil.createKey(lastValues); - } - @Override public void delete(String windowInstanceId, String queueId, Class clazz) { this.delete(windowInstanceId, queueId, clazz, null); @@ -232,24 +232,24 @@ public void delete(String windowInstanceId, String queueId, Class clazz, SQLC } } - public static abstract class WindowBaseValueIterator implements Iterator { - protected long partitionNum = -1; - - public void setPartitionNum(long partitionNum) { - this.partitionNum = partitionNum; - } - } - @Override public void clearCache(ISplit split, Class clazz) { localStorage.clearCache(split, clazz); } - public IWindowStorage getLocalStorage() { + public IWindowStorage getLocalStorage() { return localStorage; } - public IWindowStorage getRemoteStorage() { + public IWindowStorage getRemoteStorage() { return remoteStorage; } + + public static abstract class WindowBaseValueIterator implements Iterator { + protected long partitionNum = -1; + + public void setPartitionNum(long partitionNum) { + this.partitionNum = partitionNum; + } + } } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/db/DBStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/db/DBStorage.java index afb18199..9a4c49db 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/db/DBStorage.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/db/DBStorage.java @@ -45,14 +45,14 @@ public class DBStorage extends AbstractWindowStorage< if (CollectionUtil.isEmpty(values)) { return null; } - String sql = ORMUtil.createBatchReplacetSQL(new ArrayList<>(values.values())); + String sql = ORMUtil.createBatchReplaceSQL(new ArrayList<>(values.values())); return sql; } @Override public String multiPutListSQL(Map> infoMap) { if (CollectionUtil.isNotEmpty(infoMap)) { List valueList = duplicate(infoMap); - return ORMUtil.createBatchReplacetSQL(valueList); + return ORMUtil.createBatchReplaceSQL(valueList); } return null; } @@ -207,16 +207,26 @@ public void delete(String windowInstanceId, String queueId, Class clazz) { return sql; } + protected Long getPartitionNum(String windowInstanceId, Class clazz, boolean isMax) { + String partitionNumSQL = isMax ? "max(partition_num)" : "min(partition_num)"; + String windowInstancePartitionId = StringUtil.createMD5Str(windowInstanceId); + String sql = "select " + partitionNumSQL + " as partition_num from " + ORMUtil.getTableName(clazz) + + " where window_instance_partition_id ='" + windowInstancePartitionId + "'"; + WindowBaseValue windowBaseValue = ORMUtil.queryForObject(sql, new HashMap<>(4), clazz); + if (windowBaseValue == null) { + return null; + } + return windowBaseValue.getPartitionNum(); + } + public static class DBIterator extends WindowBaseValueIterator { - private LinkedList container = new LinkedList<>(); int batchSize = 1000; + String sql; + private LinkedList container = new LinkedList<>(); private boolean exist = true; - private long maxPartitionIndex; private Class clazz; - String sql; - public DBIterator(String queueId, String windowInstanceId, String keyPrex, Class clazz, long maxPartitionIndex) { String windowInstancePartitionId = StringUtil.createMD5Str(windowInstanceId); @@ -267,16 +277,4 @@ public T next() { } - protected Long getPartitionNum(String windowInstanceId, Class clazz, boolean isMax) { - String partitionNumSQL = isMax ? "max(partition_num)" : "min(partition_num)"; - String windowInstancePartitionId = StringUtil.createMD5Str(windowInstanceId); - String sql = "select " + partitionNumSQL + " as partition_num from " + ORMUtil.getTableName(clazz) - + " where window_instance_partition_id ='" + windowInstancePartitionId + "'"; - WindowBaseValue windowBaseValue = ORMUtil.queryForObject(sql, new HashMap<>(4), clazz); - if (windowBaseValue == null) { - return null; - } - return windowBaseValue.getPartitionNum(); - } - } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocksdb/RocksdbStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocksdb/RocksdbStorage.java index cdce1746..4f8c7c51 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocksdb/RocksdbStorage.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocksdb/RocksdbStorage.java @@ -32,7 +32,6 @@ import org.apache.rocketmq.streams.common.utils.Base64Utils; import org.apache.rocketmq.streams.common.utils.CollectionUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; - import org.apache.rocketmq.streams.common.utils.SerializeUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.state.kv.rocksdb.RocksDBOperator; @@ -51,8 +50,35 @@ public class RocksdbStorage extends AbstractWindowSto protected static String DB_PATH = "/tmp/rocksdb"; protected static String UTF8 = "UTF8"; protected static AtomicBoolean hasCreate = new AtomicBoolean(false); - protected static RocksDB rocksDB = new RocksDBOperator().getInstance(); - protected WriteOptions writeOptions = new WriteOptions(); + protected static RocksDB rocksDB; + // protected WriteOptions writeOptions = new WriteOptions(); + + public RocksdbStorage() { + rocksDB = new RocksDBOperator().getInstance(); + } + + public RocksdbStorage(String rocksdbFileName) { + rocksDB = new RocksDBOperator(rocksdbFileName).getInstance(); + } + + /** + * 把byte转化成值 + * + * @param bytes + * @return + */ + protected static String getValueFromByte(byte[] bytes) { + try { + return new String(bytes, UTF8); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } + + public static void main(String[] args) { + String x = "2012-01-03 00:03:09"; + System.out.println(x.substring(x.length() - 2, x.length())); + } @Override public void removeKeys(Collection keys) { @@ -110,29 +136,24 @@ public void multiPut(Map values) { @Override public Map multiGet(Class clazz, List keys) { - if (keys == null || keys.size() == 0) { + if (keys == null || keys.isEmpty()) { return new HashMap<>(); } List keyByteList = new ArrayList<>(); - List keyStrList = new ArrayList<>(); for (String key : keys) { keyByteList.add(getKeyBytes(key)); - keyStrList.add(key); } try { - Map jsonables = new HashMap<>(); - // List list= rocksDB.multiGetAsList(keyByteList); - Map map = rocksDB.multiGet(keyByteList); - int i = 0; - Iterator> it = map.entrySet().iterator(); - while (it.hasNext()) { - Entry entry = it.next(); - String key = getValueFromByte(entry.getKey()); - T value = SerializeUtil.deserialize(entry.getValue(),clazz); - jsonables.put(key, value); + Map jsons = new HashMap<>(); + List valueByteList = rocksDB.multiGetAsList(keyByteList); + for (int i = 0; i < keyByteList.size(); i++) { + String key = getValueFromByte(keyByteList.get(i)); + if (valueByteList.get(i) != null) { + T value = SerializeUtil.deserialize(valueByteList.get(i), clazz); + jsons.put(key, value); + } } - // for(byte[] bytes:list){ - return jsonables; + return jsons; } catch (RocksDBException e) { throw new RuntimeException("can not get value from rocksdb ", e); } @@ -178,19 +199,16 @@ public Map multiGet(Class clazz, List keys) { } try { Map> resultMap = new HashMap<>(); - Map map = rocksDB.multiGet(keyByteList); - int i = 0; - Iterator> it = map.entrySet().iterator(); - while (it.hasNext()) { - Entry entry = it.next(); - String key = getValueFromByte(entry.getKey()); - String value = getValueFromByte(entry.getValue()); + List valueByteList = rocksDB.multiGetAsList(keyByteList); + for (int i = 0; i < keyByteList.size(); i++) { + String key = getValueFromByte(keyByteList.get(i)); + String value = getValueFromByte(valueByteList.get(i)); JSONArray array = JSONArray.parseArray(value); List valueList = new ArrayList<>(); for (int index = 0; index < array.size(); index++) { String objectString = array.getString(index); byte[] bytes = Base64Utils.decode(objectString); - T valueObject = SerializeUtil.deserialize(bytes,clazz); + T valueObject = SerializeUtil.deserialize(bytes, clazz); valueList.add(valueObject); } resultMap.put(key, valueList); @@ -245,14 +263,31 @@ protected WindowBaseValueIterator getByKeyPrefix(String keyPrefix, Class(keyPrefix, clazz, needKey); } + /** + * 把key转化成byte + * + * @param key + * @return + */ + protected byte[] getKeyBytes(String key) { + try { + if (StringUtil.isEmpty(key)) { + return null; + } + return key.getBytes(UTF8); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException("get bytes error ", e); + } + } + public static class LocalIterator extends WindowBaseValueIterator { protected volatile boolean hasNext = true; protected AtomicBoolean hasInit = new AtomicBoolean(false); - ReadOptions readOptions = new ReadOptions(); - private RocksIterator iter; protected String keyPrefix; protected Class clazz; protected boolean needKey; + ReadOptions readOptions = new ReadOptions(); + private RocksIterator iter; public LocalIterator(String keyPrefix, Class clazz, boolean needKey) { readOptions.setPrefixSameAsStart(true).setTotalOrderSeek(true); @@ -277,7 +312,7 @@ public T next() { hasNext = false; return null; } - T windowBaseValue = (T) SerializeUtil.deserialize(iter.value(),clazz); + T windowBaseValue = (T) SerializeUtil.deserialize(iter.value(), clazz); // T windowBaseValue = ReflectUtil.forInstance(clazz); // windowBaseValue.toObject(value); if (needKey) { @@ -296,40 +331,4 @@ public T next() { } } - - /** - * 把key转化成byte - * - * @param key - * @return - */ - protected byte[] getKeyBytes(String key) { - try { - if (StringUtil.isEmpty(key)) { - return null; - } - return key.getBytes(UTF8); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException("get bytes error ", e); - } - } - - /** - * 把byte转化成值 - * - * @param bytes - * @return - */ - protected static String getValueFromByte(byte[] bytes) { - try { - return new String(bytes, UTF8); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException(e); - } - } - - public static void main(String[] args) { - String x = "2012-01-03 00:03:09"; - System.out.println(x.substring(x.length() - 2, x.length())); - } } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocksdb/SelfAdaptionRocksdbStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocksdb/SelfAdaptionRocksdbStorage.java new file mode 100644 index 00000000..b1daee15 --- /dev/null +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocksdb/SelfAdaptionRocksdbStorage.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.window.storage.rocksdb; + +import java.util.Collection; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import org.apache.rocketmq.streams.window.state.WindowBaseValue; +import org.apache.rocketmq.streams.window.storage.WindowStorage; + +public class SelfAdaptionRocksdbStorage extends RocksdbStorage { + + protected boolean isCacheModel = true;//刚启动,默认是cache模式,当数据量比较大时,会降级成rocksdb模式,可以减少序列化和反序列化的消耗 + protected int cacheMaxSize = 200000; + protected TreeMap cache = new TreeMap(new Comparator() { + + @Override public int compare(String o1, String o2) { + return o1.compareTo(o2); + } + }); + protected TreeMap> cache2ValueList = new TreeMap(new Comparator() { + + @Override public int compare(String o1, String o2) { + return o1.compareTo(o2); + } + }); + + public static void main(String[] args) { + TreeMap cache = new TreeMap(new Comparator() { + + @Override public int compare(String o1, String o2) { + return o1.compareTo(o2); + } + }); + cache.put("1", 1); + cache.put("9", 2); + cache.put("5", 5); + cache.put("24", 4); + cache.put("23", 3); + cache.put("6", 6); + Map.Entry entry = cache.ceilingEntry("2"); + + System.out.println(entry.getKey()); + Iterator iterator = cache.keySet().iterator(); + + while (iterator.hasNext()) { + String key = (String) iterator.next(); + System.out.println(key); + } + + } + + @Override + protected synchronized void deleteRange(String startKey, Class clazz) { + if (!isCacheModel) { + super.deleteRange(startKey, clazz); + return; + } + Map prefixMap = this.getPrefixSubMap(startKey, this.cache); + if (prefixMap != null) { + for (String key : prefixMap.keySet()) { + cache.remove(key); + } + } + + Map prefixMapForList = this.getPrefixSubMap(startKey, this.cache2ValueList); + if (prefixMap != null) { + for (String key : prefixMapForList.keySet()) { + this.cache2ValueList.remove(key); + } + } + + } + + @Override + public synchronized WindowStorage.WindowBaseValueIterator getByKeyPrefix(String keyPrefix, Class clazz, boolean needKey) { + if (!isCacheModel) { + return new LocalIterator(keyPrefix, clazz, needKey); + } + + Map prefixMap = this.getPrefixSubMap(keyPrefix, this.cache); + if (prefixMap == null) { + return null; + } + Iterator> it = prefixMap.entrySet().iterator(); + return new WindowStorage.WindowBaseValueIterator() { + protected boolean hasNext = true; + + @Override public boolean hasNext() { + return it.hasNext() && hasNext; + } + + @Override public T next() { + Map.Entry entry = it.next(); + String key = entry.getKey(); + T windowBaseValue = entry.getValue(); + if (needKey) { + windowBaseValue.setMsgKey(key); + } + while (windowBaseValue.getPartitionNum() < this.partitionNum) { + entry = it.next(); + key = entry.getKey(); + windowBaseValue = entry.getValue(); + if (windowBaseValue == null) { + hasNext = false; + return null; + } + } + if (windowBaseValue.getPartitionNum() < this.partitionNum) { + entry = it.next(); + } + windowBaseValue = entry.getValue(); + return windowBaseValue; + } + }; + } + + @Override public synchronized void multiPut(Map map) { + if (!isCacheModel) { + super.multiPut(map); + return; + } + + this.cache.putAll(map); + if (getCacheSize() > this.cacheMaxSize) { + this.isCacheModel = false; + synCache2RocksDB(); + } + + } + + @Override synchronized public void multiPutList(Map> elements) { + if (!isCacheModel) { + super.multiPutList(elements); + return; + } + + this.cache2ValueList.putAll(elements); + if (getCacheSize() > this.cacheMaxSize) { + this.isCacheModel = false; + synCache2RocksDB(); + } + } + + @Override public synchronized Map multiGet(Class clazz, List keys) { + if (!isCacheModel) { + return super.multiGet(clazz, keys); + } + if (keys == null) { + return new HashMap<>(); + } + Map result = new HashMap<>(); + for (String key : keys) { + T value = this.cache.get(key); + if (value != null) { + result.put(key, value); + } + + } + return result; + } + + @Override public synchronized Map> multiGetList(Class clazz, List keys) { + if (!isCacheModel) { + return super.multiGetList(clazz, keys); + } + if (keys == null) { + return new HashMap<>(); + } + Map> result = new HashMap<>(); + for (String key : keys) { + List value = this.cache2ValueList.get(key); + if (value != null) { + result.put(key, value); + } + } + return result; + } + + @Override public void removeKeys(Collection keys) { + if (!isCacheModel) { + super.removeKeys(keys); + return; + } + + if (keys == null) { + return; + } + for (String key : keys) { + this.cache2ValueList.remove(key); + this.cache.remove(key); + } + + } + + private int getCacheSize() { + return this.cache.size() + this.cache2ValueList.size(); + } + + private void synCache2RocksDB() { + if (this.cache != null) { + super.multiPut(this.cache); + this.cache.clear(); + } + if (this.cache2ValueList != null) { + super.multiPutList(this.cache2ValueList); + this.cache2ValueList.clear(); + } + + } + + protected Map getPrefixSubMap(String prefix, TreeMap treeMap) { + Iterator iterator = treeMap.keySet().iterator(); + Map result = new HashMap<>(); + boolean isMatchPrefix = false; + while (iterator.hasNext()) { + String key = iterator.next(); + if (key.startsWith(prefix)) { + isMatchPrefix = true; + result.put(key, treeMap.get(key)); + } else { + if (isMatchPrefix) { + break; + } + } + } + return result; + } + +} diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/trigger/WindowTrigger.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/trigger/WindowTrigger.java index 69b6494a..d87b1022 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/trigger/WindowTrigger.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/trigger/WindowTrigger.java @@ -23,89 +23,79 @@ import java.util.List; import java.util.PriorityQueue; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageFlushCallBack; import org.apache.rocketmq.streams.common.channel.sinkcache.impl.AbstractMultiSplitMessageCache; import org.apache.rocketmq.streams.common.channel.sinkcache.impl.MessageCache; -import org.apache.rocketmq.streams.common.channel.source.AbstractSupportShuffleSource; import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.context.AbstractContext; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; +import org.apache.rocketmq.streams.common.threadpool.ScheduleFactory; import org.apache.rocketmq.streams.common.utils.DateUtil; import org.apache.rocketmq.streams.window.debug.DebugWriter; import org.apache.rocketmq.streams.window.model.WindowInstance; +import org.apache.rocketmq.streams.window.operator.AbstractShuffleWindow; import org.apache.rocketmq.streams.window.operator.AbstractWindow; import org.apache.rocketmq.streams.window.operator.impl.SessionOperator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class WindowTrigger extends AbstractSupportShuffleSource implements IStreamOperator { - protected static final Log LOG = LogFactory.getLog(WindowTrigger.class); - private AbstractWindow window; +public class WindowTrigger { + private static final Logger LOGGER = LoggerFactory.getLogger(WindowTrigger.class); //这个时间是在于数据很离散,无法触发窗口的时候做的补位 protected transient Long eventTimeLastUpdateTime; - protected transient ScheduledExecutorService fireCheckScheduler;//检查窗口实例是否可以触发 protected transient ConcurrentHashMap windowInstances = new ConcurrentHashMap();//保存所有注册的窗口实例,多个相同实例注册,只保留一个 + protected Object locker = new Object(); //所有注册的窗口实例,按触发顺序排序,如果触发时间相同,按开始时间排序 protected transient PriorityQueue orderWindowInstancs = new PriorityQueue(new Comparator() { - @Override - public int compare(WindowInstance o1, WindowInstance o2) { - int value = o1.getFireTime().compareTo(o2.getFireTime()); - if (value != 0) { - return value; + @Override public int compare(WindowInstance o1, WindowInstance o2) { + synchronized (locker) { + int value = o1.getFireTime().compareTo(o2.getFireTime()); + if (value != 0) { + return value; + } + return o2.getStartTime().compareTo(o1.getStartTime()); } - return o2.getStartTime().compareTo(o1.getStartTime()); + } }); - //所有可以触发的窗口实例,放到缓存中,有缓存线程调度执行触发逻辑,同一个分片的窗口实例按顺序串行触发,各个分片可以并行 protected transient MessageCache fireInstanceCache = new WindowInstanceCache(); //正在触发中的windowintance protected transient ConcurrentHashMap firingWindowInstances = new ConcurrentHashMap<>(); + private AbstractShuffleWindow window; - public WindowTrigger(AbstractWindow window) { + public WindowTrigger(AbstractShuffleWindow window) { this.window = window; } - @Override - protected boolean initConfigurable() { - fireCheckScheduler = new ScheduledThreadPoolExecutor(2); - setReceiver(window.getFireReceiver()); - fireInstanceCache.openAutoFlush(); - return super.initConfigurable(); - } - - @Override - protected boolean startSource() { + public boolean start() { //检查window instance,如果已经到了触发时间,且符合触发条件,直接触发,如果到了触发时间,还未符合触发条件。则放入触发列表。下次调度时间是下一个最近触发的时间 - - fireCheckScheduler.scheduleWithFixedDelay(new Runnable() { - @Override - public void run() { + fireInstanceCache.openAutoFlush(); + ScheduleFactory.getInstance().execute(window.getNameSpace() + "-" + window.getName() + "-win_trigger_schedule", new Runnable() { + @Override public void run() { try { if (orderWindowInstancs.size() == 0) { return; } - WindowInstance windowInstance = orderWindowInstancs.peek(); - while (windowInstance != null) { - boolean isStartNow = false; - if (SessionOperator.SESSION_WINDOW_BEGIN_TIME.equalsIgnoreCase(windowInstance.getStartTime())) { - isStartNow = true; - } - boolean success = executeFireTask(windowInstance, isStartNow); - if (success) { - windowInstances.remove(windowInstance.createWindowInstanceTriggerId()); - orderWindowInstancs.remove(windowInstance); - windowInstance = orderWindowInstancs.peek(); - } else { - break; - } + synchronized (locker) { + WindowInstance windowInstance = orderWindowInstancs.peek(); + while (windowInstance != null) { + boolean isStartNow = false; + if (SessionOperator.SESSION_WINDOW_BEGIN_TIME.equalsIgnoreCase(windowInstance.getStartTime())) { + isStartNow = true; + } + boolean success = executeFireTask(windowInstance, isStartNow); + if (success) { + windowInstances.remove(windowInstance.createWindowInstanceTriggerId()); + orderWindowInstancs.remove(windowInstance); + windowInstance = orderWindowInstancs.peek(); + } else { + break; + } + } } + } catch (Exception e) { e.printStackTrace(); } @@ -127,7 +117,7 @@ public void registFireWindowInstanceIfNotExist(WindowInstance windowInstance, Ab window.registerWindowInstance(windowInstance); offerWindowInstance(windowInstance); } - LOG.debug("register window instance into manager, instance key: " + windowInstanceTriggerId); + LOGGER.debug("register window instance into manager, instance key: " + windowInstanceTriggerId); } /** @@ -149,7 +139,7 @@ protected void offerWindowInstance(WindowInstance windowInstance) { if (this.firingWindowInstances.containsKey(triggerId)) { return; } - synchronized (this) { + synchronized (locker) { if (this.firingWindowInstances.containsKey(windowInstance.createWindowInstanceTriggerId())) { return; } @@ -174,7 +164,7 @@ public boolean executeFireTask(WindowInstance windowInstance, boolean startNow) return true; } //start firing - DebugWriter.getDebugWriter(window.getConfigureName()).writeFireWindowInstance(windowInstance, eventTimeLastUpdateTime, this.window.getMaxEventTime(windowInstance.getSplitId()), fireResult.getReason()); + DebugWriter.getDebugWriter(window.getName()).writeFireWindowInstance(windowInstance, eventTimeLastUpdateTime, this.window.getMaxEventTime(windowInstance.getSplitId()), fireResult.getReason()); firingWindowInstances.put(windowInstanceTriggerId, windowInstance); if (startNow) { fireWindowInstance(windowInstance); @@ -194,20 +184,20 @@ public boolean executeFireTask(WindowInstance windowInstance, boolean startNow) protected void fireWindowInstance(WindowInstance windowInstance) { try { if (windowInstance == null) { - LOG.error("window instance is null!"); + LOGGER.error("window instance is null!"); return; } String windowInstanceTriggerId = windowInstance.createWindowInstanceTriggerId(); if (window == null) { - LOG.error(windowInstanceTriggerId + "'s window object have been removed!"); + LOGGER.error(windowInstanceTriggerId + "'s window object have been removed!"); return; } if (windowInstance.getLastMaxUpdateTime() == null) { windowInstance.setLastMaxUpdateTime(window.getMaxEventTime(windowInstance.getSplitId())); } - int fireCount = window.fireWindowInstance(windowInstance, null); - LOG.debug("fire instance(" + windowInstanceTriggerId + " fire count is " + fireCount); + int fireCount = window.fire(windowInstance); + LOGGER.debug("fire instance(" + windowInstanceTriggerId + " fire count is " + fireCount); firingWindowInstances.remove(windowInstanceTriggerId); } catch (Exception e) { e.printStackTrace(); @@ -223,7 +213,7 @@ protected void fireWindowInstance(WindowInstance windowInstance) { protected FireResult canFire(WindowInstance windowInstance) { String windowInstanceTriggerId = windowInstance.createWindowInstanceTriggerId(); if (window == null) { - LOG.warn(windowInstanceTriggerId + " can't find window!"); + LOGGER.warn(windowInstanceTriggerId + " can't find window!"); return new FireResult(); } Date fireTime = DateUtil.parseTime(windowInstance.getFireTime()); @@ -251,18 +241,13 @@ protected FireResult canFire(WindowInstance windowInstance) { if (isTest) { int gap = (int) (System.currentTimeMillis() - eventTimeLastUpdateTime); if (window.getMsgMaxGapSecond() != null && gap > window.getMsgMaxGapSecond() * 1000) { - LOG.warn("the fire reason is exceed the gap " + gap + " window instance id is " + windowInstanceTriggerId); + LOGGER.warn("the fire reason is exceed the gap " + gap + " window instance id is " + windowInstanceTriggerId); return new FireResult(true, 1); } } return new FireResult(); } - @Override - public Object doMessage(IMessage message, AbstractContext context) { - return null; - } - public synchronized void fireWindowInstance(String queueId) { List windowInstanceList = new ArrayList<>(); ConcurrentHashMap newWindowInstanceMap = new ConcurrentHashMap(); @@ -277,8 +262,7 @@ public synchronized void fireWindowInstance(String queueId) { } windowInstances = newWindowInstanceMap; Collections.sort(windowInstanceList, new Comparator() { - @Override - public int compare(WindowInstance o1, WindowInstance o2) { + @Override public int compare(WindowInstance o1, WindowInstance o2) { int value = o1.getFireTime().compareTo(o2.getFireTime()); if (value != 0) { return value; @@ -287,21 +271,23 @@ public int compare(WindowInstance o1, WindowInstance o2) { } }); for (WindowInstance windowInstance : windowInstanceList) { - // System.out.println("fire by finish flag"); + // System.out.println("fire by finish flag"); fireWindowInstance(windowInstance); } } + public void destroy() { + ScheduleFactory.getInstance().cancel(window.getNameSpace() + "-" + window.getName() + "-win_trigger_schedule"); + } + protected class WindowInstanceCache extends AbstractMultiSplitMessageCache { public WindowInstanceCache() { super(new IMessageFlushCallBack() { - @Override - public boolean flushMessage(List windowInstances) { + @Override public boolean flushMessage(List windowInstances) { Collections.sort(windowInstances, new Comparator() { - @Override - public int compare(WindowInstance o1, WindowInstance o2) { + @Override public int compare(WindowInstance o1, WindowInstance o2) { int value = o1.getFireTime().compareTo(o2.getFireTime()); if (value != 0) { return value; @@ -318,8 +304,7 @@ public int compare(WindowInstance o1, WindowInstance o2) { }); } - @Override - protected String createSplitId(WindowInstance windowInstance) { + @Override protected String createSplitId(WindowInstance windowInstance) { return windowInstance.getSplitId(); } } @@ -347,23 +332,4 @@ public int getReason() { } } - @Override - public boolean supportNewSplitFind() { - return true; - } - - @Override - public boolean supportRemoveSplitFind() { - return false; - } - - @Override - public boolean supportOffsetRest() { - return false; - } - - @Override - protected boolean isNotDataSplit(String queueId) { - return false; - } } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/util/ShuffleUtil.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/util/ShuffleUtil.java index 94d76055..0d5ce404 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/util/ShuffleUtil.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/util/ShuffleUtil.java @@ -16,25 +16,25 @@ */ package org.apache.rocketmq.streams.window.util; -import com.alibaba.fastjson.JSONArray; import com.alibaba.fastjson.JSONObject; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.topology.shuffle.IShuffleKeyGenerator; +import org.apache.rocketmq.streams.common.context.MessageHeader; import org.apache.rocketmq.streams.common.utils.StringUtil; -import org.apache.rocketmq.streams.window.model.WindowCache; +import org.apache.rocketmq.streams.window.WindowConstants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ShuffleUtil { - private static final Log LOG = LogFactory.getLog(ShuffleUtil.class); - public static IMessage createShuffleMsg(IMessage msg, String shuffleKey,JSONObject msgHeader){ + private static final Logger LOGGER = LoggerFactory.getLogger(ShuffleUtil.class); + + public static IMessage createShuffleMsg(IMessage msg, String shuffleKey, JSONObject msgHeader) { if (msg.getHeader().isSystemMessage()) { return null; } if (StringUtil.isEmpty(shuffleKey)) { shuffleKey = ""; - LOG.debug("there is no group by value in message! " + msg.getMessageBody().toString()); + LOGGER.debug("there is no group by value in message! " + msg.getMessageBody().toString()); //continue; } @@ -42,21 +42,29 @@ public static IMessage createShuffleMsg(IMessage msg, String shuffleKey,JSONObje String offset = msg.getHeader().getOffset(); String queueId = msg.getHeader().getQueueId(); - body.put(WindowCache.ORIGIN_OFFSET, offset); - body.put(WindowCache.ORIGIN_QUEUE_ID,queueId); - body.put(WindowCache.ORIGIN_QUEUE_IS_LONG, msg.getHeader().getMessageOffset().isLongOfMainOffset()); - if(msgHeader==null){ - body.put(WindowCache.ORIGIN_MESSAGE_HEADER, JSONObject.toJSONString(msg.getHeader())); - }else { - body.put(WindowCache.ORIGIN_MESSAGE_HEADER, msgHeader); + body.put(WindowConstants.ORIGIN_OFFSET, offset); + body.put(WindowConstants.ORIGIN_QUEUE_ID, queueId); + body.put(WindowConstants.ORIGIN_QUEUE_IS_LONG, msg.getHeader().getMessageOffset().isLongOfMainOffset()); + if (msgHeader == null) { + body.put(WindowConstants.ORIGIN_MESSAGE_HEADER, JSONObject.toJSONString(msg.getHeader())); + } else { + body.put(WindowConstants.ORIGIN_MESSAGE_HEADER, msgHeader); } - body.put(WindowCache.ORIGIN_MESSAGE_TRACE_ID, msg.getHeader().getTraceId()); - body.put(WindowCache.SHUFFLE_KEY, shuffleKey); + body.put(WindowConstants.ORIGIN_MESSAGE_TRACE_ID, msg.getHeader().getTraceId()); + body.put(WindowConstants.SHUFFLE_KEY, shuffleKey); return msg; } - public static IMessage createShuffleMsg(IMessage msg, String shuffleKey){ - return createShuffleMsg(msg,shuffleKey,null); + public static MessageHeader getMessageHeader(JSONObject msg) { + + String headerStr = msg.getString(WindowConstants.ORIGIN_MESSAGE_HEADER); + MessageHeader header = JSONObject.toJavaObject(JSONObject.parseObject(headerStr), MessageHeader.class); + + return header; + } + + public static IMessage createShuffleMsg(IMessage msg, String shuffleKey) { + return createShuffleMsg(msg, shuffleKey, null); } } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/util/WindowChannellUtil.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/util/WindowChannellUtil.java index 7354ad2e..7618ddd4 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/util/WindowChannellUtil.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/util/WindowChannellUtil.java @@ -26,6 +26,7 @@ import org.apache.rocketmq.streams.common.channel.impl.memory.MemorySource; import org.apache.rocketmq.streams.common.channel.source.ISource; import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.configuration.SystemContext; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.serviceloader.ServiceLoaderComponent; @@ -41,7 +42,7 @@ public class WindowChannellUtil { * @return */ protected static IChannelBuilder createBuilder(String connector) { - String type = ComponentCreator.getProperties().getProperty(connector); + String type = SystemContext.getProperty(connector); if (StringUtil.isEmpty(type)) { return null; } diff --git a/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/RocksdbTest.java b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/RocksdbTest.java index 992f0bb3..8ba60603 100644 --- a/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/RocksdbTest.java +++ b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/RocksdbTest.java @@ -73,7 +73,7 @@ public void testLen() { @Test public void testSer() throws UnsupportedEncodingException { long start = System.currentTimeMillis(); - for (int i = 0; i < 1000000; i++) { + for (int i = 0; i < 10000000; i++) { Person person = Person.createPerson("NAMESPACE" + i); byte[] bytes = SerializeUtil.serialize(person); person = SerializeUtil.deserialize(bytes); diff --git a/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/SelfAdaptionRocksdbStorageTest.java b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/SelfAdaptionRocksdbStorageTest.java new file mode 100644 index 00000000..656115c7 --- /dev/null +++ b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/SelfAdaptionRocksdbStorageTest.java @@ -0,0 +1,99 @@ +/* + * 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; + +import com.alibaba.fastjson.JSONObject; +import java.io.UnsupportedEncodingException; +import java.util.HashSet; +import java.util.Set; +import org.apache.rocketmq.streams.window.state.impl.WindowValue; +import org.apache.rocketmq.streams.window.storage.WindowStorage; +import org.apache.rocketmq.streams.window.storage.rocksdb.RocksdbStorage; +import org.apache.rocketmq.streams.window.storage.rocksdb.SelfAdaptionRocksdbStorage; +import org.junit.Test; +import org.rocksdb.RocksDBException; + +public class SelfAdaptionRocksdbStorageTest { + + @Test + public void testSelfAdaptionRocksdbStorage() { + SelfAdaptionRocksdbStorage storage = new SelfAdaptionRocksdbStorage(); + execute(storage); + } + + /** + * 1000W:179901 + * 1000w(随机读):255052ms + * 2000w(随机读):551183ms + * 1000W(NO Read):134853ms + * 10000w(随机读):9263000 + * + * @throws UnsupportedEncodingException + * @throws RocksDBException + */ + @Test + public void testCPUCost() throws UnsupportedEncodingException, RocksDBException { + RocksdbStorage rocksDB = new RocksdbStorage(); + execute(rocksDB); + } + + protected void execute(RocksdbStorage storage) { + long start = System.currentTimeMillis(); + int size = 200000; + for (int i = 0; i < size; i++) { + WindowValue windowValue = new WindowValue(); + + JSONObject msg = new JSONObject(); + msg.put("name", "chris" + i); + msg.put("age", i); + msg.put("address", "fddfdsfdsffddsdfsfdsfFDFDFDSDFSFDDFDSFSfddsf434234324234324234323423423423243243423443324342432" + + "99887787878776767好iuhnkjsaddasddassaddsadsdsdssddsadasdasdsddsadsnkjjk67768767678786867fdsdsffdsfdfdsfdsfd" + i); + windowValue.setAggColumnMap(msg); + storage.put(msg.getString("name"), windowValue); + } + Set keys = new HashSet<>(); + keys.add("chris" + 10); + keys.add("chris" + 100); + keys.add("chris" + 1000); + keys.add("chris" + 10000); + storage.removeKeys(keys); + + WindowStorage.WindowBaseValueIterator iterator = null;// storage.getByKeyPrefix("chris",WindowValue.class,true); + int i = 0; + while (iterator.hasNext()) { + + WindowValue windowValue = iterator.next(); + +// if(!windowValue.getAggColumnResultByKey("name").equals("chris"+i)){ +// System.out.println(i); +// } +// +// if(!windowValue.getAggColumnResultByKey("age").equals(i)){ +// System.out.println(i); +// } +// +// if(!windowValue.getAggColumnResultByKey("address").equals("fddfdsfdsffddsdfsfdsfFDFDFDSDFSFDDFDSFSfddsf434234324234324234323423423423243243423443324342432" + +// "99887787878776767好iuhnkjsaddasddassaddsadsdsdssddsadasdasdsddsadsnkjjk67768767678786867fdsdsffdsfdfdsfdsfd" + i)){ +// System.out.println(i); +// } + i++; + } + System.out.println(i); + System.out.println("finish cost is " + (System.currentTimeMillis() - start)); + + } +} diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/IntelligenceNameListFunction.java b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/WindowTriggerTest.java similarity index 82% rename from rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/IntelligenceNameListFunction.java rename to rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/WindowTriggerTest.java index 02d67fb3..633255bb 100644 --- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/IntelligenceNameListFunction.java +++ b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/WindowTriggerTest.java @@ -14,11 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.dim.function.script; +package org.apache.rocketmq.streams; -import org.apache.rocketmq.streams.script.annotation.Function; +import org.junit.Test; -@Function -public class IntelligenceNameListFunction { +public class WindowTriggerTest { + @Test + public void testWindowTrigger() { + } } diff --git a/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/configurable/model/Person.java b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/configurable/model/Person.java index 29aa27d7..5df61a64 100644 --- a/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/configurable/model/Person.java +++ b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/configurable/model/Person.java @@ -33,7 +33,7 @@ public static Person createPerson(String namespace) { Person person = new Person(); person.setNameSpace(namespace); person.setType("person"); - person.setConfigureName("Chris"); + person.setName("Chris"); person.setName("Chris"); List addresses = new ArrayList<>(); addresses.add("huilongguan"); diff --git a/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/window/WindowInstanceTest.java b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/window/WindowInstanceTest.java index e427ce3f..2c3f5171 100644 --- a/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/window/WindowInstanceTest.java +++ b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/window/WindowInstanceTest.java @@ -43,7 +43,7 @@ public class WindowInstanceTest { public void testWindowInstanceSave() { WindowOperator window = new WindowOperator(); window.setNameSpace("namespace_chris"); - window.setConfigureName("name"); + window.setName("name"); WindowInstance windowInstance = window.createWindowInstance("2021-07-09 11:00:00", "2021-07-09 11:10:00", "2021-07-09 11:10:00", "1"); ORMUtil.batchReplaceInto(windowInstance); WindowInstance queryWindowInstance = ORMUtil.queryForObject("select * from window_instance where window_instance_key='" + windowInstance.getWindowInstanceKey() + "'", null, WindowInstance.class); @@ -54,7 +54,6 @@ public void testWindowInstanceSave() { public void testWindowInstanceNormalMode() { WindowOperator window = new WindowOperator(); window.init(); -// window.doProcessAfterRefreshConfigurable(null); window.setFireMode(0); window.setTimeFieldName("time"); window.setSlideInterval(5); @@ -62,7 +61,7 @@ public void testWindowInstanceNormalMode() { window.setWaterMarkMinute(5); window.setEmitAfterValue(60L); window.setNameSpace("namespace_chris"); - window.setConfigureName("name"); + window.setName("name"); JSONObject msg = new JSONObject(); msg.put("time", "2021-07-09 11:00:01"); List windowInstances = window.queryOrCreateWindowInstance(new Message(msg), "1"); @@ -78,7 +77,7 @@ public void testWindowInstanceMode1() { window.setSizeInterval(5); window.setWaterMarkMinute(1000); window.setNameSpace("namespace_chris"); - window.setConfigureName("name"); + window.setName("name"); JSONObject msg = new JSONObject(); msg.put("time", "2021-07-09 11:00:01"); List windowInstances = window.queryOrCreateWindowInstance(new Message(msg), "1"); @@ -99,7 +98,7 @@ public void testWindowInstanceMode1ExceedWaterMark() { window.setSizeInterval(5); window.setWaterMarkMinute(5); window.setNameSpace("namespace_chris"); - window.setConfigureName("name"); + window.setName("name"); JSONObject msg = new JSONObject(); msg.put("time", "2021-07-09 11:00:01"); List windowInstances = window.queryOrCreateWindowInstance(new Message(msg), "1"); @@ -116,7 +115,7 @@ public void testWindowInstanceMode2() { window.setSizeInterval(5); window.setWaterMarkMinute(1000); window.setNameSpace("namespace_chris"); - window.setConfigureName("name"); + window.setName("name"); JSONObject msg = new JSONObject(); msg.put("time", "2021-07-09 11:00:01"); List windowInstances = window.queryOrCreateWindowInstance(new Message(msg), "1"); @@ -199,7 +198,7 @@ public void testHitWindowInstance() { window.setSizeInterval(5); window.setWaterMarkMinute(0); window.setNameSpace("namespace_test"); - window.setConfigureName("window_name"); + window.setName("window_name"); JSONObject msg = new JSONObject(); String eventTime = "2021-08-27 18:03:00"; @@ -232,7 +231,7 @@ public void testHitWindowInstance() { window.setSizeInterval(5); window.setWaterMarkMinute(120); window.setNameSpace("namespace_test"); - window.setConfigureName("window_name"); + window.setName("window_name"); msg = new JSONObject(); eventTime = "2021-08-27 17:03:00"; @@ -264,7 +263,7 @@ public void testHitWindowInstance() { window.setSizeInterval(5); window.setWaterMarkMinute(120); window.setNameSpace("namespace_test"); - window.setConfigureName("window_name"); + window.setName("window_name"); msg = new JSONObject(); eventTime = "2021-08-27 17:03:00"; diff --git a/rocketmq-streams-window/src/test/resources/log4j.xml b/rocketmq-streams-window/src/test/resources/log4j.xml old mode 100755 new mode 100644