summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--config/grootstream.yaml19
-rw-r--r--config/grootstream_job_example.yaml10
-rw-r--r--config/template/grootstream_job_template.yaml15
-rw-r--r--config/template/mock_schema/session_record_mock_desc.json76
-rw-r--r--config/udf.plugins10
-rw-r--r--docs/connector/config-encryption-decryption.md8
-rw-r--r--docs/connector/connector.md27
-rw-r--r--docs/connector/formats/csv.md73
-rw-r--r--docs/connector/formats/raw.md2
-rw-r--r--docs/connector/sink/kafka.md27
-rw-r--r--docs/connector/sink/starrocks.md83
-rw-r--r--docs/connector/source/kafka.md7
-rw-r--r--docs/develop-guide.md22
-rw-r--r--docs/env-config.md3
-rw-r--r--docs/grootstream-config.md45
-rw-r--r--docs/grootstream-design-cn.md2194
-rw-r--r--docs/images/groot_stream_architecture.jpgbin5263679 -> 5472871 bytes
-rw-r--r--docs/processor/aggregate-processor.md23
-rw-r--r--docs/processor/split-processor.md49
-rw-r--r--docs/processor/udaf.md33
-rw-r--r--docs/processor/udf.md132
-rw-r--r--docs/processor/udtf.md56
-rw-r--r--groot-bootstrap/pom.xml14
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/AES128GCM96Shade.java72
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/AES256GCM96Shade.java72
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/AESShade.java (renamed from groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/AESConfigShade.java)9
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/Base64Shade.java (renamed from groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/Base64ConfigShade.java)4
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/ConfDecryptCommand.java2
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/ConfEncryptCommand.java5
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/ExecuteCommandArgs.java1
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/SM4GCM96Shade.java73
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/SM4Shade.java (renamed from groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/SM4ConfigShade.java)6
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/enums/TargetType.java1
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractExecutor.java17
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractProcessorExecutor.java203
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/FilterExecutor.java72
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/JobExecution.java46
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/JobRuntimeEnvironment.java5
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/Node.java1
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/PostprocessingExecutor.java2
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/PreprocessingExecutor.java2
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/ProcessingExecutor.java2
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/SplitExecutor.java62
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/utils/ConfigBuilder.java4
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/utils/CryptoShadeUtils.java (renamed from groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/utils/ConfigShadeUtils.java)36
-rw-r--r--groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/utils/EnvironmentUtil.java10
-rw-r--r--groot-bootstrap/src/main/resources/META-INF/services/com.geedgenetworks.common.config.ConfigShade3
-rw-r--r--groot-bootstrap/src/main/resources/META-INF/services/com.geedgenetworks.common.crypto.CryptoShade6
-rw-r--r--groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/GrootStreamServerTest.java54
-rw-r--r--groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobAggTest.java (renamed from groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobSplitWithAggTest.java)35
-rw-r--r--groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobDosTest.java93
-rw-r--r--groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobEtlTest.java (renamed from groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/SimpleJobTest.java)40
-rw-r--r--groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobExecutionTest.java327
-rw-r--r--groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobSplitTest.java13
-rw-r--r--groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/utils/ConfigShadeTest.java81
-rw-r--r--groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/utils/CryptoShadeTest.java106
-rw-r--r--groot-bootstrap/src/test/resources/META-INF/services/com.geedgenetworks.common.config.ConfigShade2
-rw-r--r--groot-bootstrap/src/test/resources/META-INF/services/com.geedgenetworks.common.crypto.CryptoShade6
-rw-r--r--groot-bootstrap/src/test/resources/grootstream_job_agg_test.yaml (renamed from groot-bootstrap/src/test/resources/grootstream_job_split_agg_test.yaml)31
-rw-r--r--groot-bootstrap/src/test/resources/grootstream_job_dos_test.yaml130
-rw-r--r--groot-bootstrap/src/test/resources/grootstream_job_etl_test.yaml39
-rw-r--r--groot-bootstrap/src/test/resources/grootstream_job_split_test.yaml9
-rw-r--r--groot-common/src/main/java/com/geedgenetworks/common/Constants.java8
-rw-r--r--groot-common/src/main/java/com/geedgenetworks/common/Event.java1
-rw-r--r--groot-common/src/main/java/com/geedgenetworks/common/config/CheckConfigUtil.java42
-rw-r--r--groot-common/src/main/java/com/geedgenetworks/common/config/CheckResult.java70
-rw-r--r--groot-common/src/main/java/com/geedgenetworks/common/config/CheckUDFContextUtil.java107
-rw-r--r--groot-common/src/main/java/com/geedgenetworks/common/config/CommonConfig.java17
-rw-r--r--groot-common/src/main/java/com/geedgenetworks/common/config/CommonConfigDomProcessor.java91
-rw-r--r--groot-common/src/main/java/com/geedgenetworks/common/config/CommonConfigOptions.java82
-rw-r--r--groot-common/src/main/java/com/geedgenetworks/common/config/KmsConfig.java15
-rw-r--r--groot-common/src/main/java/com/geedgenetworks/common/config/KnowledgeBaseConfig.java19
-rw-r--r--groot-common/src/main/java/com/geedgenetworks/common/config/SSLConfig.java16
-rw-r--r--groot-common/src/main/java/com/geedgenetworks/common/config/UDFContextConfigOptions.java54
-rw-r--r--groot-common/src/main/java/com/geedgenetworks/common/crypto/CryptoShade.java (renamed from groot-common/src/main/java/com/geedgenetworks/common/config/ConfigShade.java)6
-rw-r--r--groot-common/src/main/java/com/geedgenetworks/common/exception/CommonErrorCode.java13
-rw-r--r--groot-common/src/main/java/com/geedgenetworks/common/udf/RuleContext.java2
-rw-r--r--groot-common/src/main/java/com/geedgenetworks/common/udf/ScalarFunction.java15
-rw-r--r--groot-common/src/main/java/com/geedgenetworks/common/udf/UDFContext.java57
-rw-r--r--groot-common/src/main/resources/grootstream.yaml18
-rw-r--r--groot-common/src/main/resources/udf.plugins13
-rw-r--r--groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/EventKafkaDeserializationSchema.java12
-rw-r--r--groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/KafkaConnectorOptionsUtil.java17
-rw-r--r--groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/KafkaSinkProvider.java5
-rw-r--r--groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/KafkaTableFactory.java8
-rw-r--r--groot-connectors/connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/GrootFlinkKafkaProducer.java23
-rw-r--r--groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/FakerUtils.java3
-rw-r--r--groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/SequenceFaker.java16
-rw-r--r--groot-connectors/connector-starrocks/pom.xml23
-rw-r--r--groot-connectors/connector-starrocks/src/main/java/com/geedgenetworks/connectors/starrocks/StarRocksTableFactory.java85
-rw-r--r--groot-connectors/connector-starrocks/src/main/java/com/starrocks/connector/flink/table/sink/EventStarRocksDynamicSinkFunctionV2.java318
-rw-r--r--groot-connectors/connector-starrocks/src/main/java/com/starrocks/connector/flink/table/sink/EventStreamLoadListener.java28
-rw-r--r--groot-connectors/connector-starrocks/src/main/resources/META-INF/services/com.geedgenetworks.core.factories.Factory1
-rw-r--r--groot-connectors/pom.xml1
-rw-r--r--groot-core/pom.xml25
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/connector/format/MapDeserialization.java8
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/factories/FactoryUtil.java10
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/filter/AviatorFilter.java15
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/filter/Filter.java11
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/pojo/KmsKey.java19
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/processor/Processor.java9
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AbstractFirstAggregation.java23
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorImpl.java29
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/FirstAggregationEventTime.java11
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/FirstAggregationProcessingTime.java11
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/KeySelector.java10
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/PreKeySelector.java10
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/processor/projection/ProjectionProcessorImpl.java30
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessorImpl.java27
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/split/Split.java10
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/split/SplitFunction.java2
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/split/SplitOperator.java14
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/AsnLookup.java68
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/CurrentUnixTimestamp.java6
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/DecodeBase64.java27
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/Domain.java155
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/Drop.java17
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/EncodeBase64.java41
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/Encrypt.java167
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/Eval.java4
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/Flatten.java2
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/FromUnixTimestamp.java47
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/GenerateStringArray.java8
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/GeoIpLookup.java140
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/Hmac.java104
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/JsonExtract.java21
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/PathCombine.java8
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/SnowflakeId.java6
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/StringJoiner.java8
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/UnixTimestampConverter.java26
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/cn/AbstractKnowledgeScalarFunction.java8
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/cn/ArrayElementsPrepend.java4
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/cn/BaseStationLookup.java81
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/cn/FieldsMerge.java4
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/cn/H3CellLookup.java84
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/cn/IntelligenceIndicatorLookup.java11
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/AES128GCM96.java84
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/AES256GCM96.java84
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/EncryptionAlgorithm.java17
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/SM4GCM96.java84
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/AbstractSingleKnowledgeBaseHandler.java16
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/AsnKnowledgeBaseHandler.java1
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/BaseStationKnowledgeBaseHandler.java90
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/IntelligenceIndicatorKnowledgeBaseHandler.java12
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectList.java25
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectSet.java25
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/FirstValue.java17
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramBaseAggregate.java242
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LastValue.java14
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LongCount.java16
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Max.java121
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Mean.java8
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Min.java120
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/NumberSum.java28
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/HlldBaseAggregate.java252
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/JsonUnroll.java15
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/PathUnroll.java118
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/Unroll.java24
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUID.java48
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDNameSpace.java43
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDv5.java68
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDv7.java47
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/utils/EncryptionAlgorithmUtils.java30
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/utils/KmsUtils.java71
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/utils/LoadIntervalDataOptions.java80
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/utils/LoadIntervalDataUtil.java86
-rw-r--r--groot-core/src/main/java/com/geedgenetworks/core/utils/SingleValueMap.java125
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/types/TypesTest.java39
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/cn/AnonymityLookupTest.java8
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/cn/AppCategoryLookupTest.java2
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/cn/BaseStationLookupTest.java69
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/cn/DnsServerInfoLookupTest.java4
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/cn/FqdnCategoryLookupTest.java2
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/cn/FqdnWhoisLookupTest.java4
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/cn/H3CellLookupTest.java65
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IcpLookupTest.java4
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IdcRenterLookupTest.java4
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IntelligenceIndicatorLookupTest.java45
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IocLookupTest.java8
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IpZoneLookupTest.java4
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/cn/LinkDirectionLookupTest.java4
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/cn/LookupTestUtils.java20
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/cn/UserDefineTagLookupTest.java16
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/cn/VpnLookupTest.java8
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/AsnLookupFunctionTest.java14
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/GeoIpLookupFunctionTest.java14
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectListTest.java23
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectSetTest.java23
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/FirstValueTest.java23
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LastValueTest.java23
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LongCountTest.java21
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MaxTest.java144
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MeanTest.java35
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MinTest.java123
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/NumberSumTest.java8
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DecodeBase64FunctionTest.java8
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DomainFunctionTest.java27
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DropFunctionTest.java1
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/EncodeBase64FunctionTest.java41
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/EncryptFunctionTest.java245
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/FlattenFunctionTest.java2
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/FromUnixTimestampTest.java31
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/GenerateStringArrayFunctionTest.java4
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/HmacFunctionTest.java136
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/JsonExtractFunctionTest.java6
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/StringJoinerFunctionTest.java4
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/UUIDTest.java139
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/UnixTimestampConverterTest.java4
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/JsonUnrollFunctionTest.java12
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnrollFunctionTest.java12
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantileTest.java176
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantilesTest.java194
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramTest.java202
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldApproxCountDistinctTest.java172
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldTest.java172
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/udf/udtf/UnrollTest.java109
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/utils/LoadIntervalDataUtilTest.java80
-rw-r--r--groot-core/src/test/java/com/geedgenetworks/core/utils/SingleValueMapTest.java98
-rw-r--r--groot-examples/cn-udf-example/pom.xml2
-rw-r--r--groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java2
-rw-r--r--groot-examples/end-to-end-example/src/main/resources/examples/grootstream_job_split_test.yaml97
-rw-r--r--groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_test.yaml47
-rw-r--r--groot-examples/end-to-end-example/src/main/resources/grootstream.yaml15
-rw-r--r--groot-examples/pom.xml9
-rw-r--r--groot-formats/format-csv/pom.xml23
-rw-r--r--groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvEventDeserializationSchema.java54
-rw-r--r--groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvEventSerializationSchema.java27
-rw-r--r--groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvFormatFactory.java190
-rw-r--r--groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvFormatOptions.java58
-rw-r--r--groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvSerializer.java181
-rw-r--r--groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvToMapDataConverter.java222
-rw-r--r--groot-formats/format-csv/src/main/resources/META-INF/services/com.geedgenetworks.core.factories.Factory1
-rw-r--r--groot-formats/format-csv/src/test/java/com/geedgenetworks/formats/csv/CsvEventSerDeSchemaTest.java219
-rw-r--r--groot-formats/format-json/src/main/java/com/geedgenetworks/formats/json/JsonEventDeserializationSchema.java19
-rw-r--r--groot-formats/format-msgpack/src/main/java/com/geedgenetworks/formats/msgpack/MessagePackEventDeserializationSchema.java12
-rw-r--r--groot-formats/format-protobuf/src/main/java/com/geedgenetworks/formats/protobuf/ProtobufEventDeserializationSchema.java21
-rw-r--r--groot-formats/pom.xml1
-rw-r--r--groot-release/pom.xml10
-rw-r--r--groot-release/src/main/assembly/assembly-bin-ci.xml2
-rw-r--r--groot-tests/test-common/src/test/java/com/geedgenetworks/test/common/container/AbstractTestFlinkContainer.java4
-rw-r--r--groot-tests/test-common/src/test/resources/grootstream.yaml7
-rw-r--r--groot-tests/test-e2e-base/src/test/java/com/geedgenetworks/test/e2e/base/InlineToPrintIT.java64
-rw-r--r--groot-tests/test-e2e-base/src/test/resources/inline_to_print.yaml194
-rw-r--r--pom.xml14
244 files changed, 10755 insertions, 2500 deletions
diff --git a/config/grootstream.yaml b/config/grootstream.yaml
index e01fda3..ec661f0 100644
--- a/config/grootstream.yaml
+++ b/config/grootstream.yaml
@@ -11,6 +11,25 @@ grootstream:
files:
- 64af7077-eb9b-4b8f-80cf-2ceebc89bea9
- 004390bc-3135-4a6f-a492-3662ecb9e289
+
+ kms:
+ # local:
+ # type: local
+ # secret_key: .geedgenetworks.
+ vault:
+ type: vault
+ url: https://192.168.40.223:8200
+ username: tsg_olap
+ password: tsg_olap
+ default_key_path: tsg_olap/transit
+ plugin_key_path: tsg_olap/plugin/gmsm
+
+ ssl:
+ skip_verification: true
+ ca_certificate_path: ./config/ssl/root.pem
+ certificate_path: ./config/ssl/worker.pem
+ private_key_path: ./config/ssl/worker.key
+
properties:
hos.path: http://192.168.44.12:9098/hos
hos.bucket.name.traffic_file: traffic_file_bucket
diff --git a/config/grootstream_job_example.yaml b/config/grootstream_job_example.yaml
index 37ef114..8c7a1b1 100644
--- a/config/grootstream_job_example.yaml
+++ b/config/grootstream_job_example.yaml
@@ -15,9 +15,9 @@ splits:
decoded_as_split:
type: split
rules:
- - name: projection_processor
+ - tag: http_tag
expression: event.decoded_as == 'HTTP'
- - name: aggregate_processor
+ - tag: dns_tag
expression: event.decoded_as == 'DNS'
processing_pipelines:
projection_processor:
@@ -66,6 +66,8 @@ application:
env:
name: example-inline-to-print
parallelism: 3
+ shade.identifier: sm4
+ kms.type: vault
pipeline:
object-reuse: true
execution:
@@ -76,11 +78,13 @@ application:
hos.bucket.name.http_file: traffic_http_file_bucket
hos.bucket.name.eml_file: traffic_eml_file_bucket
hos.bucket.name.policy_capture_file: traffic_policy_capture_file_bucket
+ projection.encrypt.schema.registry.uri: 192.168.44.12:9999/v1/schema/session_record?option=encrypt_fields
topology:
- name: inline_source
downstream: [decoded_as_split]
- name: decoded_as_split
- downstream: [ projection_processor ,aggregate_processor]
+ tags: [http_tag, dns_tag]
+ downstream: [ projection_processor, aggregate_processor]
- name: projection_processor
downstream: [ print_sink ]
- name: aggregate_processor
diff --git a/config/template/grootstream_job_template.yaml b/config/template/grootstream_job_template.yaml
index 0ca2d68..b26fbb2 100644
--- a/config/template/grootstream_job_template.yaml
+++ b/config/template/grootstream_job_template.yaml
@@ -10,7 +10,7 @@ sources: # [object] Define connector source
type: kafka # [string] Source Type
schema: # [object] Source Schema, config through fields or local_file or url. if not set schema, all fields(Map<String, Object>) will be output.
#fields: "struct<log_id:bigint, recv_time:bigint,client_ip: string>"
- local_file: /../schema/kafka_source_schema.json # [string] Local File Path for Schema
+ local_file: $GROOT_HOME/config/dat/schema/kafka_source_schema.json # [string] Local File Path for Schema
#url: http:// # [string] URL for Schema
properties: # [object] Kafka source properties
topic: SESSION-RECORD # [string] Topic Name, consumer will subscribe this topic.
@@ -45,9 +45,9 @@ sources: # [object] Define connector source
kafka.security.protocol: SSL
kafka.ssl.endpoint.identification.algorithm: ""
- kafka.ssl.keystore.location: /data/tsg/olap/flink/topology/data/keystore.jks
+ kafka.ssl.keystore.location: $GROOT_HOME/config/dat/keystore.jks
kafka.ssl.keystore.password: 86cf0e2ffba3f541a6c6761313e5cc7e
- kafka.ssl.truststore.location: /data/tsg/olap/flink/topology/data/truststore.jks
+ kafka.ssl.truststore.location: $GROOT_HOME/config/dat/truststore.jks
kafka.ssl.truststore.password: 86cf0e2ffba3f541a6c6761313e5cc7e
kafka.ssl.key.password: 86cf0e2ffba3f541a6c6761313e5cc7e
#kafka.security.protocol: SASL_PLAINTEXT
@@ -100,7 +100,7 @@ sources: # [object] Define connector source
data: CIin2awGEICAoLC/hYzKAhoEQkFTRSCch8z3wtqEhAQo6o/Xmc0xMMCy15nNMTjWIkDRCEiIp9msBlCIp9msBloIMjE0MjYwMDNg//8DaP//A3JqeyJ0YWdzIjpbeyJ0YWciOiJkYXRhX2NlbnRlciIsInZhbHVlIjoiY2VudGVyLXh4Zy05MTQwIn0seyJ0YWciOiJkZXZpY2VfZ3JvdXAiLCJ2YWx1ZSI6Imdyb3VwLXh4Zy05MTQwIn1dfXoPY2VudGVyLXh4Zy05MTQwggEOZ3JvdXAteHhnLTkxNDCKAQ0xOTIuMTY4LjQwLjgxkAEEmAEBoAEBqAGQwAGyAQdbMSwxLDJd4gEDt+gY4gINMTkyLjU2LjE1MS44MOgCoeYD8gIHV2luZG93c/oCGOe+juWbvS5Vbmtub3duLlVua25vd24uLrIDDTE5Mi41Ni4yMjIuOTO4A/ZwwgMFTGludXjKAxjnvo7lm70uVW5rbm93bi5Vbmtub3duLi6SBAN0Y3CaBBFFVEhFUk5FVC5JUHY0LlRDULAMBLgMBcAM9gHIDJEOoA2AAagN8cr+jgKwDezksIAPwg0RYTI6ZmE6ZGM6NTY6Yzc6YjPKDRE0ODo3Mzo5Nzo5NjozODoyMNINETQ4OjczOjk3Ojk2OjM4OjIw2g0RYTI6ZmE6ZGM6NTY6Yzc6YjM=
type: base64
format: protobuf
- protobuf.descriptor.file.path: ..\session_record_test.desc
+ protobuf.descriptor.file.path: $GROOT_HOME/config/dat/schema/session_record_test.desc
protobuf.message.name: SessionRecord
ipfix_source: # [object] IPFIX source connector name, must be unique. It used to define the source node of the job topology.
@@ -328,6 +328,7 @@ processing_pipelines: # [object] Define Processors for processing pipelines.
group_by_fields: [ recv_time, sled_ip ] # [array of string] Group By Fields
window_type: tumbling_processing_time # [string] Window Type, tumbling_processing_time, tumbling_event_time, sliding_processing_time, sliding_event_time
window_size: 60
+ mini_batch: true # [boolean] Enable Local Aggregation, default is false
functions:
- function: NUMBER_SUM
lookup_fields: [ received_bytes, sent_bytes ]
@@ -420,7 +421,7 @@ sinks: # [object] Define connector sink
clickhouse_sink: # [object] ClickHouse sink connector name, must be unique. It used to define the sink node of the job topology.
type: clickhouse
schema:
- local_file: /../schema/clickhouse_sink_schema.json
+ local_file: $GROOT_HOME/config/dat/schema/clickhouse_sink_schema.json
properties:
host: 127.0.0.1:9001
table: inline_source_test_local
@@ -445,9 +446,11 @@ application: # [object] Application Configuration
env: # [object] Define job runtime environment variables
name: inline-to-print-job # [string] Job Name
parallelism: 3 # [number] Job-Level Parallelism
- shade.identifier: default # [string] Shade Identifier, Using to encrypt and decrypt sensitive configuration. Support enum: default, aes, base64. if set default, it will not encrypt and decrypt sensitive configuration.
+ shade.identifier: default # [string] Config Shade Identifier, Using to encrypt and decrypt sensitive configuration. Support enum: default, aes, base64. if set default, it will not encrypt and decrypt sensitive configuration.
+ kms.type: local # [string] Key Management Service Type, default is local. Support enum: local, vault.
pipeline:
object-reuse: true # [boolean] Object Reuse, default is false
+
topology: # [array of object] Node List. It will be used build data flow for job dag graph.
- name: inline_source # [string] Node Name, must be unique. It will be used as the name of the corresponding Flink operator. eg. kafka_source the processor type as SOURCE.
#parallelism: 1 # [number] Operator-Level Parallelism.
diff --git a/config/template/mock_schema/session_record_mock_desc.json b/config/template/mock_schema/session_record_mock_desc.json
index c8c4acf..90060a6 100644
--- a/config/template/mock_schema/session_record_mock_desc.json
+++ b/config/template/mock_schema/session_record_mock_desc.json
@@ -115,12 +115,43 @@
"end": "103.144.108.255"
},
{
+ "name": "client_ip_tags",
+ "type": "String",
+ "array": true,
+ "options": [
+ "Country:United States",
+ "ASN:63278",
+ "Cloud Provider:IBM Cloud",
+ "Country Code:US",
+ "CDN Provider:Light CDN",
+ "ASN:6423"
+
+ ],
+ "arrayLenMin":1,
+ "arrayLenMax":5
+ },
+ {
"name": "server_ip",
"type": "IPv4",
"start": "1.0.0.0",
"end": "162.105.10.255"
},
{
+ "name": "server_ip_tags",
+ "type": "String",
+ "array": true,
+ "options": [
+ "Country:China",
+ "ASN:15169",
+ "Cloud Provider:Alibaba Cloud",
+ "Country Code:CN",
+ "CDN Provider:Akamai",
+ "Super Administrative Area:Guangdong"
+ ],
+ "arrayLenMin":1,
+ "arrayLenMax":5
+ },
+ {
"name": "c2s_ttl",
"type": "Number",
"options": [
@@ -167,12 +198,43 @@
"end": "162.105.10.255"
},
{
+ "name": "client_ip_tags",
+ "type": "String",
+ "array": true,
+ "options": [
+ "Country:China",
+ "ASN:15169",
+ "Cloud Provider:Alibaba Cloud",
+ "Country Code:CN",
+ "CDN Provider:Akamai",
+ "Super Administrative Area:Guangdong"
+ ],
+ "arrayLenMin":1,
+ "arrayLenMax":5
+ },
+ {
"name": "server_ip",
"type": "IPv4",
"start": "103.144.108.1",
"end": "103.144.108.255"
},
{
+ "name": "server_ip_tags",
+ "type": "String",
+ "array": true,
+ "options": [
+ "Country:United States",
+ "ASN:63278",
+ "Cloud Provider:IBM Cloud",
+ "Country Code:US",
+ "CDN Provider:Light CDN",
+ "ASN:6423"
+
+ ],
+ "arrayLenMin":1,
+ "arrayLenMax":5
+ },
+ {
"name": "c2s_ttl",
"type": "Number",
"options": [
@@ -340,6 +402,20 @@
"nullRate": 0.1
},
{
+ "name": "server_fqdn_tags",
+ "type": "String",
+ "array": true,
+ "options": [
+ "Category Name:Entertainment and Arts",
+ "IoC:Malware",
+ "Category Name:Home and Garden",
+ "Category Name:Translation",
+ "IoC:Spam"
+ ],
+ "arrayLenMin":1,
+ "arrayLenMax":5
+ },
+ {
"name": "server_port",
"type": "Number",
"options": [
diff --git a/config/udf.plugins b/config/udf.plugins
index e4f940f..3d6a353 100644
--- a/config/udf.plugins
+++ b/config/udf.plugins
@@ -4,17 +4,22 @@ com.geedgenetworks.core.udf.DecodeBase64
com.geedgenetworks.core.udf.Domain
com.geedgenetworks.core.udf.Drop
com.geedgenetworks.core.udf.EncodeBase64
+com.geedgenetworks.core.udf.Encrypt
com.geedgenetworks.core.udf.Eval
com.geedgenetworks.core.udf.Flatten
com.geedgenetworks.core.udf.FromUnixTimestamp
com.geedgenetworks.core.udf.GenerateStringArray
com.geedgenetworks.core.udf.GeoIpLookup
+com.geedgenetworks.core.udf.Hmac
com.geedgenetworks.core.udf.JsonExtract
com.geedgenetworks.core.udf.PathCombine
com.geedgenetworks.core.udf.Rename
com.geedgenetworks.core.udf.SnowflakeId
com.geedgenetworks.core.udf.StringJoiner
com.geedgenetworks.core.udf.UnixTimestampConverter
+com.geedgenetworks.core.udf.uuid.UUID
+com.geedgenetworks.core.udf.uuid.UUIDv5
+com.geedgenetworks.core.udf.uuid.UUIDv7
com.geedgenetworks.core.udf.udaf.NumberSum
com.geedgenetworks.core.udf.udaf.CollectList
com.geedgenetworks.core.udf.udaf.CollectSet
@@ -28,4 +33,7 @@ com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogram
com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogramQuantile
com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogramQuantiles
com.geedgenetworks.core.udf.udtf.JsonUnroll
-com.geedgenetworks.core.udf.udtf.Unroll \ No newline at end of file
+com.geedgenetworks.core.udf.udtf.Unroll
+com.geedgenetworks.core.udf.udtf.PathUnroll
+com.geedgenetworks.core.udf.udaf.Max
+com.geedgenetworks.core.udf.udaf.Min \ No newline at end of file
diff --git a/docs/connector/config-encryption-decryption.md b/docs/connector/config-encryption-decryption.md
index c2b05f6..91ca80e 100644
--- a/docs/connector/config-encryption-decryption.md
+++ b/docs/connector/config-encryption-decryption.md
@@ -127,13 +127,13 @@ Next, I'll show how to quickly use groot-stream's own `aes` encryption:
## How to implement user-defined encryption and decryption
-1. Create a new class and implement interface `ConfigShade`, this interface has the following methods:
+1. Create a new class and implement interface `CryptoShade`, this interface has the following methods:
```java
- public interface ConfigShade {
+ public interface CryptoShade {
/**
* The unique identifier of the current interface, used it to select the correct {@link
- * ConfigShade}
+ * CryptoShade}
*/
String getIdentifier();
@@ -157,6 +157,6 @@ Next, I'll show how to quickly use groot-stream's own `aes` encryption:
}
}
```
-2. Add `com.geedgenetworks.common.config.ConfigShade` in `resources/META-INF/services`
+2. Add `com.geedgenetworks.common.crypto.CryptoShade` in `resources/META-INF/services`
3. Change the option `shade.identifier` to the value that you defined in `ConfigShade#getIdentifier`of you config file.
diff --git a/docs/connector/connector.md b/docs/connector/connector.md
index 766b73e..93d64b0 100644
--- a/docs/connector/connector.md
+++ b/docs/connector/connector.md
@@ -1,3 +1,12 @@
+# Table of Contents
+- [Source Connector](#source-connector)
+ - [Common Source Options](#common-source-options)
+ - [Schema Field Projection](#schema-field-projection)
+ - [Schema Config](#schema-config)
+ - [Mock Data Type](#mock-data-type)
+- [Sink Connector](#sink-connector)
+ - [Common Sink Options](#common-sink-options)
+
# Source Connector
Source Connector contains some common core features, and each source connector supports them to varying degrees.
@@ -62,13 +71,12 @@ schema:
To retrieve the schema from a local file using its absolute path.
> Ensures that the file path is accessible to all nodes in your Flink cluster.
->
-> ```yaml
-> schema:
-> # by array
-> fields:
-> local_file: "/path/to/schema.json"
-> ```
+
+ ```yaml
+schema:
+ # Note: Only support avro schema format
+ local_file: "/path/to/schema.json"
+```
### URL
@@ -76,9 +84,8 @@ Some connectors support periodically fetching and updating the schema from a URL
```yaml
schema:
- # by array
- fields:
- url: "https://localhost:8080/schema.json"
+ # Note: Only support avro schema format
+ url: "https://localhost:8080/schema.json"
```
## Mock Data Type
diff --git a/docs/connector/formats/csv.md b/docs/connector/formats/csv.md
new file mode 100644
index 0000000..ca8d10b
--- /dev/null
+++ b/docs/connector/formats/csv.md
@@ -0,0 +1,73 @@
+# CSV
+
+> Format CSV
+>
+> ## Description
+>
+> The CSV format allows to read and write CSV data based on an CSV schema. Currently, the CSV schema is derived from table schema.
+> **The CSV format must config schema for source/sink**.
+
+| Name | Supported Versions | Maven |
+|--------------|--------------------|---------------------------------------------------------------------------------------------------------------------------|
+| Format CSV | Universal | [Download](http://192.168.40.153:8099/service/local/repositories/platform-release/content/com/geedgenetworks/format-csv/) |
+
+## Format Options
+
+| Name | Type | Required | Default | Description |
+|-----------------------------|-----------|----------|---------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| format | String | Yes | (none) | Specify what format to use, here should be 'csv'. |
+| csv.field.delimiter | String | No | , | Field delimiter character (',' by default), must be single character. You can use backslash to specify special characters, e.g. '\t' represents the tab character. |
+| csv.disable.quote.character | Boolean | No | false | Disabled quote character for enclosing field values (false by default). If true, option 'csv.quote.character' can not be set. |
+| csv.quote.character | String | No | " | Quote character for enclosing field values (" by default). |
+| csv.allow.comments | Boolean | No | false | Ignore comment lines that start with '#' (disabled by default). If enabled, make sure to also ignore parse errors to allow empty rows. |
+| csv.ignore.parse.errors | Boolean | No | false | Skip fields and rows with parse errors instead of failing. Fields are set to null in case of errors. |
+| csv.array.element.delimiter | String | No | ; | Array element delimiter string for separating array and row element values (';' by default). |
+| csv.escape.character | String | No | (none) | Escape character for escaping values (disabled by default). |
+| csv.null.literal | String | No | (none) | Null literal string that is interpreted as a null value (disabled by default). |
+
+# How to use
+
+## Inline uses example
+
+data:
+
+```json
+{
+ "log_id": 1,
+ "recv_time": 1712827485,
+ "client_ip": "192.168.0.1"
+}
+```
+
+```yaml
+sources:
+ inline_source:
+ type: inline
+ schema:
+ fields: "log_id:int, recv_time:bigint, client_ip:string"
+ properties:
+ data: "1,1712827485,192.168.0.1"
+ format: csv
+
+sinks:
+ print_sink:
+ type: print
+ schema:
+ fields: "log_id:int, recv_time:bigint, client_ip:string"
+ properties:
+ format: csv
+
+application:
+ env:
+ name: example-inline-to-print
+ parallelism: 3
+ pipeline:
+ object-reuse: true
+ topology:
+ - name: inline_source
+ downstream: [print_sink]
+ - name: print_sink
+ downstream: []
+
+```
+
diff --git a/docs/connector/formats/raw.md b/docs/connector/formats/raw.md
index 853ac79..06ea8bc 100644
--- a/docs/connector/formats/raw.md
+++ b/docs/connector/formats/raw.md
@@ -4,7 +4,7 @@
>
> ## Description
>
-> The Raw format allows to read and write raw (byte based) values as a single column.
+> The Raw format allows to read and write raw (byte based) values as a single column, the column name is raw default, it can also be explicitly defined as other name.
| Name | Supported Versions | Maven |
|------------|--------------------|---------------------------------------------------------------------------------------------------------------------------|
diff --git a/docs/connector/sink/kafka.md b/docs/connector/sink/kafka.md
index 716a179..78b7f34 100644
--- a/docs/connector/sink/kafka.md
+++ b/docs/connector/sink/kafka.md
@@ -20,19 +20,20 @@ In order to use the Kafka connector, the following dependencies are required. Th
Kafka sink custom properties. if properties belongs to Kafka Producer Config, you can use `kafka.` prefix to set.
-| Name | Type | Required | Default | Description |
-|------------------------------------|---------|----------|---------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| topic | String | Yes | (none) | Topic name is required. It used to write data to kafka. |
-| kafka.bootstrap.servers | String | Yes | (none) | A list of host/port pairs to use for establishing the initial connection to the Kafka cluster. This list should be in the form `host1:port1,host2:port2,...`. |
-| log.failures.only | Boolean | No | true | Defines whether the producer should fail on errors, or only log them. If this is set to true, then exceptions will be only logged, if set to false, exceptions will be eventually thrown and cause the streaming program to fail (and enter recovery). |
-| format | String | No | json | Data format. The default value is `json`. The Optional values are `json`, `protobuf`. |
-| [format].config | / | No | (none) | Data format properties. Please refer to [Format Options](../formats) for details. |
-| rate.limiting.strategy | String | No | (none) | The rate limiting strategy to use. The Optional values are `none`, `sliding_window`. |
-| rate.limiting.window.size | Integer | No | 5 | The window size of the rate limiting. For example, limit rate less than 10Mbps in 5 seconds time interval. |
-| rate.limiting.limit.rate | String | No | 10Mbps | A maximum rate of traffic that can be transmitted over a network or between networks. The units of the bytes rate are Mbps, Kbps,and bps. For example, 10Mbps, 100Kbps, 1000bps. |
-| rate.limiting.block.duration | String | No | 5min | If the rate limit is exceeded, the data will be blocked for the specified duration. The units of the duration are seconds, minutes, and hours. For example, 10s, 1m, 1h. |
-| rate.limiting.block.reset.duration | String | No | 30s | The time interval for resetting the rate limit. The units of the duration are seconds, minutes, and hours. For example, 10s, 1m, 1h. |
-| kafka.config | / | No | (none) | Kafka producer properties. Please refer to [Kafka Producer Config](https://kafka.apache.org/documentation/#producerconfigs) for details. |
+| Name | Type | Required | Default | Description |
+|-------------------------------------|---------|----------|---------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| topic | String | Yes | (none) | Topic name is required. It used to write data to kafka. |
+| kafka.bootstrap.servers | String | Yes | (none) | A list of host/port pairs to use for establishing the initial connection to the Kafka cluster. This list should be in the form `host1:port1,host2:port2,...`. |
+| log.failures.only | Boolean | No | true | Defines whether the producer should fail on errors, or only log them. If this is set to true, then exceptions will be only logged, if set to false, exceptions will be eventually thrown and cause the streaming program to fail (and enter recovery). |
+| format | String | No | json | Data format. The default value is `json`. The Optional values are `json`, `protobuf`. |
+| [format].config | Map | No | (none) | Data format properties. Please refer to [Format Options](../formats) for details. |
+| headers.config | Map | No | (none) | Kafka record headers info. exp: 'headers.key: value' will put key and value into record headers. |
+| rate.limiting.strategy | String | No | (none) | The rate limiting strategy to use. The Optional values are `none`, `sliding_window`. |
+| rate.limiting.window.size | Integer | No | 5 | The window size of the rate limiting. For example, limit rate less than 10Mbps in 5 seconds time interval. |
+| rate.limiting.limit.rate | String | No | 10Mbps | A maximum rate of traffic that can be transmitted over a network or between networks. The units of the bytes rate are Mbps, Kbps,and bps. For example, 10Mbps, 100Kbps, 1000bps. |
+| rate.limiting.block.duration | String | No | 5min | If the rate limit is exceeded, the data will be blocked for the specified duration. The units of the duration are seconds, minutes, and hours. For example, 10s, 1m, 1h. |
+| rate.limiting.block.reset.duration | String | No | 30s | The time interval for resetting the rate limit. The units of the duration are seconds, minutes, and hours. For example, 10s, 1m, 1h. |
+| kafka.config | Map | No | (none) | Kafka producer properties. Please refer to [Kafka Producer Config](https://kafka.apache.org/documentation/#producerconfigs) for details. |
## Example
diff --git a/docs/connector/sink/starrocks.md b/docs/connector/sink/starrocks.md
new file mode 100644
index 0000000..f07e432
--- /dev/null
+++ b/docs/connector/sink/starrocks.md
@@ -0,0 +1,83 @@
+# Starrocks
+
+> Starrocks sink connector
+>
+> ## Description
+>
+> Sink connector for Starrocks, know more in https://docs.starrocks.io/zh/docs/loading/Flink-connector-starrocks/.
+
+## Sink Options
+
+Starrocks sink custom properties. If properties belongs to Starrocks Flink Connector Config, you can use `connection.` prefix to set.
+
+| Name | Type | Required | Default | Description |
+|---------------------|---------|----------|---------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| log.failures.only | Boolean | No | true | Optional flag to whether the sink should fail on errors, or only log them; If this is set to true, then exceptions will be only logged, if set to false, exceptions will be eventually thrown, true by default. |
+| connection.jdbc-url | String | Yes | (none) | The address that is used to connect to the MySQL server of the FE. You can specify multiple addresses, which must be separated by a comma (,). Format: jdbc:mysql://<fe_host1>:<fe_query_port1>,<fe_host2>:<fe_query_port2>,<fe_host3>:<fe_query_port3>.. |
+| connection.load-url | String | Yes | (none) | The address that is used to connect to the HTTP server of the FE. You can specify multiple addresses, which must be separated by a semicolon (;). Format: <fe_host1>:<fe_http_port1>;<fe_host2>:<fe_http_port2>.. |
+| connection.config | Map | No | (none) | Starrocks Flink Connector Options, know more in https://docs.starrocks.io/docs/loading/Flink-connector-starrocks/#options. |
+
+## Example
+
+This example read data of inline test source and write to Starrocks table `test`.
+
+```yaml
+sources: # [object] Define connector source
+ inline_source:
+ type: inline
+ schema:
+ fields: # [array of object] Schema field projection, support read data only from specified fields.
+ - name: log_id
+ type: bigint
+ - name: recv_time
+ type: bigint
+ - name: server_fqdn
+ type: string
+ - name: server_domain
+ type: string
+ - name: client_ip
+ type: string
+ - name: server_ip
+ type: string
+ - name: server_asn
+ type: string
+ - name: decoded_as
+ type: string
+ - name: device_group
+ type: string
+ - name: device_tag
+ type: string
+ properties:
+ #
+ # [string] Event Data, it will be parsed to Map<String, Object> by the specified format.
+ #
+ data: '{"recv_time": 1705565615, "log_id":206211012872372224, "tcp_rtt_ms":128,"decoded_as":"HTTP", "http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.ct.cn","http_url":"www.ct.cn/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.11.22.22","server_ip":"8.8.8.8","client_port":42751,"server_port":80,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":5575,"sent_pkts":97,"sent_bytes":5892,"received_pkts":250,"received_bytes":333931,"tcp_c2s_ip_fragments":0,"tcp_s2c_ip_fragments":0,"tcp_c2s_rtx_pkts":0,"tcp_c2s_rtx_bytes":0,"tcp_s2c_rtx_pkts":0,"tcp_s2c_rtx_bytes":0,"tcp_c2s_o3_pkts":0,"tcp_s2c_o3_pkts":0,"tcp_c2s_lost_bytes":0,"tcp_s2c_lost_bytes":0,"flags":26418,"flags_identify_info":[100,1,100,60,150,100,1,2],"app_transition":"http.1111.test_1_1","decoded_as":"HTTP","server_fqdn":"www.ct.cn","app":"test_1_1","decoded_path":"ETHERNET.IPv4.TCP.http","fqdn_category_list":[1767],"t_vsys_id":1,"vsys_id":1,"session_id":290538039798223400,"tcp_handshake_latency_ms":41,"client_os_desc":"Windows","server_os_desc":"Linux","data_center":"center-xxg-tsgx","device_group":"group-xxg-tsgx","device_tag":"{\"tags\":[{\"tag\":\"data_center\",\"value\":\"center-xxg-tsgx\"},{\"tag\":\"device_group\",\"value\":\"group-xxg-tsgx\"}]}","device_id":"9800165603247024","sled_ip":"192.168.40.39","dup_traffic_flag":0}'
+ format: json
+ json.ignore.parse.errors: false
+
+sinks:
+ starrocks_sink:
+ type: starrocks
+ properties:
+ "log.failures.only": false
+ "connection.jdbc-url": "jdbc:mysql://192.168.40.222:9030"
+ "connection.load-url": "192.168.40.222:8030"
+ "connection.database-name": "test"
+ "connection.table-name": "test"
+ "connection.username": "root"
+ "connection.password": ""
+ "connection.sink.buffer-flush.interval-ms": "30000"
+
+application: # [object] Define job configuration
+ env:
+ name: groot-stream-job-inline-to-starrocks
+ parallelism: 3
+ pipeline:
+ object-reuse: true
+ topology:
+ - name: inline_source
+ downstream: [ starrocks_sink ]
+ - name: starrocks_sink
+ downstream: [ ]
+```
+
diff --git a/docs/connector/source/kafka.md b/docs/connector/source/kafka.md
index 07dff22..680d1c1 100644
--- a/docs/connector/source/kafka.md
+++ b/docs/connector/source/kafka.md
@@ -24,6 +24,13 @@ Kafka source custom properties. if properties belongs to Kafka Consumer Config,
| [format].config | Map | No | (none) | Data format properties. Please refer to [Format Options](../formats) for details. |
| kafka.config | Map | No | (none) | Kafka consumer properties. Please refer to [Kafka Consumer Config](https://kafka.apache.org/documentation/#consumerconfigs) for details. |
+## Internal Fields
+
+| Name | Type | Description |
+|-------------|---------------------|-------------------------------------|
+| __timestamp | Long | The timestamp of this kafka record. |
+| __headers | Map[String, String] | The headers of this kafka record. |
+
## Example
This example read data of kafka topic `SESSION-RECORD` and print to console.
diff --git a/docs/develop-guide.md b/docs/develop-guide.md
index 2742cee..927d2d3 100644
--- a/docs/develop-guide.md
+++ b/docs/develop-guide.md
@@ -15,6 +15,28 @@
| groot-docs | Docs module of groot-stream, which is responsible for providing documents. |
| groot-release | Release module of groot-stream, which is responsible for providing release scripts. |
+## Event Model
+Groot Stream based all stream processing on data records common known as events. A event is a collection of key-value pairs(fields). As follows:
+
+```json
+{
+ "__timestamp": "<Timestamp in UNIX epoch format (milliseconds)>",
+ "__headers": "Map<String, String> headers of the source that delivered the event",
+ "__window_start_timestamp" : "<Timestamp in UNIX epoch format (milliseconds)>",
+ "__window_end_timestamp" : "<Timestamp in UNIX epoch format (milliseconds)>",
+ "key1": "<value1>",
+ "key2": "<value2>",
+ "keyN": "<valueN>"
+}
+```
+Groot Stream add internal fields during pipeline processing. A few notes about internal fields:
+- Internal fields start with a double underscore `__`.
+- Each source can add one or many internal fields to the each event. For example, the Kafka source adds both a `__timestamp` and a `__input_id` field.
+- Treat internal fields as read-only. Modifying them can result in unintended consequences to your data flows.
+- Internal fields only exist for the duration of the event processing pipeline. They are not documented under sources or sinks.
+- If you do not configure a timestamp for extraction, the Pipeline process assigns the current time (in UNIX epoch format) to the __timestamp field.
+- If you have multiple sources, you can determine the origin of the event by examining the `__headers` field. For example, the Kafka source appends the topic name as the `__input_id` key in the `__headers`.
+
## How to write a high quality Git commit message
> [purpose] [module name] [sub-module name] Description (JIRA Issue ID)
diff --git a/docs/env-config.md b/docs/env-config.md
index 8e22a53..58f7e71 100644
--- a/docs/env-config.md
+++ b/docs/env-config.md
@@ -36,6 +36,9 @@ This parameter is used to define the runtime mode of the job, the default value
Specify the method of encryption, if you didn't have the requirement for encrypting or decrypting sensitive information in the configuration file, this option can be ignored.
For more details, you can refer to the documentation [config-encryption-decryption](connector/config-encryption-decryption.md)
+### kms.type
+Specify Key Management System (KMS) type, default is `local`. You can integrate an external KMS, such as `vault`. For more details, you can refer to the documentation [KMS](grootstream-config.md#kms).
+
### pipeline.object-reuse
This parameter is used to enable/disable object reuse for the execution of the job. If it is not specified, the default value is `false`.
diff --git a/docs/grootstream-config.md b/docs/grootstream-config.md
index 9dd442f..b7fd037 100644
--- a/docs/grootstream-config.md
+++ b/docs/grootstream-config.md
@@ -77,6 +77,51 @@ grootstream:
- asn_builtin.mmdb
- asn_user_defined.mmdb
```
+
+## KMS
+Key Management System(KMS). It is a service that provides a secure way to create, manage, and control encryption keys used to encrypt data. KMS is used to protect sensitive information by ensuring that encryption keys are kept secure and accessible only to authorized users and applications.
+
+| Name | Type | Required | Default | Description |
+|:-----| :----- | :------- | :-- ---- |:------------------------------------------------ |
+| type | String | Yes | local | The type of the Key Management Service. Enum: local, vault. |
+| url | String | No | (none) | The kms server's URL (e.g., `http://localhost:8200`). |
+| token | String | No | (none) | The authentication token |
+| default_key_path | String | No | (none) | HashiCorp Vault default key path. for example, `transit/` |
+| plugin_key_path | String | No | (none) | HashiCorp Vault plugin key path. for example, `plugin/gmsm` |
+
+```yaml
+ kms:
+ local:
+ type: local
+ vault:
+ type: vault
+ url: <vault-url>
+ token: <vault-token>
+ default_key_path: <vault-key-path>
+ plugin_key_path: <vault-plugin-key-path>
+```
+
+## SSL
+
+The client SSL configuration.
+
+| Name | Type | Required | Default | Description |
+|:-----| :----- | :------- | :-- ---- |:------------------------------------------------ |
+| skip_verification | Boolean | Yes | true | Ignore SSL certificate verification |
+| certificate_path | String | Yes | (none) | Path to the client's private key file |
+| private_key_path | String | Yes | (none) | Path to the client's certificate file |
+| ca_certificate_path | Boolean | Yes | false | Path to the root CA certificate for server verification |
+
+```yaml
+ ssl:
+ skip_verification: true
+ private_key_path: /path/to/certs/worker.key
+ certificate_path: /path/to/certs/worker.pem
+ ca_certificate_path: /path/to/certs/root.pem
+```
+
+
+
## Properties
Global user-defined variables can be set in the `properties` section using key-value pairs, where the key represents a configuration property and the value specifies the desired setting.
The properties can be used in the configuration file by using `props.${property_name}`. \ No newline at end of file
diff --git a/docs/grootstream-design-cn.md b/docs/grootstream-design-cn.md
new file mode 100644
index 0000000..41fcd0d
--- /dev/null
+++ b/docs/grootstream-design-cn.md
@@ -0,0 +1,2194 @@
+# Groot Stream 设计方案
+
+# 目录
+- [概述](#概述)
+- [系统架构](#系统架构)
+- [全局配置 grootstream.yaml](#全局配置-grootstreamyaml)
+- [任务配置](#任务配置)
+ - [接入数据源(Sources)](#接入数据源sources)
+ - [Source 公共配置](#source-公共配置)
+ - [Schema配置](#schema配置)
+ - [Fields](#fields)
+ - [Local File](#local-file)
+ - [URL](#url)
+ - [Kafka Source](#kafka-source)
+ - [IPFIX Collector(UDP)](#ipfix-collectorudp)
+ - [File Source](#file-source)
+ - [Mock Source](#mock-source)
+ - [Inline Source](#inline-source)
+ - [过滤器(Filters)](#过滤器filters)
+ - [分流器(Splits)](#分流器splits)
+ - [任务处理器 (Processors)](#任务处理器-processors)
+ - [Projection Processor](#projection-processor)
+ - [Aggregate Processor](#aggregate-processor)
+ - [Table Processor](#table-processor)
+ - [输出Sinks](#输出sinks)
+ - [Kafka Sink](#kafka-sink)
+ - [ClickHouse Sink](#clickhouse-sink)
+ - [Print Sink](#print-sink)
+ - [Formats](#formats)
+ - [JSON](#json)
+ - [MessagePack](#messagepack)
+ - [Protobuf](#protobuf)
+ - [Raw](#raw)
+ - [任务编排](#任务编排)
+ - [函数定义](#函数定义)
+ - [内置UDF](#内置udf)
+ - [标量函数](#标量函数)
+ - [聚合函数](#聚合函数)
+ - [表格函数](#表格函数)
+ - [CN扩展UDF](#cn扩展udf)
+ - [实现原则](#实现原则)
+ - [相关问题](#相关问题)
+
+# 概述
+Groot Stream 是一个实时数据流处理平台,提供了灵活的数据定制管道,能够高效的从多种数据源收集数据,并对其进行加工和转换。具体包括过滤、解析、重组和数据聚合,以便更好的处理和管理数据。
+
+主要优势:
+
+- 实时数据处理:利用Flink作为底层引擎,可以针对大规模实时数据流提供高吞吐、低延迟的实时处理能力。
+- 插件化管理:可自定义Functions, Packs, Sources 和Sinks,用于满足不同应用场景下的数据流定制需求。
+- 降低开发成本:通过YML模版定制数据处理拓扑,无需编写代码快速实现ETL需求。替代现有Real-time Log Streaming ,Data Transporter ETL 和Gohangout数据加载模块。
+
+应用场景:
+
+- 数据汇聚场景
+ - 构建QuickConnect拓扑,各个分中心数据被集中汇聚到国家中心。
+- 数据流定制
+ - 会话日志经过预处理后发给不同的系统或第三方厂商。
+ - 定义Filter 匹配符合条件的日志,然后预处理Pipeline对日志进行反序列化,增加处理时间,抽取域名等操作。
+ - Router A 经过 TSG Projection处理器,执行ID-Mapping映射Subscriber ID,发送到TSG系统中。
+ - Router B 经过CN Projection处理器,增加IoC标签库映射字段,删除不需要的字段,发送到CN系统中。
+ - Router C 经过第三方厂商 Projection处理器,过滤SSL、HTTP 日志,抽取部分字段发送到第三方厂商中。
+ - 将会话日志按应用层协议分流,分发到不同Topic中。
+ - 过滤匹配SSL日志,分发到SSL Topic。
+ - 过滤匹配邮件日志,分发到Email Topic。
+- 数据聚合
+
+# 系统架构
+![Groot Stream Workflow](images/groot_stream_architecture.jpg)
+- **Sources**
+ - 接收多种数据源或收集器的连续数据输入, 包括Kafka、IPFIX Collector 或UDP 等。
+ - 配置参数包括基础配置和Source配置。例如Type 为Kafka,则需要增加Source参数kafka.bootstrap.servers, topics和kafka.consumer.group.id 等。
+- **Filters**
+ - 对数据源中的日志进行筛选和过滤,缩小处理日志的范围。
+ - 通过定义过滤表达式,指定数据中某些属性、条件或规则,基于该表达式匹配符合条件的数据。例如:common_c2s_bytes_num <= 2147483647 && common_s2c_bytes_num<= 2147483647 ,过滤掉不符合Integer取值范围的数据。
+
+- **QuickConnect**
+ - 基于最小化配置,快速构建Sources和Sinks之间的数据管道,可用于原型、测试或跨域数据汇聚。
+ - 通过在管道中插入Processors 或Pack。
+
+- **Pipelines**
+ - 在数据流的不同处理阶段可以引用不同类型的Pipelines,所有Pipelines(一系列Functions组成)架构和内部结构一致,只分为Projection和Aggregate两种类型。按Pipeline所在数据流的位置可分为:
+ - **Pre-processing Pipelines :可选,**前处理数据管道对输入日志进行格式化或执行一系列全局处理函数(例如:从原始日志中提取感兴趣的字段)。
+ - **Processing Pipelines:**业务处理管道
+ - **Post-processing Pipelines ,可选,**后处理数据管道,发送到目的地之前对日志进行格式化或执行一系列全局处理函数(例如:对输出的日志进行格式验证、类型转换)
+ - 数据流处理基本单元为处理器,按功能分为无状态和有状态处理器。每个处理器可以连接多个函数,组成一个Pipeline。
+ - 投影处理器(Projection Processor):针对每条日志选择所需的列或属性。它属于无状态处理器,期间会严格按照处理器定义的函数(UDFs)顺序执行。例如:获取顶级域名,字符串转换、类型转换或反序列化等运算符函数组成一个Pipeline。
+ - 聚合处理器(Aggregate Processor):多条日志进行分组聚合统计。它属于有状态处理器,期间可经过一系列自定义聚合函数(UDAFs)。例如:计算不同IP的总带宽,不同域名总会话数等聚合函数组成一个Pipeline。
+ - 表格处理器(Table Processor):一条日志展开为多条输出。它属于无状态处理器,期间可经过一系列自定义聚合函数(UDTFs)。例如:将某个JSON格式的属性展开为多条,其他属性复制,将多条日志输出。
+- **Sinks**
+ - 发送数据到多个目的地, 具体包括Kafka、HBase 或 Mysql 等。
+ - 每种Sink包括基础配置和Sink配置。例如Type 为Kafka,则需要Sink参数Kafka.bootstrap.servers, kafka.topic和kafka.producer.ack 等。
+- **Packs**
+ - 复杂业务逻辑处理器,一般应用于无法通过函数实现的场景。例如:动态知识库加载及动态schema的数据序列化。
+
+# 全局配置 grootstream.yaml
+
+```yaml
+grootstream:
+# 知识库配置
+ knowledge_base:
+ - name: tsg_ip_asn # 知识库名称
+ fs_type: http # 文件系统类型(http,local,hdfs..)
+ fs_path: http://127.0.0.1:9999/v1/knowledge_base # 文件路径(单机模式hdfs://{ip}:{port}/{path},集群模式hdfs://{nameservice}/{path})
+ files:
+ - f9f6bc91-2142-4673-8249-e097c00fe1ea # 知识库文件名
+ # ....
+
+ - name: tsg_ip_location
+ # ....
+ kms:
+ local:
+ type: local
+ vault:
+ type: vault
+ url: <vault-url>
+ token: <vault-token>
+ default_key_path: <default-vault-key-path>
+ plugin_key_path: <plugin-vault-key-path>
+
+ ssl: ## SSL/TLS 客户端链接配置
+ skip_verification: true # 忽略SSL证书校验
+ private_key_path: /path/to/certs/worker.key # 客户端私钥文件路径
+ certificate_path: /path/to/certs/worker.pem # 客户端证书文件路径
+ ca_certificate_path: /path/to/certs/root.pem # CA 根证书路径
+
+ properties: # 用户自定义属性的支持从函数中获取,使用方式见函数定义
+ hos.path: http://127.0.0.1:9093
+ hos.bucket.name.traffic_file: traffic_file_bucket
+ hos.bucket.name.troubleshooting_file: troubleshooting_file_bucket
+ scheduler.knowledge_base.update.interval.minutes: 1 #知识库文件定时更新时间
+```
+
+| 属性名 | 必填 | 默认值 | 类型 | 描述 |
+| -------------- | ---- | ------ | ------------------- | ---------------------------------------------- |
+| knowledge_base | Y | - | Object | 知识库配置 |
+| kms | N | - | Object | kms (key management system, 密钥管理系统) 配置 |
+| ssl | N | - | Object | ssl配置 |
+| properties | N | - | Map(String, Object) | 自定义属性配置:key-value 格式 |
+
+
+
+# 任务配置
+
+## 接入数据源(Sources)
+
+### **Source 公共配置**
+
+```yaml
+sources:
+ kafka_source:
+ type : kafka # source connector 类型
+ # source表schema, 通过fields/local_file/url三种方式配置: 配置则转换校验, 只输出配置的列;没配置输出全部列, 不进行类型转换和校验
+ schema:
+ fields:
+ - name: common_recv_time
+ type: bigint
+ - name: common_log_id
+ type: bigint
+ # local_file: "schema/test_schema.json"
+ # url: "http://127.0.0.1/schema.json"
+ # watermark_timestamp: recv_time
+ # watermark_timestamp_unit: ms
+ # watermark_lag: 60
+ properties: # source connector 配置
+ prop_key1: prop_value1
+ prop_key2: prop_value2
+ #...
+```
+
+| 属性名 | 必填 | 默认值 | 类型 | 描述 |
+|--------------------------|-------|-----------|-----------|------------------------------------------------------------------------------------------|
+| **type** | Y | - | String | source唯一标识 |
+| schema | N | - | Map | source表schema,配置则只输出配置的列,同时会进行类型转换和校验。 |
+| watermark_timestamp | N | - | String | watermark timestamp字段名称。 |
+| watermark_timestamp_unit | N | ms | String | watermark timestamp字段单位,可选值:ms(milliseconds),s(seconds)。如果配置watermark_timestamp,此字段是必须的。 |
+| watermark_lag | N | - | Long | watermark out-of-order milliseconds。如果配置watermark_timestamp,此字段是必须的。 |
+| properties | Y | - | Object | source属性配置 |
+
+### schema配置
+
+支持通过fields/local_file/url三种方式配置,只能同时配置一种方式。
+
+#### Fields
+
+支持配置属性列表和sql风格字符串(hive sql)
+
+example:
+
+```yaml
+ schema:
+ fields:
+ - name: common_recv_time
+ type: bigint
+ - name: common_log_id
+ type: bigint
+```
+
+支持的数据类型:
+
+| 类型 | 对应java类型 | 描述 |
+|---------|-----------------------|----------------------------------------------------------------------------|
+| string | String | 字符串 |
+| int | Integer | int |
+| bigint | Long | bigint |
+| float | Float | float |
+| double | Double | double |
+| boolean | Boolean | boolean |
+| binary | byte[] | 字节数组 |
+| struct | Map<String, Object> | 结构体。例如:struct<id:int, client_ip:string, data:struct<id:int, name:string>>。 |
+| array | List<Object> | 数组。例如:array<int>, array<struct<id:int, client_ip:string>>。 |
+
+#### Local File
+
+读取本地文件中的schema定义,只支持tsg avro schema格式
+
+- example
+
+```yaml
+ schema:
+ local_file: "schema/test_schema.json"
+```
+
+- test_schema.json
+
+```yaml
+ {
+ "type": "record",
+ "name": "test",
+ "fields" : [
+ {"name": "log_id", "type": "long"},
+ {"name": "recv_time", "type": "long"},
+ {"name": "client_ip", "type": "string","doc": {"visibility": "enabled"}}
+ ]
+}
+```
+
+#### URL
+
+读取http url返回的schema定义,只支持tsg avro schema格式,支持动态更新schema,支持动态schema的connector有:clickhouse sink.
+
+example:
+
+```yaml
+ schema:
+ url: "http://127.0.0.1/schema.json"
+```
+
+### Kafka Source
+
+```yaml
+sources: # [object]
+ kafka_source: # [object] Source Name
+ # source标识
+ type : kafka
+ # 数据schema: 配置则转换校验, 只输出配置的列;没配置输出全部列, 不进行类型转换和校验
+ fields:
+ - name: common_recv_time
+ type: bigint
+ - name: common_log_id
+ type: bigint
+ # source属性配置
+ properties:
+ topic: SESSION-RECORD-COMPLETED
+ kafka.bootstrap.servers: 192.168.44.11:9092
+ kafka.session.timeout.ms: 60000
+ kafka.max.poll.records: 3000
+ kafka.max.partition.fetch.bytes: 31457280
+ kafka.group.id: SESSION-RECORD-COMPLETED-GROUP-GROOT-STREAM-20231021
+ kafka.auto.offset.reset: latest
+ format: json
+```
+
+| 属性名 | 必填 | 默认值 | 类型 | 描述 |
+|-----------------------------|----|------|--------|---------------------------------------------------|
+| **topic** | Y | - | String | Kafka Topic名称。支持 Topic列表,用分号分隔,如'topic-1;topic-2' |
+| **kafka.bootstrap.servers** | Y | - | String | Kafka Broker 地址 |
+| **format** | Y | JSON | String | format,用来反序列化消息JSONProtobufCSV... |
+| Kafka Properties | N | - | | kafka Consumer Properties,以"kafka."作为前缀 |
+| Format properties | N | - | | format properties,以Format类型作为前缀。例如: “protobuf.” |
+
+### IPFIX Collector(UDP)
+
+```yaml
+sources: # [object]
+ ipfix_source: # [object] Source Name
+ # source标识
+ type : ipfix
+ # 数据schema: 配置则转换校验, 只输出配置的列;没配置输出全部列, 不进行类型转换和校验
+ fields:
+ - name: recv_time
+ type: bigint
+ - name: log_id
+ type: bigint
+ # source属性配置
+ properties:
+ port.range: 12345-12347
+ max.packet.size: 65535
+ max.receive.buffer.size: 104857600
+ service.discovery.registry.mode: 0
+ service.discovery.service.name: udp_ipfix
+ service.discovery.health.check.interval: 5
+ service.discovery.nacos.server.addr: 192.168.44.12:8848
+ service.discovery.nacos.username: nacos
+ service.discovery.nacos.password: nacos
+ service.discovery.nacos.namespace: test
+ service.discovery.nacos.group: IPFIX
+ service.discovery.consul.server.addr: 192.168.41.30
+ service.discovery.consul.server.port: 8500
+ service.discovery.consul.token:
+```
+
+| 属性名 | 必填 | 默认值 | 类型 | 描述 |
+|--------------------------------------------|------------|-------------|----------|-----------------------------------------------------------------------------|
+| port.range | Y | - | String | IPFIX Collector的UDP端口,指定单个端口或端口范围。例如指定单个端口为4739,指定端口范围为12345-12347。 |
+| max.packet.size | N | 65535 | Integer | 单条UDP数据包的最大大小,最大值为65535(Bytes)。 |
+| max.receive.buffer.size | N | 104857600 | Integer | UDP接收缓存区大小(Bytes)。 |
+| service.discovery.registry.mode | N | - | Integer | 服务发现的注册模式,0为nacos,1为consul,其他为不使用服务发现。 |
+| service.discovery.service.name | N | - | String | 服务发现中的serviceName。 |
+| service.discovery.health.check.interval | N | - | Integer | 服务发现健康检查的时间间隔,单位秒。 |
+| service.discovery.nacos.server.addr | N | - | String | nacos服务的地址,格式为ip:port, service.discovery.registry.mode为0时必须指定。 |
+| service.discovery.nacos.username | N | - | String | nacos的用户名,service.discovery.registry.mode为0时必须指定。 |
+| service.discovery.nacos.password | N | - | String | nacos的密码,service.discovery.registry.mode为0时必须指定。 |
+| service.discovery.nacos.namespace | N | - | String | nacos中的命名空间,service.discovery.registry.mode为0时可设置,不设置为public。 |
+| service.discovery.nacos.group | N | - | String | nacos中的所属组,service.discovery.registry.mode为0时可设置,不设置为DEFAULT。 |
+| service.discovery.consul.server.ip | N | - | String | consul服务的ip,service.discovery.registry.mode为1时必须指定。 |
+| service.discovery.consul.server.port | N | - | Integer | consul服务的端口,service.discovery.registry.mode为1时必须指定。 |
+| service.discovery.consul.token | N | - | String | consul的token,service.discovery.registry.mode为1且consul开启验证时必须指定。 |
+
+### File Source
+
+从text file读取数据,支持本地文件和hdfs文件,用于测试以及从文件回放数据,这个source每个1s发送2条数据
+
+```yaml
+sources:
+ file_source:
+ type: file
+ properties:
+ # path: 'hdfs://ns1/test/logs.json'
+ path: './logs.json'
+ rows.per.second: 2
+ format: json
+```
+
+| 属性名 | 必填 | 默认值 | 类型 | 描述 |
+|---------------------------|-------|------|---------|---------------------------------------------------------------------------------------------------------------------------------------------|
+| **path** | Y | - | String | 文件路径,以[hdfs://](hdfs://ns1/test/logs.json)开头为hdfs文件,其它为本地文件系统文件。例如:./logs/logs.json, [hdfs://ns1/test/logs.json](hdfs://ns1/test/logs.json) |
+| **format** | Y | - | String | 使用的format |
+| rows.per.second | N | 1000 | Integer | 每秒生成行数 |
+| number.of.rows | N | -1 | Long | 总生成行数,默认此source是无界流(会循环从文件生成数据),当配置大于0时此source为有界流 |
+| millis.per.row | N | 0 | Long | 每行生成花费毫秒数,当大于0时,rows.per.second配置不生效 |
+| read.local.file.in.client | N | true | Boolean | 是否在客户端读取本地文件,客户端读取限制文件大小最大为128MB。当为false时,在taskmanager端读取文件,必须在每个taskmanager的path存放文件 |
+
+put file to hdfs:
+
+```shell
+# maka dir
+hadoop fs -mkdir hdfs://ns1/test
+
+# put local file to hdfs
+hadoop fs -put logs.json hdfs://ns1/test
+
+# list hdfs dir
+hadoop fs -ls logs.json hdfs://ns1/test
+```
+
+### **Mock Source**
+
+mock数据源,用于生成测试数据
+
+```yaml
+sources:
+ mock_source:
+ type : mock
+ properties:
+ mock.desc.file.path: './mock_example.json'
+ rows.per.second: 1
+```
+
+| 属性名 | 必填 | 默认值 | 类型 | 描述 |
+|---------------------------|-----|-------|---------|----------------------------------------------------|
+| **mock.desc.file.path** | Y | - | String | mock schema文件路径 |
+| rows.per.second | N | 1000 | Integer | 每秒生成行数 |
+| number.of.rows | N | -1 | Long | 总生成行数,默认此source是无界流(会循环从文件生成数据),当配置大于0时此source为有界流 |
+| millis.per.row | N | 0 | Long | 每行生成花费毫秒数,当大于0时,rows.per.second配置不生效 |
+
+#### mock desc 文件配置
+
+mock desc为json配置,配置每个字段的mock规则,格式:
+
+```json
+ [
+ {
+ "name": "field_name1",
+ "type": "type1",
+ "arg": "arg"
+ },
+ {
+ "name": "field_name2",
+ "type": "type2",
+ "arg": "arg"
+ }
+
+]
+```
+
+
+
+#### mock type
+
+| type | 参数 | 说明 | 返回数据类型 | 例子 |
+|:----------------|:--------------------------------------------------------------------------------------------------------------------------------------------------------------------|:--------------------------------------------------------------------------------------:|-------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| Number | min(number):range起始值(包含),默认:0。max(number):range结束值(不包含),默认:int32.max。options(array<number>):number列表,配置options则[start, end)失效,默认:null。random(boolean):随机模式,默认:true。 | 用于生成number | 根据start、end、options的值,推测返回类型为:int或bigint或double | 随机生成[0, 10000)范围的int数据:{"name":"int_random","type":"Number","min":0,"max":10000}递增方式生成[0, 10000)范围的int数据:{"name":"int_inc","type":"Number","min":0,"max":10000,"random":false}从int列表生成int数据:{"name":"int_options","type":"Number","options":[20,22,25,30]}随机生成[0, 10000)范围的double数据:{"name":"double_random","type":"Number","min":0.0,"max":10000.0} |
+| Sequence | start(bigint):range起始值(包含),默认:0。step(bigint):步长,默认:1。 | 用于生成bigint序列, 类似等差数列 | bigint | 生成0,1,2...序列:{"name":"sub_id","type":"Sequence","start":0}生成0,2,4...序列:{"name":"sub_id","type":"Sequence","start":0,"step":2} |
+| UniqueSequence | start(bigint):range起始值(包含),默认:0。 | 用于生成唯一bigint序列,0,1,2...和Sequence的区别: Sequence每个线程单独生成序列 UniqueSequence生成数字整个程序保证唯一 | bigint | 生成0,1,2...序列:{"name":"id","type":"UniqueSequence","start":0} |
+| String | regex(string):根据正则随机生成符合正则的字符串,默认:[a-zA-Z]{0,5}。options(array`<string>`):string列表,配置options则regex失效,默认:null。random(boolean):随机模式,默认:true。 | 用于生成string | string | 随机生成长度我5-10的小写英文字符串:{"name":"str_regex","type":"String","regex":"[a-z]{5,10}"}从string列表生成string数据:{"name":"str_options","type":"String","options":["a","b","c"]} |
+| Timestamp | unit(string):second或millis,生成秒或者毫秒时间戳,默认:second。 | 用于生成时间戳(当前时间) | bigint | 生成unix时间戳:{"name":"timestamp","type":"Timestamp"}生成毫秒时间戳:{"name":"timestamp_ms","type":"Timestamp","unit":"millis"} |
+| FormatTimestamp | format(string):format,默认:yyyy-MM-dd HH:mm:ss。utc(boolean):使用utc时区,默认:false,当地时区。 | 用于生成时间字符串(当前时间) | string | 生成时间字符串:{"name":"timestamp_str","type":"FormatTimestamp","format":"yyyy-MM-dd HH:mm:ss"}生成时间字符串:{"name":"timestamp_str","type":"FormatTimestamp","format":"yyyy-MM-dd HH:mm:ss.SSS"} |
+| IPv4 | start(string):起始值ip(包含),默认:0.0.0.0。end(string):结束ip(包含),默认:255.255.255.255。 | 用于生成start-end范围的ip地址 | string | 随机生成192.168.20.1-192.168.20.255范围的ip地址:{"name":"ip","type":"IPv4","start":"192.168.20.1","end":"192.168.20.255"} |
+| Expression | expression(string):Datafaker expression,默认:null。 | 用于使用datafaker库的expression生成随机字符串文档:https://www.datafaker.net/documentation/expressions | string | 生成人名:{"name":"name","type":"Expression","expression":"#{[Name.name](http://Name.name)}"}生成邮箱地址:{"name":"emailAddress","type":"Expression","expression":"#{internet.emailAddress}"} |
+| Hlld | itemCount(bigint):总基数(总唯一元素数量),默认:1000000。batchCount(int):每次生成的hll随机添加的元素数量,默认:10000。precision(int):hll的精度,范围[4, 18],默认:12。 | 用于生成Hlld Sketch,hll算法的一种实现 | string(字节数组的base64) | 生成ip hll 每次大约包含1000个ip:{ "name": "ip_cnt", "type": "Hlld", "itemCount": 100000, "batchCount": 1000 } |
+| HdrHistogram | max(bigint):histogram最大值,默认:100000。batchCount(int):每次生成的histogram随机添加的元素数量,默认:1000。numberOfSignificantValueDigits(int):histogram的精度,范围[1, 5],默认:1。 | 用于生成HdrHistogram Sketch,一种分位数Histogram Sketch | string(字节数组的base64) | 生成延时的Histogram,每次包含1000个ms延时: { "name": "ms_his", "type": "HdrHistogram", "max": 100000, "batchCount": 1000} |
+| Eval | expression(string):AviatorScript expression,默认:null。 | 计算列,通过其它列计算值AviatorScript文档:https://www.yuque.com/boyan-avfmj/aviatorscript | 返回类型依赖expression,可能为任何类型 | 根据已有的in_bytes(bigint), out_bytes(bigint)列计算bytes(bigint)列其值为其它两个的和:{"name": "bytes", "type": "Eval", "expression": "in_bytes + out_bytes"} |
+| Object | fields(array):每个字段的生成规则,可以使用所有type,默认:null。 | 用于生成struct/object类型fields内容和mock desc文件根配置一样,描述每个字段的生成规则 | struct/object | 生成object:{"name":"object","type":"Object","fields":[{"name":"str","type":"String","regex":"[a-z]{5,10}","nullRate":0.1},{"name":"cate","type":"String","options":["a","b","c"]}]} |
+| Union | unionFields(array):每组字段生成规则,默认:null。每个元素的字段:fields(array):和Object配置一样weight(int):此组字段权重,根据权重按照比例生成数据random(boolean):随机模式,默认:true。 | 用于生成有关联的字段 | 各个字段配置类型 | 生成object_id、item_id字段,当object_id = 10时,item_id从[1, 2, 3, 4, 5]生成数据,当object_id = 20时,item_id从[6, 7]生成数据,第一种数据占比5/7,第二种数据占比2/7 |
+
+- Union 举例
+
+```json
+{
+ "name": "unionFields",
+ "type": "Union",
+ "random": false,
+ "unionFields": [
+ {
+ "weight": 5,
+ "fields": [
+ {
+ "name": "object_id",
+ "type": "Number",
+ "options": [10]
+ },
+ {
+ "name": "item_id",
+ "type": "Number",
+ "options": [1, 2, 3, 4, 5],
+ "random": false
+ }
+ ]
+ },
+ {
+ "weight": 2,
+ "fields": [
+ {
+ "name": "object_id",
+ "type": "Number",
+ "options": [20]
+ },
+ {
+ "name": "item_id",
+ "type": "Number",
+ "options": [6, 7],
+ "random": false
+ }
+ ]
+ }
+ ]
+}
+```
+
+type通用参数:
+
+| 参数 | 说明 | 例子 |
+|-------------------------|-----------------------------------------|-----------------------------------------------------------------------------------------------------------------|
+| nullRate(double) | 生成数据null值比率,默认是1,没有null值。 | 随机生成字符串,null值占10%:{"name":"str_regex","type":"String","regex":"[a-z]{5,10}","nullRate":0.1} |
+| array(double) | 是否是数组类型,默认false。 | 生成数组字符串:{"name":"array_str","type":"String","regex":"[a-z]{5,10}","array":true,"arrayLenMin":1,"arrayLenMax":3} |
+| arrayLenMin(int) | 数组最小长度(包含),默认0。array属性为true时才生效。 | |
+| arrayLenMax(int) | 数组最大长度(包含),默认5。array属性为true时才生效。 | |
+
+#### mock 示例
+
+**各个类型生成查看**
+
+配置:
+
+```json
+[
+ {
+ "name": "id",
+ "type": "UniqueSequence",
+ "start": 0
+ },
+ {
+ "name": "sub_id",
+ "type": "Sequence",
+ "start": 0
+ },
+ {
+ "name": "int_random",
+ "type": "Number",
+ "min": 0,
+ "max": 10000
+ },
+ {
+ "name": "int_inc",
+ "type": "Number",
+ "min": 0,
+ "max": 10000,
+ "random": false
+ },
+ {
+ "name": "int_options",
+ "type": "Number",
+ "options": [20, 22, 25, 30],
+ "random": true
+ },
+ {
+ "name": "int_options_round_robin",
+ "type": "Number",
+ "options": [20, 22, 25, 30],
+ "random": false
+ },
+ {
+ "name": "double_random",
+ "type": "Number",
+ "min": 0.0,
+ "max": 10000.0
+ },
+ {
+ "name": "str_regex",
+ "type": "String",
+ "regex": "[a-z]{5,10}",
+ "nullRate": 0.1
+ },
+ {
+ "name": "str_options",
+ "type": "String",
+ "options": ["a", "b", "c"]
+ },
+ {
+ "name": "str_options_round_robin",
+ "type": "String",
+ "options": ["a", "b", "c"],
+ "random": false
+ },
+ {
+ "name": "timestamp",
+ "type": "Timestamp"
+ },
+ {
+ "name": "timestamp_ms",
+ "type": "Timestamp",
+ "unit": "millis"
+ },
+ {
+ "name": "timestamp_str",
+ "type": "FormatTimestamp",
+ "format": "yyyy-MM-dd HH:mm:ss"
+ },
+ {
+ "name": "ip",
+ "type": "IpV4",
+ "start": "192.168.20.1",
+ "end": "192.168.20.255"
+ },
+ {
+ "name": "array_str",
+ "type": "String",
+ "options": ["a", "b", "c"],
+ "array": true,
+ "arrayLenMin": 1,
+ "arrayLenMax": 3
+ },
+ {
+ "name": "array_object",
+ "type": "Object",
+ "fields": [
+ {
+ "name": "str",
+ "type": "String",
+ "regex": "[a-z]{5,10}",
+ "nullRate": 0.1
+ },
+ {
+ "name": "name",
+ "type": "Expression",
+ "expression": "#{Name.name}"
+ },
+ {
+ "name": "emailAddress",
+ "type": "Expression",
+ "expression": "#{internet.emailAddress}"
+ }
+ ]
+ }
+]
+```
+
+生成数据:
+
+```
+{"id":0,"sub_id":0,"int_random":7604,"int_inc":0,"int_options":30,"int_options_round_robin":20,"double_random":2329.3205359759163,"str_regex":"wxzrpn","str_options":"b","str_options_round_robin":"a","timestamp":1717493414,"timestamp_ms":1717493414603,"timestamp_str":"2024-06-04 17:30:14","ip":"192.168.20.24","array_str":["b"],"array_object":{"str":"wvrzqde","name":"Berry Gorczany","emailAddress":"[email protected]"}}
+{"id":1,"sub_id":1,"int_random":5760,"int_inc":1,"int_options":30,"int_options_round_robin":22,"double_random":9644.141255418077,"str_regex":"oadbz","str_options":"a","str_options_round_robin":"b","timestamp":1717493415,"timestamp_ms":1717493415603,"timestamp_str":"2024-06-04 17:30:15","ip":"192.168.20.127","array_str":["c"],"array_object":{"str":"bkcwtpl","name":"Alba Gottlieb","emailAddress":"[email protected]"}}
+{"id":2,"sub_id":2,"int_random":3775,"int_inc":2,"int_options":20,"int_options_round_robin":25,"double_random":9573.948656302768,"str_regex":"rlhtrk","str_options":"b","str_options_round_robin":"c","timestamp":1717493416,"timestamp_ms":1717493416603,"timestamp_str":"2024-06-04 17:30:16","ip":"192.168.20.20","array_str":["b"],"array_object":{"name":"Celestina O'Reilly","emailAddress":"[email protected]"}}
+{"id":3,"sub_id":3,"int_random":7877,"int_inc":3,"int_options":22,"int_options_round_robin":30,"double_random":8921.757584727951,"str_regex":"spydx","str_options":"c","str_options_round_robin":"a","timestamp":1717493417,"timestamp_ms":1717493417603,"timestamp_str":"2024-06-04 17:30:17","ip":"192.168.20.218","array_str":["a","a"],"array_object":{"name":"Dr. Nichole McGlynn","emailAddress":"[email protected]"}}
+{"id":4,"sub_id":4,"int_random":8248,"int_inc":4,"int_options":30,"int_options_round_robin":20,"double_random":4105.3600047674545,"str_regex":"rbjelg","str_options":"b","str_options_round_robin":"b","timestamp":1717493418,"timestamp_ms":1717493418602,"timestamp_str":"2024-06-04 17:30:18","ip":"192.168.20.146","array_str":["b"],"array_object":{"str":"ekbyer","name":"Raul Leannon","emailAddress":"[email protected]"}}
+{"id":5,"sub_id":5,"int_random":3663,"int_inc":5,"int_options":22,"int_options_round_robin":22,"double_random":7486.737315942628,"str_regex":"qyqqiyj","str_options":"c","str_options_round_robin":"c","timestamp":1717493419,"timestamp_ms":1717493419610,"timestamp_str":"2024-06-04 17:30:19","ip":"192.168.20.90","array_str":["c","b"],"array_object":{"str":"dbepb","name":"Moshe Powlowski","emailAddress":"[email protected]"}}
+{"id":6,"sub_id":6,"int_random":6967,"int_inc":6,"int_options":22,"int_options_round_robin":25,"double_random":6742.751027323034,"str_regex":"slfghf","str_options":"a","str_options_round_robin":"a","timestamp":1717493420,"timestamp_ms":1717493420602,"timestamp_str":"2024-06-04 17:30:20","ip":"192.168.20.72","array_str":["b","b"],"array_object":{"name":"Alvera Graham","emailAddress":"[email protected]"}}
+{"id":7,"sub_id":7,"int_random":5340,"int_inc":7,"int_options":25,"int_options_round_robin":30,"double_random":7259.505902869291,"str_regex":"yarcof","str_options":"c","str_options_round_robin":"b","timestamp":1717493421,"timestamp_ms":1717493421614,"timestamp_str":"2024-06-04 17:30:21","ip":"192.168.20.44","array_str":["a"],"array_object":{"str":"dxianwxv","name":"Pedro Kerluke","emailAddress":"[email protected]"}}
+{"id":8,"sub_id":8,"int_random":8365,"int_inc":8,"int_options":25,"int_options_round_robin":20,"double_random":7142.049302311821,"str_options":"c","str_options_round_robin":"c","timestamp":1717493422,"timestamp_ms":1717493422603,"timestamp_str":"2024-06-04 17:30:22","ip":"192.168.20.197","array_str":["b"],"array_object":{"str":"mximiyd","name":"Herman Runte","emailAddress":"[email protected]"}}
+{"id":9,"sub_id":9,"int_random":5944,"int_inc":9,"int_options":30,"int_options_round_robin":22,"double_random":1420.8479774375382,"str_regex":"eahpq","str_options":"b","str_options_round_robin":"a","timestamp":1717493423,"timestamp_ms":1717493423602,"timestamp_str":"2024-06-04 17:30:23","ip":"192.168.20.44","array_str":["a","a","b"],"array_object":{"str":"kseeqicxuh","name":"Kaitlyn Douglas","emailAddress":"[email protected]"}}
+{"id":10,"sub_id":10,"int_random":9357,"int_inc":10,"int_options":30,"int_options_round_robin":25,"double_random":2451.2488213660886,"str_regex":"agwxbf","str_options":"b","str_options_round_robin":"b","timestamp":1717493424,"timestamp_ms":1717493424607,"timestamp_str":"2024-06-04 17:30:24","ip":"192.168.20.19","array_str":["b","c"],"array_object":{"str":"iidogsi","name":"Luigi McClure PhD","emailAddress":"[email protected]"}}
+```
+
+**object类型以及Union类型生成**
+
+配置:
+
+```json
+[
+ { "name": "name", "type": "String", "options": ["object_statistics"] },
+ { "name": "timestamp_ms", "type": "Timestamp", "unit": "millis" },
+ { "name": "tags", "type": "Object", "fields": [
+ { "name": "vsys_id", "type": "Number", "options": [1] },
+ { "name": "template_id", "type": "Number", "options": [1] },
+ { "name": "chart_id", "type": "Number", "options": [1] },
+ { "name": "version", "type": "Number", "options": [1] },
+ { "name": "unionFields", "type": "Union", "unionFields": [
+ { "weight": 2, "fields": [
+ { "name": "object_type", "type": "String", "options": ["ip"] },
+ { "name": "object_id", "type": "Number", "options": [7562] },
+ { "name": "item_id", "type": "Number", "options": [7835, 7819] }
+ ]
+ },
+ { "weight": 2, "fields": [
+ { "name": "object_type", "type": "String", "options": ["fqdn"] },
+ { "name": "object_id", "type": "Number", "options": [13087] },
+ { "name": "item_id", "type": "Number", "options": [229604,229603] }
+ ]
+ }
+ ]
+ }
+ ]
+ },
+ { "name": "fields", "type": "Object", "fields": [
+ { "name": "in_bytes", "type": "Number", "min": 10000, "max": 200000},
+ { "name": "out_bytes", "type": "Number", "min": 10000, "max": 200000},
+ { "name": "new_in_sessions", "type": "Number", "min": 10, "max": 200},
+ { "name": "new_out_sessions", "type": "Number", "min": 10, "max": 200}
+ ]
+ }
+]
+```
+
+生成数据:
+
+```
+{"name":"object_statistics","timestamp_ms":1717573879804,"tags":{"vsys_id":1,"template_id":1,"chart_id":1,"version":1,"object_type":"ip","object_id":7562,"item_id":7819},"fields":{"in_bytes":47083,"out_bytes":68389,"new_in_sessions":142,"new_out_sessions":92}}
+{"name":"object_statistics","timestamp_ms":1717573879807,"tags":{"vsys_id":1,"template_id":1,"chart_id":1,"version":1,"object_type":"fqdn","object_id":13087,"item_id":229603},"fields":{"in_bytes":81118,"out_bytes":107287,"new_in_sessions":98,"new_out_sessions":86}}
+{"name":"object_statistics","timestamp_ms":1717573879808,"tags":{"vsys_id":1,"template_id":1,"chart_id":1,"version":1,"object_type":"ip","object_id":7562,"item_id":7835},"fields":{"in_bytes":61395,"out_bytes":111095,"new_in_sessions":87,"new_out_sessions":149}}
+{"name":"object_statistics","timestamp_ms":1717573879808,"tags":{"vsys_id":1,"template_id":1,"chart_id":1,"version":1,"object_type":"fqdn","object_id":13087,"item_id":229603},"fields":{"in_bytes":145986,"out_bytes":12166,"new_in_sessions":169,"new_out_sessions":127}}
+{"name":"object_statistics","timestamp_ms":1717573880806,"tags":{"vsys_id":1,"template_id":1,"chart_id":1,"version":1,"object_type":"fqdn","object_id":13087,"item_id":229604},"fields":{"in_bytes":112797,"out_bytes":120310,"new_in_sessions":12,"new_out_sessions":177}}
+{"name":"object_statistics","timestamp_ms":1717573880806,"tags":{"vsys_id":1,"template_id":1,"chart_id":1,"version":1,"object_type":"fqdn","object_id":13087,"item_id":229604},"fields":{"in_bytes":180960,"out_bytes":118214,"new_in_sessions":106,"new_out_sessions":73}}
+{"name":"object_statistics","timestamp_ms":1717573880806,"tags":{"vsys_id":1,"template_id":1,"chart_id":1,"version":1,"object_type":"ip","object_id":7562,"item_id":7819},"fields":{"in_bytes":91394,"out_bytes":105840,"new_in_sessions":74,"new_out_sessions":177}}
+{"name":"object_statistics","timestamp_ms":1717573880806,"tags":{"vsys_id":1,"template_id":1,"chart_id":1,"version":1,"object_type":"ip","object_id":7562,"item_id":7835},"fields":{"in_bytes":79266,"out_bytes":95721,"new_in_sessions":50,"new_out_sessions":88}}```
+```
+
+### Inline Source
+
+用于简单测试format,function,sink等,这个source每个1s发送一条配置的data数据
+
+```yaml
+sources:
+ inline_source:
+ type : inline
+ fields:
+ - name: log_id
+ type: bigint
+ - name: recv_time
+ type: bigint
+ - name: client_ip
+ type: string
+ properties:
+ # 单条数据
+ data: '{"log_id": 1, "recv_time":"111", "client_ip":"192.168.0.1"}'
+ # 多条数据
+ # data: '[{"log_id": 1, "recv_time":"111", "client_ip":"192.168.0.1"}, {"log_id": 2, "recv_time":"222", "client_ip":"192.168.0.2"}]'
+ # data: '["1,111,192.168.0.1", "2,222,192.168.0.2"]'
+ format: json
+ json.ignore.parse.errors: false
+```
+
+| 属性名 | 必填 | 默认值 | 类型 | 描述 |
+|-------------------|----|--------|----------|---------------------------------------------------|
+| **data** | Y | - | String | source发送的数据,如果是json array形式则当做单独解析发送array每个元素 |
+| **format** | Y | - | String | 使用的format |
+| **type** | N | string | String | 数据类型:string(UTF8字符串),hex(十六进制编码),base64(base64编码) |
+| interval.per.row | N | 1s | Duration | 发送每行数据间隔时间 |
+| repeat.count | N | -1 | Integer | 重复发送data测试,负数则一直循环重复发送 |
+| format properties | N | - | String | format properties配置,key为format值.+原始key |
+
+## 过滤器(Filters)
+
+```yaml
+filters:
+ http_filter:
+ type: aviator
+ properties:
+ expression: event.decoded_as == 'HTTP' && event.server_port = 80
+```
+
+| 属性名 | 默认值 | 类型 | 必填 | 描述 |
+|----------------|-----|--------|----|------------------------------------|
+| **name** | - | String | Y | 过滤器名称,唯一标识,用于任务编排。例如:“http_filter“ |
+| **type** | - | String | Y | 数据源类型。例如:aviator |
+| **properties** | | | | |
+| expression | - | String | N | 基于AviatorScript语法,过滤符合条件的事件; |
+
+## 分流器(Splits)
+
+```yaml
+splits:
+ decoded_as_split:
+ type: split
+ rules:
+ - tag: http_tag
+ expression: event.decoded_as == 'HTTP'
+ - tag: dns_tag
+ expression: event.decoded_as == 'DNS'
+```
+
+| 属性名 | 默认值 | 类型 | 必填 | 描述 |
+|------------|-----|--------|----|-----------------------------------------|
+| **name** | - | String | Y | 过滤器名称,唯一标识,用于任务编排。例如:“decode_as_filter“ |
+| **type** | - | String | Y | 数据源类型。例如:split |
+| **rules** | | | | |
+| tag | - | String | Y | 分流标记,同时需要在topology中配置,具体参见任务编排 |
+| expression | - | String | Y | 基于AviatorScript语法,将符合条件的数据分流至下游算子; |
+
+
+## 任务处理器 (Processors)
+
+### Pre-processing Pipeline
+
+```yaml
+pre_processing_pipelines:
+ common_pre_processor:
+ type: projection
+ output_fields: []
+ functions:
+ - function: CURRENT_UNIX_TIMESTAMP
+ lookup_fields: []
+ output_fields: [processing_time]
+ parameters:
+ precision: milliseconds
+```
+
+### Processing Pipeline
+
+```yaml
+processing_pipelines:
+ session_record_processor:
+ type: projection
+ output_fields: []
+ functions:
+ - function: DOMAIN
+ lookup_fields: [http_host, ssl_sni, quic_sni]
+ output_fields: [server_domain]
+ option: FIRST_SIGNIFICANT_SUBDOMAIN
+ - function: ASN_LOOKUP
+ lookup_fields: [server_ip]
+ output_fields: [server_asn]
+ parameters:
+ option: IP_TO_ASN
+ vendor_id: tsg_asnlookup
+ - name: BASE64_DECODE_TO_STRING
+ lookup_fields: [mail_subject,mail_subject_charset]
+ output_fields: [mail_subject]
+ aggregate_processor:
+ type: aggregate
+ group_by_fields: [server_ip,server_port,client_ip,client_port]
+ window_type: tumbling_processing_time # tumbling_event_time,sliding_processing_time,sliding_event_time
+ window_size: 60
+ window_slide: 10 #滑动窗口步长
+ mini_batch: true #是否开启预聚合优化
+ functions:
+ - function: NUMBER_SUM
+ lookup_fields: [ sent_pkts ]
+ output_fields: [ sent_pkts_sum ]
+ table_processor:
+ type: table
+ functions:
+ - function: JSON_UNROLL
+ lookup_fields: [ device_tag ]
+ output_fields: [ new_name2 ]
+ parameters:
+ path: tags
+ new_path: newtags
+```
+
+#### Projection Processor
+
+| 属性名 | 默认值 | 类型 | 必填 | 描述 |
+|---------------|-----|---------------|----|-------------------------|
+| name | - | String | Y | Processor名称,唯一标识,用于任务编排 |
+| type | - | String | Y | 数据源类型:projection |
+| output_fields | - | Array(String) | N | 输出指定字段,默认发送全部字段。 |
+| remove_fields | - | Array(String) | N | 删除指定字段,默认为空。 |
+| functions | - | List(UDF) | Y | 自定义函数列表 |
+| | | | | |
+
+#### Aggregate Processor
+
+| 属性名 | 默认值 | 类型 | 必填 | 描述 |
+|------------------------|-----|------------|----|------------------------------------------------------------------------------------------------|
+| name | - | String | Y | Processor名称,唯一标识,用于任务编排 |
+| type | - | String | Y | 数据源类型:aggregate |
+| group_by_fields | - | Integer | Y | 聚合的维度列 |
+| window_type | - | Enum | Y | 时间窗口类型:tumbling_processing_time,tumbling_event_time,sliding_processing_time,sliding_event_time |
+| window_size | - | Integer | Y | 窗口的大小,单位秒 |
+| window_slide | - | Integer | N | 滑动窗口需要指定滑动步长,单位秒 |
+| window_timestamp_field | - | String | N | 窗口开始的时间戳(ms)做为value输出的字段名 |
+| mini_batch | - | Boolean | N | 默认为false,是否开启预聚合优化,在按照key进行聚合之前,先在本地进行汇聚,进而降低网络传输数据量 |
+| functions | - | List(UDAF) | Y | 自定义函数列表 |
+
+#### Table Processor
+
+| 属性名 | 默认值 | 类型 | 必填 | 描述 |
+|-----------|-----|------------|----|-------------------------|
+| name | - | String | Y | Processor名称,唯一标识,用于任务编排 |
+| type | - | String | Y | 数据源类型:table |
+| functions | - | List(UDTF) | Y | 自定义函数列表 |
+
+## 输出(Sinks)
+
+### Sink通用配置
+
+```yaml
+sinks:
+ kafka_sink:
+ # sink标识
+ type: kafka
+ # sink schema
+ # schema:
+ # sink属性配置
+ properties:
+ prop_key1: prop_value1
+ prop_key2: prop_value2
+ #...
+```
+
+
+
+| 属性名 | 必填 | 默认值 | 类型 | 描述 |
+|----------------|-------|---------|---------|------------------|
+| **type** | Y | - | String | sink唯一标识 |
+| `schema` | N | - | Map | 同source schema |
+| properties | Y | - | Object | sink属性配置 |
+
+### Kafka Sink
+
+```yaml
+sinks:
+ kafka_sink:
+ type: kafka
+ properties:
+ topic: SESSION-RECORD-JSON
+ kafka.bootstrap.servers: 192.168.44.12:9092
+ kafka.retries: 0
+ kafka.linger.ms: 10
+ kafka.request.timeout.ms: 30000
+ kafka.batch.size: 262144
+ kafka.buffer.memory: 134217728
+ kafka.max.request.size: 10485760
+ kafka.compression.type: snappy
+ format: json
+```
+
+| 属性名 | 必填 | 默认值 | 类型 | 描述 |
+|------------------------------------|----|--------|----------|--------------------------------------------------------|
+| **topic** | Y | - | String | Kafka Topic名称。 |
+| **kafka.bootstrap.servers** | Y | - | String | Kafka Broker 地址 |
+| **format** | Y | - | String | format,用来序列化消息JSONProtobufCSV... |
+| log.failures.only | N | true | Boolean | producer发生error时只打印日志, 否则抛出异常程序停止(重试) |
+| rate.limiting.strategy | N | none | String | 限速策略:none:不限速(默认)sliding_window:限速,使用滑动窗口计算速率 |
+| rate.limiting.limit.rate | N | 10Mbps | String | 限制的最大速率:单位必须是Mbps、Kbps、bps,例如:10Mbps, 10Kbps, 10240bps |
+| rate.limiting.window.size | N | 5 | Integer | 窗口大小,单位秒 |
+| rate.limiting.block.duration | N | 5min | Duration | 对首次超出限流数据阻塞,最长阻塞多长时间后超出限流数据全部丢弃 |
+| rate.limiting.block.reset.duration | N | 30s | Duration | 超速阻塞后速率恢复正常多长时间后重置超速阻塞状态 |
+| Kafka properties | N | - | String | kafka consumer/producer properties配置,key为kafka.+原始key |
+| format properties | N | - | String | format properties配置,key为format值.+原始key |
+
+### ClickHouse Sink
+
+```yaml
+sinks:
+ clickhouse_sink:
+ type: clickhouse
+ properties:
+ host: 192.168.40.222:9001,192.168.40.223:9001
+ table: tsg_galaxy_v3.session_record_local_old
+ batch.size: 100000
+ batch.interval: 30s
+ connection.user: default
+ connection.password: galaxy2019
+```
+
+| 属性名 | 必填 | 默认值 | 类型 | 描述 |
+|----------------------------|----|---------|------------|---------------------------------------------------------------|
+| **host** | Y | - | String | clickhouse host和tcp port信息。格式:host1:port,host2:port ...。 |
+| **table** | Y | - | String | clickhouse table name. |
+| **batch.size** | N | 100000 | Integer | 最大flush size,超过size会立刻flush。 |
+| **batch.byte.size** | N | 200mb | MemorySize | 最大flush buffer字节大小,超过会立刻flush。 |
+| **batch.interval** | N | 30s | Duration | 最大flush间隔,超过会立刻flush。 |
+| connection.user | Y | - | String | clickhouse 连接 用户名 |
+| connection.password | Y | - | String | clickhouse 连接 密码 |
+| connection.database | N | default | String | clickhouse 连接 默认数据库 |
+| connection.connect_timeout | N | 30 | Integer | 连接超时(单位秒) |
+| connection.query_timeout | N | 300 | Integer | 查询超时(单位秒) |
+| connection properties | N | - | String | clickhouse jdbc connection properties配置,key为connection.+原始key |
+
+### Print Sink
+
+用来测试的sink,把元素输出到标准输出或输出日志。
+
+```yaml
+sinks:
+ print_sink:
+ type: print
+ properties:
+ format: json
+```
+
+| 属性名 | 必填 | 默认值 | 类型 | 描述 |
+|-------------------|----|--------|--------|----------------------------------------|
+| **format** | Y | - | String | format,用来序列化消息 |
+| **mode** | N | stdout | Enum | 输出模式,可选值:stdout,log_info,log_warn,null |
+| format properties | N | - | String | format properties配置,key为format值.+原始key |
+
+## Formats
+
+### JSON
+
+```yaml
+sources:
+ kafka_source:
+ type : kafka
+ properties:
+ topic: SESSION-RECORD-COMPLETED
+ kafka.bootstrap.servers: 192.168.44.11:9092
+ kafka.session.timeout.ms: 60000
+ kafka.max.poll.records: 3000
+ kafka.max.partition.fetch.bytes: 31457280
+ kafka.group.id: SESSION-RECORD-COMPLETED-GROUP-GROOT-STREAM-20231021
+ kafka.auto.offset.reset: latest
+ format: json
+ json.ignore.parse.errors: true
+```
+
+| 属性名 | 必填 | 默认值 | 类型 | 描述 |
+|---------------------|----|-------|---------|------------------------|
+| ignore.parse.errors | N | false | Boolean | json解析时发生错误时忽略,否则抛出异常。 |
+
+### MessagePack
+
+```yaml
+kafka_source_msgpack:
+ type : kafka
+ properties:
+ topic: msgpack-test
+ format: msgpack
+ kafka.bootstrap.servers: 192.168.44.12:9092
+ kafka.session.timeout.ms: 60000
+ kafka.max.poll.records: 3000
+ kafka.max.partition.fetch.bytes: 31457280
+ kafka.group.id: msgpack-test
+ kafka.auto.offset.reset: latest
+
+inline_source_msgpack:
+ type : inline
+ properties:
+ data: g6Zsb2dfaWQBqXJlY3ZfdGltZc5mF3f5qWNsaWVudF9pcKsxOTIuMTY4LjAuMQ==
+ type: base64
+ format: msgpack
+```
+
+- 只需要指定format为msgpack,没有其它的参数。
+
+- 支持所有数据类型的解析,包括复杂数据类型struct,array,以及binary。
+
+### Protobuf
+
+```yaml
+sources:
+ inline_source_protobuf:
+ type : inline
+ properties:
+ data: CIin2awGEICAoLC/hYzKAhoEQkFTRSCch8z3wtqEhAQo6o/Xmc0xMMCy15nNMTjWIkDRCEiIp9msBlCIp9msBloIMjE0MjYwMDNg//8DaP//A3JqeyJ0YWdzIjpbeyJ0YWciOiJkYXRhX2NlbnRlciIsInZhbHVlIjoiY2VudGVyLXh4Zy05MTQwIn0seyJ0YWciOiJkZXZpY2VfZ3JvdXAiLCJ2YWx1ZSI6Imdyb3VwLXh4Zy05MTQwIn1dfXoPY2VudGVyLXh4Zy05MTQwggEOZ3JvdXAteHhnLTkxNDCKAQ0xOTIuMTY4LjQwLjgxkAEEmAEBoAEBqAGQwAGyAQdbMSwxLDJd4gEDt+gY4gINMTkyLjU2LjE1MS44MOgCoeYD8gIHV2luZG93c/oCGOe+juWbvS5Vbmtub3duLlVua25vd24uLrIDDTE5Mi41Ni4yMjIuOTO4A/ZwwgMFTGludXjKAxjnvo7lm70uVW5rbm93bi5Vbmtub3duLi6SBAN0Y3CaBBFFVEhFUk5FVC5JUHY0LlRDULAMBLgMBcAM9gHIDJEOoA2AAagN8cr+jgKwDezksIAPwg0RYTI6ZmE6ZGM6NTY6Yzc6YjPKDRE0ODo3Mzo5Nzo5NjozODoyMNINETQ4OjczOjk3Ojk2OjM4OjIw2g0RYTI6ZmE6ZGM6NTY6Yzc6YjM=
+ type: base64
+ format: protobuf
+ protobuf.descriptor.file.path: ./config/session_record_test.desc
+ protobuf.message.name: SessionRecord
+```
+
+
+
+| 属性名 | 必填 | 默认值 | 类型 | 描述 |
+|----------------------|----|-------|---------|-----------------------------------------------------------------|
+| descriptor.file.path | Y | - | String | The Protobuf descriptor file path. |
+| message.name | Y | - | String | The protobuf MessageName to look for in the descriptor file. |
+| ignore.parse.errors | N | false | Boolean | protobuf解析时发生错误时忽略,否则抛出异常。 |
+| emit.default.values | N | false | Boolean | protobuf解析时是否设置默认值。不建议配置,严重影响性能。基本数据类型建议使用optional配置来显式处理null值。 |
+
+protobuf 类型与内置类型对应表:
+
+| protobuf类型 | 类型(原始对应类型) | 可以转换的类型 | 描述 |
+|--------------------------------------|------------|----------------------------------------------|---------------------------------------------------------------------|
+| int3,uint32,sint32,fixed32,sfixed32 | int | int, bigint, float, double(序列化时支持string类型转换) | 建议使用int32 其次使用sint32,不建议使用uint32(java读取出来是int类型 第一位代表符号位,可能读取出来是负数) |
+| int64,uint64,sint64,fixed64,sfixed64 | bigint | int, bigint, float, double(序列化时支持string类型转换) | 建议使用int64,其次使用sint64 |
+| float | float | int, bigint, float, double(序列化时支持string类型转换) | 建议使用double |
+| double | double | int, bigint, float, double(序列化时支持string类型转换) | 建议使用double |
+| bool | boolean | boolean, int(0为false, 非0为true) | 不建议使用bool,使用int32代替 |
+| enum | int | int | 不建议使用enum,使用int32代替 |
+| string | string | string(序列化时支持所有类型,调用toString方法) | |
+| bytes | binary | binary | |
+| message (结构体类型) | struct | struct | |
+| repeated type (数组类型) | array | array | |
+
+protobuf format使用步骤:
+
+1. 定义proto文件(只支持proto3语法),int double等数值类型有null值时添加optional,建议int double总是添加optional选项。
+2. 生成desc二进制文件(使用23.4版本)
+
+示例:定义proto文件
+
+```
+syntax = "proto3";
+
+// [START java_declaration]
+// option java_multiple_files = true;
+option java_package = "com.geedgenetworks.proto";
+option java_outer_classname = "SessionRecordProtos";
+// [END java_declaration]
+
+message SessionRecord {
+ optional int64 recv_time = 1;
+ optional int64 log_id = 2;
+ string decoded_as = 3;
+ optional int64 session_id = 4;
+ optional int64 start_timestamp_ms = 5;
+ optional int64 end_timestamp_ms = 6;
+ optional int32 duration_ms = 7;
+ optional int32 tcp_handshake_latency_ms = 8;
+ optional int64 ingestion_time = 9;
+ optional int64 processing_time = 10;
+ string device_id = 11;
+ optional int32 out_link_id = 12;
+ optional int32 in_link_id = 13;
+ string device_tag = 14;
+ string data_center = 15;
+ string device_group = 16;
+ string sled_ip = 17;
+ optional int32 address_type = 18;
+ optional int32 vsys_id = 19;
+ optional int32 t_vsys_id = 20;
+ optional int64 flags = 21;
+ string flags_identify_info = 22;
+ repeated int64 security_rule_list = 23;
+ string security_action = 24;
+ repeated int64 monitor_rule_list = 25;
+ repeated int64 shaping_rule_list = 26;
+ repeated int64 proxy_rule_list = 27;
+ repeated int64 statistics_rule_list = 28;
+ repeated int64 sc_rule_list = 29;
+ repeated int64 sc_rsp_raw = 30;
+ repeated int64 sc_rsp_decrypted = 31;
+ string proxy_action = 32;
+ optional int32 proxy_pinning_status = 33;
+ optional int32 proxy_intercept_status = 34;
+ string proxy_passthrough_reason = 35;
+ optional int32 proxy_client_side_latency_ms = 36;
+ optional int32 proxy_server_side_latency_ms = 37;
+ string proxy_client_side_version = 38;
+ string proxy_server_side_version = 39;
+ optional int32 proxy_cert_verify = 40;
+ string proxy_intercept_error = 41;
+ optional int32 monitor_mirrored_pkts = 42;
+ optional int32 monitor_mirrored_bytes = 43;
+ string client_ip = 44;
+ optional int32 client_port = 45;
+ string client_os_desc = 46;
+ string client_geolocation = 47;
+ optional int64 client_asn = 48;
+ string subscriber_id = 49;
+ string imei = 50;
+ string imsi = 51;
+ string phone_number = 52;
+ string apn = 53;
+ string server_ip = 54;
+ optional int32 server_port = 55;
+ string server_os_desc = 56;
+ string server_geolocation = 57;
+ optional int64 server_asn = 58;
+ string server_fqdn = 59;
+ string server_domain = 60;
+ string app_transition = 61;
+ string app = 62;
+ string app_debug_info = 63;
+ string app_content = 64;
+ repeated int64 fqdn_category_list = 65;
+ string ip_protocol = 66;
+ string decoded_path = 67;
+ optional int32 dns_message_id = 68;
+ optional int32 dns_qr = 69;
+ optional int32 dns_opcode = 70;
+ optional int32 dns_aa = 71;
+ optional int32 dns_tc = 72;
+ optional int32 dns_rd = 73;
+ optional int32 dns_ra = 74;
+ optional int32 dns_rcode = 75;
+ optional int32 dns_qdcount = 76;
+ optional int32 dns_ancount = 77;
+ optional int32 dns_nscount = 78;
+ optional int32 dns_arcount = 79;
+ string dns_qname = 80;
+ optional int32 dns_qtype = 81;
+ optional int32 dns_qclass = 82;
+ string dns_cname = 83;
+ optional int32 dns_sub = 84;
+ string dns_rr = 85;
+ optional int32 dns_response_latency_ms = 86;
+ string http_url = 87;
+ string http_host = 88;
+ string http_request_line = 89;
+ string http_response_line = 90;
+ string http_request_body = 91;
+ string http_response_body = 92;
+ optional int32 http_proxy_flag = 93;
+ optional int32 http_sequence = 94;
+ string http_cookie = 95;
+ string http_referer = 96;
+ string http_user_agent = 97;
+ optional int64 http_request_content_length = 98;
+ string http_request_content_type = 99;
+ optional int64 http_response_content_length = 100;
+ string http_response_content_type = 101;
+ string http_set_cookie = 102;
+ string http_version = 103;
+ optional int32 http_status_code = 104;
+ optional int32 http_response_latency_ms = 105;
+ optional int32 http_session_duration_ms = 106;
+ optional int64 http_action_file_size = 107;
+ string ssl_version = 108;
+ string ssl_sni = 109;
+ string ssl_san = 110;
+ string ssl_cn = 111;
+ optional int32 ssl_handshake_latency_ms = 112;
+ string ssl_ja3_hash = 113;
+ string ssl_ja3s_hash = 114;
+ string ssl_cert_issuer = 115;
+ string ssl_cert_subject = 116;
+ optional int32 ssl_esni_flag = 117;
+ optional int32 ssl_ech_flag = 118;
+ string dtls_cookie = 119;
+ string dtls_version = 120;
+ string dtls_sni = 121;
+ string dtls_san = 122;
+ string dtls_cn = 123;
+ optional int32 dtls_handshake_latency_ms = 124;
+ string dtls_ja3_fingerprint = 125;
+ string dtls_ja3_hash = 126;
+ string dtls_cert_issuer = 127;
+ string dtls_cert_subject = 128;
+ string mail_protocol_type = 129;
+ string mail_account = 130;
+ string mail_from_cmd = 131;
+ string mail_to_cmd = 132;
+ string mail_from = 133;
+ string mail_password = 134;
+ string mail_to = 135;
+ string mail_cc = 136;
+ string mail_bcc = 137;
+ string mail_subject = 138;
+ string mail_subject_charset = 139;
+ string mail_attachment_name = 140;
+ string mail_attachment_name_charset = 141;
+ string mail_eml_file = 142;
+ string ftp_account = 143;
+ string ftp_url = 144;
+ string ftp_link_type = 145;
+ string quic_version = 146;
+ string quic_sni = 147;
+ string quic_user_agent = 148;
+ string rdp_cookie = 149;
+ string rdp_security_protocol = 150;
+ string rdp_client_channels = 151;
+ string rdp_keyboard_layout = 152;
+ string rdp_client_version = 153;
+ string rdp_client_name = 154;
+ string rdp_client_product_id = 155;
+ string rdp_desktop_width = 156;
+ string rdp_desktop_height = 157;
+ string rdp_requested_color_depth = 158;
+ string rdp_certificate_type = 159;
+ optional int32 rdp_certificate_count = 160;
+ optional int32 rdp_certificate_permanent = 161;
+ string rdp_encryption_level = 162;
+ string rdp_encryption_method = 163;
+ string ssh_version = 164;
+ string ssh_auth_success = 165;
+ string ssh_client_version = 166;
+ string ssh_server_version = 167;
+ string ssh_cipher_alg = 168;
+ string ssh_mac_alg = 169;
+ string ssh_compression_alg = 170;
+ string ssh_kex_alg = 171;
+ string ssh_host_key_alg = 172;
+ string ssh_host_key = 173;
+ string ssh_hassh = 174;
+ string sip_call_id = 175;
+ string sip_originator_description = 176;
+ string sip_responder_description = 177;
+ string sip_user_agent = 178;
+ string sip_server = 179;
+ string sip_originator_sdp_connect_ip = 180;
+ optional int32 sip_originator_sdp_media_port = 181;
+ string sip_originator_sdp_media_type = 182;
+ string sip_originator_sdp_content = 183;
+ string sip_responder_sdp_connect_ip = 184;
+ optional int32 sip_responder_sdp_media_port = 185;
+ string sip_responder_sdp_media_type = 186;
+ string sip_responder_sdp_content = 187;
+ optional int32 sip_duration_s = 188;
+ string sip_bye = 189;
+ optional int32 rtp_payload_type_c2s = 190;
+ optional int32 rtp_payload_type_s2c = 191;
+ string rtp_pcap_path = 192;
+ optional int32 rtp_originator_dir = 193;
+ string stratum_cryptocurrency = 194;
+ string stratum_mining_pools = 195;
+ string stratum_mining_program = 196;
+ string stratum_mining_subscribe = 197;
+ optional int64 sent_pkts = 198;
+ optional int64 received_pkts = 199;
+ optional int64 sent_bytes = 200;
+ optional int64 received_bytes = 201;
+ optional int64 tcp_c2s_ip_fragments = 202;
+ optional int64 tcp_s2c_ip_fragments = 203;
+ optional int64 tcp_c2s_lost_bytes = 204;
+ optional int64 tcp_s2c_lost_bytes = 205;
+ optional int64 tcp_c2s_o3_pkts = 206;
+ optional int64 tcp_s2c_o3_pkts = 207;
+ optional int64 tcp_c2s_rtx_pkts = 208;
+ optional int64 tcp_s2c_rtx_pkts = 209;
+ optional int64 tcp_c2s_rtx_bytes = 210;
+ optional int64 tcp_s2c_rtx_bytes = 211;
+ optional int32 tcp_rtt_ms = 212;
+ optional int64 tcp_client_isn = 213;
+ optional int64 tcp_server_isn = 214;
+ string packet_capture_file = 215;
+ string in_src_mac = 216;
+ string out_src_mac = 217;
+ string in_dest_mac = 218;
+ string out_dest_mac = 219;
+ string tunnels = 220;
+ optional int32 dup_traffic_flag = 221;
+ string tunnel_endpoint_a_desc = 222;
+ string tunnel_endpoint_b_desc = 223;
+}
+```
+
+生成desc二进制文件
+
+```
+protoc --descriptor_set_out=session_record_test.desc session_record_test.proto
+```
+
+### Raw
+
+Raw format允许读写原始(字节数组)值作为单个列。主要用于不涉及修改message从kakfa到kakfa同步topic场景。只需要指定format为raw,没有其它的参数。
+
+```yaml
+
+sources:
+ inline_source:
+ type: inline
+ properties:
+ data: 123abc
+ format: raw
+
+sinks:
+ print_sink:
+ type: print
+ properties:
+ format: raw
+```
+
+# 任务编排
+
+```yaml
+application:
+ env:
+ name: example-inline-to-print
+ parallelism: 3
+ shade.identifier: aes
+ kms.type: local
+ pipeline:
+ object-reuse: true
+ execution:
+ restart:
+ strategy: none
+ properties: # job级别变量,同名情况下会覆盖全局变量
+ hos.bucket.name.rtp_file: traffic_rtp_file_bucket
+ hos.bucket.name.http_file: traffic_http_file_bucket
+ hos.bucket.name.eml_file: traffic_eml_file_bucket
+ hos.bucket.name.policy_capture_file: traffic_policy_capture_file_bucket
+ # RestfulAPI 取需要加密的字段,返回数据类型为Array
+ projection.encrypt.schema.registry.uri: 127.0.0.1:9999/v1/schema/session_record?option=encrypt_fields
+ topology:
+ - name: inline_source
+ downstream: [decoded_as_split]
+ - name: decoded_as_split
+ tags: [http_tag, dns_tag] #需要在分流处理器的rules中进行定义,分流规则按照数组中的顺序对应downstream中的处理器,支持Pipelines,Sinks,Filters
+ downstream: [ projection_processor, aggregate_processor]
+ - name: projection_processor
+ downstream: [ print_sink ]
+ - name: aggregate_processor
+ downstream: [ print_sink ]
+ - name: print_sink
+ downstream: []
+```
+
+# 函数定义
+
+## 内置UDF
+
+ 函数可读取job配置文件(grootstream_job.yaml),每个函数在处理器管道中(Processor Pipeline )独立运行,互不影响。一个函数包括名称、传递数据(Event)、函数上下文信息(UDF Context) 及执行方法 evaluate(Event)。
+
+- Function Name :函数名称,命名全大写单词之间用下划线分割,用于函数注册。
+- Event:处理的事件,数据组织Map<field_name, field_value> event结构。
+- UDF Context 函数执行环境上下文,包括输入数据,配置信息及其它状态信息。
+ - filter :过滤表达式;String类型,默认为空,它用于筛选需要经过函数处理的事件,具体过滤方式参考AviatorScript语法。
+ - lookup_fields:查找的字段;Array[String]类型,允许指定多个字段,在事件中查找字段名对应的值。
+ - output_fields:输出的字段;Array[String]类型,允许指定多个字段,用于将函数执行的结果附加到事件中。如果输出字段与查找字段相匹配,它们将覆盖原有字段的值;如果不匹配,将会在日志中添加一个新字段。
+ - parameters:扩展参数;选填,Map<String, Object>
+
+
+
+> 函数表达式:FUNCTION_NAME(filter, lookup_fields, output_fields[, parameters])
+
+### 标量函数
+
+ #### ASN Lookup
+
+查找IP所属AS号。
+
+- Parameters
+ - kb_name=`<string>` // 使用的知识库的名称 ,需要预先在全局配置中进行注册。
+ - option = `<string>`
+ - IP_TO_ASN
+
+```yaml
+ - function: ASN_LOOKUP
+ lookup_fields: [ server_ip ]
+ output_fields: [ server_asn ]
+ parameters:
+ option: IP_TO_ASN
+ kb_name: tsg_ip_asn
+```
+
+ #### Base64 Decode
+
+将 Base64 编码二进制数据解码转换为字符串。
+
+Parameters:
+
+- value_field =<String>
+- charset_field=<String> 可选,默认为UTF-8
+
+```yaml
+- function: BASE64_DECODE_TO_STRING
+ output_fields: [mail_subject]
+ parameters:
+ value_field: mail_subject
+ charset_field: mail_subject_charset
+```
+
+ #### Base64 Encode
+
+将 Base64 二进制数据编码转换为字符串。
+
+Parameters:
+
+- value_field =<String>
+
+```yaml
+- function: BASE64_ENCODE_TO_STRING
+ output_fields: [packet]
+ parameters:
+ value_field: packet
+```
+
+ #### Current Unix Timestamp
+
+获取系统当前时间戳。
+
+- Parameters
+ - precision=seconds | milliseconds
+
+```yaml
+- function: CURRENT_UNIX_TIMESTAMP
+ output_fields: [ processing_time ]
+ parameters:
+ precision: milliseconds
+
+```
+
+ #### Domain
+
+域名处理函数。
+
+Parameters:
+
+- option = `<string>`
+ - TOP_LEVEL_DOMAIN 顶级域名
+ - FIRST_SIGNIFICANT_SUBDOMAIN 获取二级有效域名
+ - FQDN 获取FQDN
+
+```yaml
+- function: DOMAIN
+ lookup_fields: [ http_host,ssl_sni,dtls_sni,quic_sni ]
+ output_fields: [ server_domain ]
+ parameters:
+ option: FIRST_SIGNIFICANT_SUBDOMAIN
+```
+
+#### Drop
+
+满足Filter表达式的日志增加删除标记,下游函数将不再执行,当前Projection Function 不再发送事件到下游。设置Event isDropped标记为true。
+
+- 日志格式数据(无嵌套),丢弃符合过滤条件的数据
+
+```shell
+- function: DROP
+ filter: event.c2s_byte_num <10
+```
+
+- 丢弃object_id为13167 数据
+
+```shell
+- function: DROP
+ filter: event.object_id == 13167
+
+# Input: {"object_id":13176,"item_id":83989295}
+```
+
+- metrics格式数据(多级嵌套),丢弃object_id为102且item_id大于等于2的数据,或object_id等于13176且item_id大于83989294的数据
+
+```shell
+- function: DROP
+ filter: (event.tags.object_id == 102 && event.tags.item_id >= 2) || (event.tags.object_id ==13176 && event.tags.item_id >= 83989294)
+
+# Input: {"tags":{"object_id":13176,"item_id":83989295},"fields":{"in_bytes":1765830,"out_bytes":27446,"bytes":1793276},"timestamp_ms":1714443502000}
+```
+
+#### Encrypt
+
+对敏感信息进行加密。支持引用动态规则,获取需要加密的字段,选择是否对当前字段进行加密
+
+Parameters:
+
+- identifier = `<string>` 加密算法唯一标识。支持:aes-128-gcm96, aes-256-gcm96, sm4-gcm96
+- default_val= `<string>` 加密失败输出该值,默认将输出原值
+
+```
+- function: ENCRYPT
+ lookup_fields: [ phone_number ]
+ output_fields: [ phone_number ]
+ parameters:
+ identifier: aes-128-gcm96
+```
+
+Note : 读取任务变量`projection.encrypt.schema.registry.uri`,返回加密字段,数据类型为Array。
+
+ #### Eval
+
+通过值表达式,获取符合条件的值,添加到字段中。同时可以选择保留或删除指定的字段。
+
+Parameters:
+
+- value_expression=`<string>` 基于表达式设置字段的值,可以是一个常量
+
+Example 1: 创建一个字段ingestion_time, 取自 recv_time值
+
+```
+- function: EVAL
+ output_fields: [ ingestion_time ]
+ parameters:
+ value_expression: 'recv_time'
+```
+
+Example 2: 创建一个字段internal_ip, 如果flags&8=8?client_ip : server_ip
+
+```
+- function: EVAL
+ output_fields: [ internal_ip ]
+ parameters:
+ value_expression: 'flags&8=8? client_ip : server_ip'
+```
+
+ #### Flatten
+
+扁平化嵌套结构使其成为顶级字段。新字段命名使用每层结构名称作为前缀,中间默认用句点“.”分隔。
+
+- Parameters
+ - prefix= `<string>` //为扁平化的字段名称指定前缀。默认为空。
+ - depth=<int> // 扁平化的嵌套级别的最大值. 设置为1,仅扁平化顶级结构。默认设置为5
+ - delimiter=<String> 组合父级与子级名称的分隔符。默认为"."。
+ - json_string_keys=Array[string] 标识哪些JsonString格式的数据需要扁平化。默认为空。
+
+Example 1: 对Metrics的fields,tags 嵌套结构进行扁平化,如果lookup_fields为空则对所有嵌套结构进行扁平化。
+
+```
+- function: FLATTEN
+ lookup_fields: [ tags, fields ]
+```
+
+Example 2: 会话日志字段encapsulation(JsonString格式)嵌套结构进行扁平化,并增加前缀tunnels,嵌套深度指定3,中间用下划线“."分隔
+
+```yaml
+- function: FLATTEN
+ lookup_fields: [ encapsulation ]
+ parameters:
+ prefix: tunnels
+ depth: 3
+ delimiter: .
+ json_string_keys: [ encapsulation]
+
+# Output: tunnels.encapsulation.ipv4.client_ip: 192.168.4.1
+```
+
+ #### From Unix Timestamp
+
+将时间戳转换为日期类型,返回UTC日期时间格式字符串,输入支持10位和13位时间戳。
+
+- Parameters
+ - precision=seconds // yyyy-MM-dd HH:mm:ss
+ - precision=milliseconds // yyyy-MM-dd HH:mm:ss:SSS
+
+```yaml
+- function: FROM_UNIX_TIMESTAMP
+ lookup_fields: [recv_time]
+ output_fields: [recv_time_string]
+ parameters:
+ precision: seconds
+```
+
+ #### Generate String Array
+
+创建字符串数组
+
+```yaml
+- function: GENERATE_STRING_ARRAY
+ lookup_fields: [ client_asn,server_asn ]
+ output_fields: [ asn_list ]
+```
+
+ #### GeoIP Lookup
+
+查找IP地理位置信息。
+
+- Parameters
+ - kb_name=`<string>` // 使用的知识库的名称 ,需要预先在全局配置中进行注册。
+ - option = `<string>`
+ - IP_TO_COUNTRY 所属国家或地区
+ - IP_TO_PROVINCE 所属省/州
+ - IP_TO_CITY 所属城市
+ - IP_TO_SUBDIVISION_ADDR 如上三级以下信息,包括区、街道等。
+ - IP_TO_DETAIL 所属详情,包括如上四级,中间用英文句点分隔
+ - IP_TO_LATLNG 所属经纬度,中间用英文逗号分隔
+ - IP_TO_PROVIDER 所属服务提供商(ISP)
+ - IP_TO_JSON 返回所属位置详情,格式为JSON
+ - IP_TO_OBJECT 返回所属位置详情,格式为Response Object
+ - geolocation_field_mappingobject_key : field_name
+
+```yaml
+- function: GEOIP_LOOKUP
+ lookup_fields: [ server_ip ]
+ output_fields: [ server_geolocation ]
+ parameters:
+ kb_name: tsg_ip_location
+ option: IP_TO_DETAIL
+```
+
+```yaml
+- function: GEOIP_LOOKUP
+ lookup_fields: [ server_ip ]
+ output_fields: [ server_geolocation ]
+ parameters:
+ kb_name: tsg_ip_location
+ option: IP_TO_OBJECT
+ geolocation_field_mapping:
+ COUNTRY: client_country_region
+ PROVINCE: client_super_admin_area
+
+# 当option为“IP_TO_OBJECT” 时,支持字段映射(geolocation_field_mapping):
+# - COUNTRY - 国家或地区
+# - PROVINCE - 省/州
+# - CITY - 城市
+# - LONGITUDE - 精度
+# - LATITUDE - 纬度
+# - ISP - 运营商
+# - ORGANIZATION - 组织
+```
+
+#### HMAC
+
+使用密钥和消息使用哈希算法生成一个固定长度的消息摘要。HMAC(Hash-based Message Authentication Code)是一种基于哈希函数的消息认证码,用于验证数据的完整性和真实性。
+
+Parameters:
+
+- secret_key = `<string>` 用于生成MAC的密钥。
+- algorithm= `<string>` 用于生成MAC的HASH算法。默认是`sha256`
+- output_format = `<string>` 输出MAC的格式。默认为`'hex'` 。支持:`base64` | `hex `。
+
+```
+- function: HMAC
+ lookup_fields: [ phone_number ]
+ output_fields: [ phone_number_hmac ]
+ parameters:
+ secret_key: ******
+ output_format: base64
+```
+
+
+
+ #### JSON Extract
+
+解析JSON字段,通过表达式抽取json部分内容。
+
+- Parameters
+ - value_expression=`<string>` //基于JsonPath表达式设置字段的值
+
+```
+JSON_EXTRACT(null, 'device_tag', 'data_center', parameters)
+- parameters:
+ - value_expression = $.tags[?(@.tag=='data_center')][0].value
+```
+
+ #### Path Combine
+
+路径合并。
+
+- Parameters
+ - path = Array[string]
+
+```yaml
+- function: PATH_COMBINE
+ lookup_fields: [ packet_capture_file ]
+ output_fields: [ packet_capture_file ]
+ parameters:
+ # 获取grootstream.yaml中properties配置的对应属性hos.path的值
+ path: [ props.hos.path, props.hos.bucket.name.traffic_file, packet_capture_file]
+
+# Output: hos_path + bucket_name + packet_capture_file
+```
+
+ #### Rename
+
+重命名字段。
+
+- Parameters
+
+ - parent_fields: Array[string] 指定哪些字段的子字段将进行重命名。如果为空,则仅会对顶级字段进行重命名,不支持对数组结构中的key进行重命名。
+ - rename_fields: 指定的字段进行重命名
+ - current_field_name : new_field_name
+ - rename_expression=`<string>` 对字段执行AviatorScript表达式,返回值作为重命名后的字段名,优先级低于rename_fields。
+
+
+
+Example 1: 移除字段名"tags_"前缀 , 重命名字段timestamp_ms为recv_time_ms
+
+```yaml
+- function: RENAME
+ parameters:
+ rename_fields:
+ timestamp_ms: recv_time_ms
+ rename_expression: key=string.replace_all(key,'tags_',''); return key;
+```
+
+Example 2: client_ip 重命名为source_ip, 包括隧道encapsulation.ipv4下的字段
+
+```yaml
+- function: RENAME
+ parameters:
+ parent_fields: [encapsulation.ipv4]
+ rename_fields:
+ client_ip: source_ip
+
+# Output: source_ip:192.168.4.1, encapsulation.ipv4.source_ip:192.168.12.12
+```
+
+#### Snowflake ID
+
+基于雪花算法生成唯一ID。
+
+Parameters:
+
+- data_center_id_num = <int> 数据中心id,用与保证生成雪花id的唯一性。
+
+````shell
+- function: SNOWFLAKE_ID
+ output_fields: [ log_id ]
+````
+
+ #### String Joiner
+
+字符串拼接,可以指定分隔符,前缀与后缀。
+
+```yaml
+- function: STRING_JOINER
+ lookup_fields: [client_ip, server_ip]
+ output_fields: [ip_string]
+ parameters:
+ delimiter: ','
+ prefix: '['
+ suffix: ']'
+
+ # Output:ip_string='[client_ip, server_ip]'
+
+```
+
+ #### Unix Timestamp Converter
+
+转换时间戳精度,返回其他精度时间戳
+
+- Parameters
+ - precision=seconds // 获取Unix时间戳并将其精确到秒级
+ - precision=milliseconds // 获取Unix时间戳并将其精确到毫秒级
+ - precision=minutes // 获取Unix时间戳将其精确到分钟级别,并以秒级格式输出
+ - interval = <int>//时长精度,单位取决于precision
+
+```yaml
+- function: UNIX_TIMESTAMP_CONVERTER
+ lookup_fields: [ __timestamp ]
+ output_fields: [ recv_time ]
+ parameters:
+ precision: seconds
+ interval: 300
+
+# __timestamp:内置参数,从数据source的摄入时间,以300秒为精度返回时间戳,若precision = minutes,则为以300分钟为精度输出。
+
+```
+
+ #### UUID
+
+使用UUIDv4标准,生成128位随机UUID。实现方式参考:https://github.com/cowtowncoder/java-uuid-generator
+
+```yaml
+- function: UUID
+ output_fields: [log_uuid]
+
+ # 3f0f8d7e-d89e-4b0a-9f2e-2eab5c99d062
+```
+
+ #### UUIDv5
+
+是一种基于 **命名空间和名称** 生成的 UUID。与 `UUIDv4` 主要依赖随机数不同,`UUIDv5` 使用 SHA-1 哈希算法将命名空间和名称组合后生成一个确定性的 UUID。这意味着对同一命名空间和相同名称的输入,`UUIDv5` 总是会生成相同的 UUID。
+
+- Parameters
+ - namespace = <Enum> 枚举值,命名空间是一个 UUID,它定义了名称所属的上下文。可指定如下命名空间:
+ - NAMESPACE_IP: 6ba7b890-9dad-11d1-80b4-00c04fd430c8
+ - NAMESPACE_DOMAIN: 6ba7b891-9dad-11d1-80b4-00c04fd430c8
+ - NAMESPACE_APP: 6ba7b892-9dad-11d1-80b4-00c04fd430c8
+ - NAMESPACE_SUBSCRIBER: 6ba7b893-9dad-11d1-80b4-00c04fd430c8
+
+```yaml
+- function: UUIDv5
+ lookup_fields: [ client_ip, server_ip ] # 基于 client_ip, server_ip的值组成UUIDv5 name 参数值与命名空间结合后,通过哈希生成唯一的 UUID。
+ output_fields: [ip_uuid]
+ parameters:
+ namespace: NAMESPACE_IP
+
+ # 2ed6657d-e927-568b-95e1-2665a8aea6a2
+```
+
+ #### UUIDv7
+
+通过时间戳和随机数生成唯一UUID,适合需要时间排序的场景,比如数据库索引和日志记录。
+
+```yaml
+- function: UUIDv7
+ output_fields: [log_uuid] # 生成基于时间戳和随机数的 UUID
+
+ # 2ed6657d-e927-568b-95e1-2665a8aea6a2
+```
+
+### 聚合函数
+
+ #### Collect List
+
+在时间窗口内将指定对象合并为List,不进行去重
+
+```yaml
+- function: COLLECT_LIST
+ lookup_fields: [client_ip]
+ output_fields: [client_ip_list]
+# Output:client_ip_list= ['192.168.4.1','192.168.4.1','192.168.4.2']
+```
+
+ #### Collect Set
+
+在时间窗口内将指定对象合并为Set,对结果进行去重。
+
+```yaml
+- function: COLLECT_SET
+ lookup_fields: [client_ip]
+ output_fields: [client_ip_set]
+
+# Output:client_ip_set= ['192.168.4.1','192.168.4.2']
+```
+
+ #### First Value
+
+返回时间窗口内第一个出现的不为空的value。
+
+```yaml
+- function: FIRST_VALUE
+ lookup_fields: [ received_bytes ]
+ output_fields: [ received_bytes_first ]
+```
+
+ #### Last Value
+
+返回时间窗口内最后一个出现的不为空的value。
+
+```yaml
+- function: LAST_VALUE
+ lookup_fields: [ received_bytes ]
+ output_fields: [ received_bytes_last ]
+```
+
+ #### Long Count
+
+在时间窗口内统计Event条数。
+
+```yaml
+- function: LONG_COUNT
+ lookup_fields: [ log_id ]
+ output_fields: [ sessions ]
+```
+
+ #### Mean
+
+在时间窗口内对指定的数值对象求平均值。
+
+Parameters
+
+- precision=<int> 返回的double类型结果精度,不配置则返回实际计算结果
+
+```yaml
+- function: MEAN
+ lookup_fields: [ received_bytes ]
+ output_fields: [ received_bytes_mean ]
+ parameters:
+ precision: 2
+```
+
+ #### Number Sum
+
+在时间窗口内对指定数字类型字段进行求和:支持 int,long,double,float类型。
+
+```yaml
+- function: NUMBER_SUM
+ lookup_fields: [received_bytes, sent_bytes]
+ output_fields: [received_bytes_sum]
+```
+
+```yaml
+- function: NUMBER_SUM
+ lookup_fields: [sent_bytes]
+
+```
+
+ #### HLLD
+
+构建HLLD Sketch,输入列可以为常规类型列或HLLD Sketch列。
+
+Parameters:
+
+- input_type(string):输入列类型。可选值:regular(输入列为常规类型列,代表单个元素), sketch(输入列为sketch类型列,sketch类型)。默认值:sketch。
+- precision(int):HLL精度。默认值:12。
+- output_format(string):输出类型格式。可选值:base64(base64字符串), binary(byte[])。默认值:base64。
+
+```yaml
+- function: HLLD
+ lookup_fields: [ ip_hlld ]
+ output_fields: [ ip_hlld ]
+ parameters:
+ input_type: sketch
+
+- function: HLLD
+ lookup_fields: [ ip ]
+ output_fields: [ ip_hlld ]
+ parameters:
+ input_type: regular
+```
+
+ #### APPROX_COUNT_DISTINCT_HLLD
+
+计算近似distinct count,输入列可以为常规类型列或HLLD Sketch列。
+
+Parameters:
+
+- input_type(string):输入列类型。可选值:regular(输入列为常规类型列,代表单个元素), sketch(输入列为sketch类型列,sketch类型)。默认值:sketch。
+- precision(int):HLL精度。默认值:12。
+
+```yaml
+- function: APPROX_COUNT_DISTINCT_HLLD
+ lookup_fields: [ ip_hlld ]
+ output_fields: [ ip_count ]
+ parameters:
+ input_type: sketch
+
+- function: APPROX_COUNT_DISTINCT_HLLD
+ lookup_fields: [ ip ]
+ output_fields: [ ip_count ]
+ parameters:
+ input_type: regular
+```
+
+ #### HDR_HISTOGRAM
+
+构建HdrHistogram Sketch,输入列可以为常规类型列或HdrHistogram Sketch列。
+
+Parameters:
+
+Parameters:
+
+- input_type(string):输入列类型。可选值:regular(输入列为常规类型列,代表单个元素), sketch(输入列为sketch类型列,sketch类型)。默认值:sketch。
+- lowestDiscernibleValue(int):除0外最小值,默认1
+- highestTrackableValue(int):直方图可以记录的最大值,默认2
+- numberOfSignificantValueDigits(int):指定数据值的精度,默认1;[1-5] 较大的值更精确,但会需要更多内存。
+- autoResize(boolean):自动调整highestTrackableValue,默认true
+- output_format(string):输出类型格式。可选值:base64(base64字符串), binary(byte[])。默认值:base64。
+
+```yaml
+ - function: HDR_HISTOGRAM
+ lookup_fields: [latency_ms_histogram]
+ output_fields: [latency_ms_histogram]
+ parameters:
+ input_type: sketch
+
+ - function: HDR_HISTOGRAM
+ lookup_fields: [latency_ms]
+ output_fields: [latency_ms_histogram]
+ parameters:
+ input_type: regular
+```
+
+ #### APPROX_QUANTILE_HDR
+
+计算近似分位数,输入列可以为常规类型列或HdrHistogram Sketch列。
+
+Parameters:
+
+- input_type(string):输入列类型。可选值:regular(输入列为常规类型列,代表单个元素), sketch(输入列为sketch类型列,sketch类型)。默认值:sketch。
+- lowestDiscernibleValue(int):除0外最小值,默认1
+- highestTrackableValue(int):直方图可以记录的最大值,默认2
+- numberOfSignificantValueDigits(int):指定数据值的精度,默认1;[1-5] 较大的值更精确,但会需要更多内存。
+- autoResize(boolean):自动调整highestTrackableValue,默认true
+- probability(double):分位数百分比,范围0-1,默认0.5
+
+```yaml
+
+ - function: APPROX_QUANTILE_HDR
+ lookup_fields: [latency_ms]
+ output_fields: [latency_ms_p95]
+ parameters:
+ input_type: regular
+ probability: 0.95
+
+
+ - function: APPROX_QUANTILE_HDR
+ lookup_fields: [latency_ms_histogram]
+ output_fields: [latency_ms_p95]
+ parameters:
+ input_type: sketch
+ probability: 0.95
+```
+
+ #### APPROX_QUANTILES_HDR
+
+计算近似分位数,输入列可以为常规类型列或HdrHistogram Sketch列。
+
+Parameters:
+
+- input_type(string):输入列类型。可选值:regular(输入列为常规类型列,代表单个元素), sketch(输入列为sketch类型列,sketch类型)。默认值:sketch。
+- lowestDiscernibleValue(int):除0外最小值,默认1
+- highestTrackableValue(int):直方图可以记录的最大值,默认2
+- numberOfSignificantValueDigits(int):指定数据值的精度,默认1;[1-5] 较大的值更精确,但会需要更多内存。
+- autoResize(boolean):自动调整highestTrackableValue,默认true
+- probabilities(double[]):分位数百分比数组,范围0-1,默认null,必须的属性。
+
+```yaml
+- function: APPROX_QUANTILES_HDR
+ lookup_fields: [latency_ms_HDR]
+ output_fields: [latency_ms_quantiles]
+ parameters:
+ input_type: sketch
+ probabilities: [0.5, 0.95, 0.99]
+
+
+- function: APPROX_QUANTILES_HDR
+ lookup_fields: [latency_ms]
+ output_fields: [latency_ms_quantiles]
+ parameters:
+ input_type: regular
+ probabilities: [0.5, 0.95, 0.99]
+
+```
+
+### 表格函数
+
+ #### Unroll
+
+展开函数用于处理一个数组类型字段 ,或配置一个用于分割字符串类型字段的表达式 , 并将该字段展开为单独的事件。支持处理 array或string类型字段。
+
+Parameters:
+
+- regex= string//用于将字符串分割为数组的正则表达式,如“,”按照逗号分割字符串,如果字段为数组类型则无需配置
+
+```yaml
+functions:
+ - function: UNROLL
+ lookup_fields: [ monitor_rule_list ]
+ output_fields: [ monitor_rule ]
+
+ # Input: Event { client_ip=‘192.168.1.1’,monitor_rule_list=[954779,930791]}
+ # Output:
+ #Event1: {client_ip=‘192.168.1.1’,monitor_rule=954779}
+ #Event2: {client_ip=‘192.168.1.1’,monitor_rule=930791}
+```
+
+ #### Json Unroll
+
+JSON 展开函数接收 JSON 对象字符串字段,将其中的对象数组展开为字符串类型单独事件,同时继承顶级字段。
+
+Parameters:
+
+- path= string//要展开的数组的路径,基于JsonPath表达式,不配置默认展开顶层数组
+- new_path= string//新元素的路径,基于JsonPath表达式,不配置默认覆盖原path
+
+```yaml
+- function: JSON_UNROLL
+ lookup_fields: [ encapsulation]
+ output_fields: [ encapsulation ]
+ parameters:
+ path: tags
+ new_path: new_tag
+# Input: Event { client_ip=‘192.168.1.1’,device_tag=‘{"tags":[{"tag":"data_center","value":"center-xxg-tsgx-1"}, {"tag":"device_group","value":"group-xxg-tsgx-2"}]}’}
+# Output:
+ #Event1:{client_ip=‘192.168.1.1’,device_tag='{"new_tag":{"tag":"data_center","value":"center-xxg-tsgx-1"}’}'
+ #Event2:{client_ip=‘192.168.1.1’,device_tag='{"new_tag":{"tag":"data_center","value":"center-xxg-tsgx-2"}’}'
+```
+
+```yaml
+- function: JSON_UNROLL
+ lookup_fields: [ encapsulation]
+ output_fields: [ encapsulation ]
+
+#Input: Event { client_ip=‘192.168.1.1’,encapsulation=‘[{"tunnels_schema_type":"GRE"},{"tunnels_schema_type":"IPv4","client_ip":"12.1.1.1","server_ip":"14.1.1.1"}]’}
+#Output:
+ #Event1:{client_ip=‘192.168.1.1’,encapsulation='{"tunnels_schema_type":"GRE"}'}
+ #Event2:{client_ip=‘192.168.1.1’,encapsulation='{"tunnels_schema_type":"IPv4","client_ip":"12.1.1.1","server_ip":"14.1.1.1"}'}
+```
+
+ #### Path Unroll
+
+ 将文件路径逐层展开,逐层输出路径和文件(可选)。
+
+Parameters:
+
+- separator= 路径分隔符(只能是单个字符),默认'/'。
+
+```yaml
+# 将一个应用层协议按层级进行拆分,应用层协议由协议解析路径和应用组成。
+- function: PATH_UNROLL
+ lookup_fields: [ decoded_path, app]
+ output_fields: [ protocol_stack_id, app_name ]
+ parameters:
+ separator: "."
+
+# Input: {"decoded_path":"ETHERNET.IPv4.TCP.ssl","app":"port_443"}
+# Output:
+ #Event1: {"protocol_stack_id":"ETHERNET"}
+ #Event2: {"protocol_stack_id":"ETHERNET.IPv4"}
+ #Event3: {"protocol_stack_id":"ETHERNET.IPv4.TCP"}
+ #Event4: {"protocol_stack_id":"ETHERNET.IPv4.TCP.ssl"}
+ #Event5: {"app_name":"port_443","protocol_stack_id":"ETHERNET.IPv4.TCP.ssl.port_443"}
+
+# Input: {"decoded_path":"ETHERNET.IPv4.TCP.ssl","app":"ssl"}
+# Output:
+ #Event1: {"protocol_stack_id":"ETHERNET"}
+ #Event2: {"protocol_stack_id":"ETHERNET.IPv4"}
+ #Event3: {"protocol_stack_id":"ETHERNET.IPv4.TCP"}
+ #Event4: {"app_name":"ssl","protocol_stack_id":"ETHERNET.IPv4.TCP.ssl"}
+
+# Input: {"decoded_path":"ETHERNET.IPv4.TCP.ssl","app":"ssl.port_444"}
+# Output:
+ #Event1: {"protocol_stack_id":"ETHERNET"}
+ #Event2: {"protocol_stack_id":"ETHERNET.IPv4"}
+ #Event3: {"protocol_stack_id":"ETHERNET.IPv4.TCP"}
+ #Event4: {"protocol_stack_id":"ETHERNET.IPv4.TCP.ssl"}
+ #Event5: {"app_name":"ssl.port_444","protocol_stack_id":"ETHERNET.IPv4.TCP.ssl.ssl.port_444"}
+
+#只有路径参数的场景(或者上例中文件字段值为null).
+- function: PATH_UNROLL
+ lookup_fields: [ decoded_path]
+ output_fields: [ protocol_stack_id]
+ parameters:
+ separator: "."
+
+# Input: {"decoded_path":"ETHERNET.IPv4.TCP.ssl"}
+# Output:
+ #Event1: {"protocol_stack_id":"ETHERNET"}
+ #Event2: {"protocol_stack_id":"ETHERNET.IPv4"}
+ #Event3: {"protocol_stack_id":"ETHERNET.IPv4.TCP"}
+ #Event4: {"protocol_stack_id":"ETHERNET.IPv4.TCP.ssl"}
+```
+
+## CN扩展UDF
+
+[CN函数库](https://docs.geedge.net/pages/viewpage.action?pageId=129087866)
+
+用户自定义插件(IN Progress)
+
+| 名称 | 描述 | 类型 | 必填 | 约束 |
+|----------------------|---------|---------------|----|---------|
+| user_define_process | | | | |
+| function_name | 调用的方法名称 | String | Y | |
+| class_name | 主类名 | String | Y | |
+| jar_name | jar包名 | String | Y | |
+| config_field | 私有属性配置 | JSON[Array] | N | 可自定义多属性 |
+| input_schema | 输入字段 | json_String | Y | |
+| output_schema | 输出字段 | json_String | Y | |
+| | | | | |
+| user_define_function | | | | |
+| name | 名称 | String | Y | |
+| description | 描述 | String | N | |
+| type | 类型 | String | Y | udfudaf |
+| class_name | 主类名 | String | Y | |
+| jar_name | jar包名 | String | Y | |
+
+# 实现原则
+
+- 包命名: com.geedgenetworks. [模块名].XXX
+- 统一依赖管理:第三方类库的依赖声明在项目的顶层 POM 文件(也称为 Project POM)中,各个子模块继承这些依赖,确保整个项目共享相同的依赖。
+- 模块之间依赖:在每个模块的 POM 文件中定义依赖其他模块的关系。
+- 每个模块按其职责命名 groot-[功能名称],例如:
+ - groot-common 公共模块,包含可复用功能、工具类或库,供其它模块引用。
+ - groot-core 核心模块,包含与业务逻辑紧密相关的核心功能、类、接口或服务。
+ - groot-bootstrap 启动模块,包含一些必要的初始化代码、配置解析或资源加载等,它属于应用程序起点,负责将一个流处理任务各个部分组装起来,使其正确运行。
+ - groot-connectors 连接器模块
+ - connecor-kafka 子模块,包含Source和Sink 功能
+ - connector-ipfix-collector 子模块,Source 功能
+ - connecotr-clickhouse 子模块,Sink 功能
+ - MockDataConnector(Source) 用于产生样例数据,用于测试、开发或演示目的的场景
+ - groot-formats format模块
+ - format-json 子模块,提供json format
+ - groot-tests 测试模块,包含多个模块,用于任务的集成和功能测试 (非单元测试)
+- 对于不受检查异常(RuntimeException)在groot-common模块定义全局的异常处理类GrootRuntimeException,基于该自定义的异常抛出并附带更清晰的错误信息,用于限定问题的范围。其他各个模块按需实现Exception用于增加更多上下文异常提示信息。
+- 自定义插件管理:Connectors(Source 和 Destination) 和 UDF 函数;
+ - UDF(用户自定义函数)—— 用于数据清洗、处理和格式转换。按实现方式可分为内置UDFs和用户扩展UDFs。
+ - UDF接口包括Function Name、传递数据(Event)、配置参数(context) 及执行方法 Evaluate(Event)
+ - 通过配置文件(udfs)管理平台已注册的函数列表
+ - 任务启动时包含两个步骤:验证所引用的函数是否在注册列表中;按照引用的顺序对函数进行实例化。
+ - 与通用工具类的关系:UDF 调用通用工具类的方法,以实现Evaluate的功能。
+ - 提供open 和 close 方法,用以对象初始化,处理连接器(如数据库连接、文件句柄等)相关的资源的打开和关闭。而open方法一次性初始化的方法,在 UDF 对象创建时执行,用于初始化对象级别的资源和状态。
+- Event 内置字段(Internal Fields) 命名以双下划线开头,仅用于数据流处理,不发送到SINK 模块。
+ - __timestamp : 数据摄入时间(Ingestion Time)。当Source无法抽取时,使用当前时间(Unix epoch格式),一般用于标识“数据的摄入时间”。例如 Kafka Source 抽取头部_time属性。
+ - __inputId: 数据来源,事件的产生源头或来源的标识符或名称。用于事件追踪和管理,以确定事件是由哪个系统、应用程序、设备或实体产生的。例如Kafka Source 记录topic 名称。
+
+# 相关问题
+
+- 知识库更新为什么不基于Flink 广播流?
+ - 广播流适用于将配置或规则低吞吐事件流广播到下游所有Task中,不适用广播知识库大文件配置(GB级别)。
+ - 采用广播流动态广播知识库元数据方式,若更新知识库,当基于每个Task(线程)分别存储,占用内存较大;如果基于进程级(静态方法/变量)共享,可能会发生线程阻塞或死锁问题。
+- 自定义函数如何提交到平台?
+- Pack 在平台里定位是什么? 如何扩展?
+- 数据分流方案?
+ - 使用Flink 侧输出流(side_output),对事件标记tag实现。
+- Aggregate Processor 函数如何定义?怎么指定dimension、Metrics ?
+ - 支持基础滑动,滚动窗口聚合计算。Dimension 基于group_by_fields 指定,Metrics 通过自定义UDAF实现。 \ No newline at end of file
diff --git a/docs/images/groot_stream_architecture.jpg b/docs/images/groot_stream_architecture.jpg
index d8f1d4b..28b553b 100644
--- a/docs/images/groot_stream_architecture.jpg
+++ b/docs/images/groot_stream_architecture.jpg
Binary files differ
diff --git a/docs/processor/aggregate-processor.md b/docs/processor/aggregate-processor.md
index 5ab0ae0..afc26f6 100644
--- a/docs/processor/aggregate-processor.md
+++ b/docs/processor/aggregate-processor.md
@@ -10,17 +10,18 @@ Within the pipeline, events are processed by each Function in order, top‑>down
## Options
Note:Default will output internal fields `__window_start_timestamp` and `__window_end_timestamp` if not set output_fields.
-| name | type | required | default value |
-|--------------------------|--------|----------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| type | String | Yes | The type of the processor, now only support `com.geedgenetworks.core.processor.aggregate.AggregateProcessor` |
-| output_fields | Array | No | Array of String. The list of fields that need to be kept. Fields not in the list will be removed. |
-| remove_fields | Array | No | Array of String. The list of fields that need to be removed. |
-| group_by_fields | Array | yes | Array of String. The list of fields that need to be grouped. |
-| window_type | String | yes | The type of window, now only support `tumbling_processing_time`, `tumbling_event_time`, `sliding_processing_time`, `sliding_event_time`. if window_type is `tumbling/sliding_event_time,` you need to set watermark. |
-| window_size | Long | yes | The duration of the window in seconds. |
-| window_slide | Long | yes | The duration of the window slide in seconds. |
-| window_timestamp_field | String | No | Set the output timestamp field name, with the unit in seconds. It is mapped to the internal field __window_start_timestamp. |
-| functions | Array | No | Array of Object. The list of functions that need to be applied to the data. |
+| name | type | required | default value |
+|------------------------|-----------|----------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| type | String | Yes | The type of the processor, now only support `com.geedgenetworks.core.processor.aggregate.AggregateProcessor` |
+| output_fields | Array | No | Array of String. The list of fields that need to be kept. Fields not in the list will be removed. |
+| remove_fields | Array | No | Array of String. The list of fields that need to be removed. |
+| group_by_fields | Array | yes | Array of String. The list of fields that need to be grouped. |
+| window_type | String | yes | The type of window, now only support `tumbling_processing_time`, `tumbling_event_time`, `sliding_processing_time`, `sliding_event_time`. if window_type is `tumbling/sliding_event_time,` you need to set watermark. |
+| window_size | Long | yes | The duration of the window in seconds. |
+| window_slide | Long | yes | The duration of the window slide in seconds. |
+| window_timestamp_field | String | No | Set the output timestamp field name, with the unit in seconds. It is mapped to the internal field __window_start_timestamp. |
+| mini_batch | Boolean | No | Specifies whether to enable local aggregate optimization. The default value is false. This can significantly reduce the state overhead and get a better throughput. |
+| functions | Array | No | Array of Object. The list of functions that need to be applied to the data. |
## Usage Example
diff --git a/docs/processor/split-processor.md b/docs/processor/split-processor.md
new file mode 100644
index 0000000..e1a1163
--- /dev/null
+++ b/docs/processor/split-processor.md
@@ -0,0 +1,49 @@
+# Split Processor
+
+> Split the output of a data processing pipeline into multiple streams based on certain conditions.
+
+## Description
+
+Using the flink side Outputs send data from a stream to multiple downstream consumers. This is useful when you want to separate or filter certain elements of a stream without disrupting the main processing flow. For example, side outputs can be used for error handling, conditional routing, or extracting specific subsets of the data.
+
+## Options
+
+| name | type | required | default value |
+|-------------------|--------|----------|--------------------------------------------------------------------------------------------|
+| type | String | Yes | The type of the processor, now only support ` com.geedgenetworks.core.split.SplitOperator` |
+| rules | Array | Yes | Array of Object. Defining rules for labeling Side Output Tag |
+| [rule.]tag | String | Yes | The tag name of the side output |
+| [rule.]expression | String | Yes | The expression to evaluate the event. |
+
+## Usage Example
+
+This example uses a split processor to split the data into two streams based on the value of the `decoded_as` field.
+
+```yaml
+splits:
+ decoded_as_split:
+ type: split
+ rules:
+ - tag: http_tag
+ expression: event.decoded_as == 'HTTP'
+ - tag: dns_tag
+ expression: event.decoded_as == 'DNS'
+
+
+topology:
+ - name: inline_source
+ downstream: [decoded_as_split]
+ - name: decoded_as_split
+ tags: [http_tag, dns_tag]
+ downstream: [ projection_processor, aggregate_processor]
+ - name: projection_processor
+ downstream: [ print_sink ]
+ - name: aggregate_processor
+ downstream: [ print_sink ]
+ - name: print_sink
+ downstream: []
+```
+
+
+
+
diff --git a/docs/processor/udaf.md b/docs/processor/udaf.md
index dd1dd70..66d6ad5 100644
--- a/docs/processor/udaf.md
+++ b/docs/processor/udaf.md
@@ -41,7 +41,7 @@ COLLECT_LIST is used to collect the value of the field in the group of events.
- lookup_fields: required. Now only support one field.
- output_fields: optional. If not set, the output field name is `lookup_field_name`.
-### Example
+Example:
```yaml
- function: COLLECT_LIST
@@ -59,7 +59,7 @@ COLLECT_SET is used to collect the unique value of the field in the group of eve
- lookup_fields: required. Now only support one field.
- output_fields: optional. If not set, the output field name is `lookup_field_name`.
-### Example
+Example
```yaml
- function: COLLECT_SET
@@ -76,7 +76,7 @@ FIRST_VALUE is used to get the first value of the field in the group of events.
- lookup_fields: required. Now only support one field.
- output_fields: optional. If not set, the output field name is `lookup_field_name`.
-### Example
+Example
```yaml
- function: FIRST_VALUE
@@ -92,7 +92,7 @@ LAST_VALUE is used to get the last value of the field in the group of events.
- lookup_fields: required. Now only support one field.
- output_fields: optional. If not set, the output field name is `lookup_field_name`.
-### Example
+Example
```yaml
- function: LAST_VALUE
@@ -109,7 +109,7 @@ LONG_COUNT is used to count the number of events in the group of events.
- lookup_fields: optional.
- output_fields: required.
-### Example
+Example
```yaml
- function: LONG_COUNT
@@ -127,7 +127,7 @@ MEAN is used to calculate the mean value of the field in the group of events. Th
- parameters: optional.
- precision: `<Integer>` required. The precision of the mean value. Default is 2.
-### Example
+Example
```yaml
- function: MEAN
@@ -144,7 +144,7 @@ NUMBER_SUM is used to sum the value of the field in the group of events. The loo
- lookup_fields: required. Now only support one field.
- output_fields: optional. If not set, the output field name is `lookup_field_name`.
-### Example
+Example
```yaml
- function: NUMBER_SUM
@@ -164,7 +164,8 @@ hlld is a high-performance C server which is used to expose HyperLogLog sets and
- precision: `<Integer>` optional. The precision of the hlld value. Default is 12.
- output_format: `<String>` optional. The output format can be either `base64(encoded string)` or `binary(byte[])`. The default is `base64`.
-### Example
+Example
+
Merge multiple string field into a HyperLogLog data structure.
```yaml
- function: HLLD
@@ -194,8 +195,8 @@ Approx Count Distinct HLLD is used to count the approximate number of distinct v
- input_type: `<String>` optional. Refer to `HLLD` function.
- precision: `<Integer>` optional. Refer to `HLLD` function.
-### Example
-
+Example
+
```yaml
- function: APPROX_COUNT_DISTINCT_HLLD
lookup_fields: [client_ip]
@@ -228,8 +229,8 @@ A High Dynamic Range (HDR) Histogram. More details can be found in [HDR Histogra
- autoResize: `<Boolean>` optional. If true, the highestTrackableValue will auto-resize. Default is true.
- output_format: `<String>` optional. The output format can be either `base64(encoded string)` or `binary(byte[])`. The default is `base64`.
-### Example
-
+Example
+
```yaml
- function: HDR_HISTOGRAM
lookup_fields: [latency_ms]
@@ -264,8 +265,8 @@ Approx Quantile HDR is used to calculate the approximate quantile value of the f
- autoResize: `<Boolean>` optional. Refer to `HDR_HISTOGRAM` function.
- probability: `<Double>` optional. The probability of the quantile. Default is 0.5.
-### Example
-
+Example
+
```yaml
- function: APPROX_QUANTILE_HDR
lookup_fields: [latency_ms]
@@ -301,8 +302,8 @@ Approx Quantiles HDR is used to calculate the approximate quantile values of the
- autoResize: `<Boolean>` optional. Refer to `HDR_HISTOGRAM` function.
- probabilities: `<Array<Double>>` required. The list of probabilities of the quantiles. Range is 0 to 1.
-### Example
-
+Example
+
```yaml
- function: APPROX_QUANTILES_HDR
lookup_fields: [latency_ms]
diff --git a/docs/processor/udf.md b/docs/processor/udf.md
index 170d86f..e480275 100644
--- a/docs/processor/udf.md
+++ b/docs/processor/udf.md
@@ -96,18 +96,19 @@ Base64 encode function is commonly used to encode the binary data to base64 stri
```BASE64_ENCODE_TO_STRING(filter, output_fields[, parameters])```
- filter: optional
-- lookup_fields: not required
+- lookup_fields: required
- output_fields: required
- parameters: required
- - value_field: `<String>` required.
+ - input_type: `<String>` required. Enum: `string`, `byte_array`. The input type of the value field.
Example:
```yaml
- function: BASE64_ENCODE_TO_STRING
+ lookup_fields: [packet]
output_fields: [packet]
parameters:
- value_field: packet
+ input_type: string
```
### Current Unix Timestamp
@@ -141,7 +142,7 @@ Domain function is used to extract the domain from the url.
- parameters: required
- option: `<String>` required. Enum: `TOP_LEVEL_DOMAIN`, `FIRST_SIGNIFICANT_SUBDOMAIN`.
-#### Option
+**Option**
- `TOP_LEVEL_DOMAIN` is used to extract the top level domain from the url. For example, `www.abc.com` will be extracted to `com`.
- `FIRST_SIGNIFICANT_SUBDOMAIN` is used to extract the first significant subdomain from the url. For example, `www.abc.com` will be extracted to `abc.com`.
@@ -184,34 +185,55 @@ Eval function is used to adds or removes fields from events by evaluating an val
- parameters: required
- value_expression: `<String>` required. Enter a value expression to set the field’s value – this can be a constant.
-Example 1:
-Add a field `ingestion_time` with value `recv_time`:
+Example 1, add a field `eval_constant_string` with string value `fixed_value`:
+```yaml
+
+- function: EVAL
+ output_fields: [eval_constant_string]
+ parameters:
+ value_expression: "'fixed_value'"
+```
+
+Example 2, add a field `eval_constant_integer` with integer value `123`:
+```yaml
+- function: EVAL
+ output_fields: [eval_constant_integer]
+ parameters:
+ value_expression: "123"
+```
+Example 3: add a field `ingestion_time` with the value of `recv_time` field.
```yaml
- function: EVAL
output_fields: [ingestion_time]
parameters:
- value_expression: recv_time
+ value_expression: recv_time # or "recv_time"
```
-Example 2:
+Example 4: add a field `internal_ip` with the expression of conditional operator.
If the value of `direction` is `69`, the value of `internal_ip` will be `client_ip`, otherwise the value of `internal_ip` will be `server_ip`.
-
```yaml
- function: EVAL
output_fields: [internal_ip]
parameters:
- value_expression: 'direction=69 ? client_ip : server_ip'
+ value_expression: "direction=69 ? client_ip : server_ip"
+```
+Use the bitwise operator to determine the value of the `direction` field.
+```yaml
+ - function: EVAL
+ output_fields: [ direction ]
+ parameters:
+ value_expression: "(flags & 24576) == 24576 ? 'double' : ((flags & 8192) == 8192 ? 'c2s' : ((flags & 16384) == 16384 ? 's2c' : 'unknown'))"
```
-
### Flatten
-Flatten the fields of nested structure to the top level. The new fields name are named using the field name prefixed with the names of the struct fields to reach it, separated by dots as default.
+Flatten the fields of nested structure to the top level. The new fields name are named using the field name prefixed with the names of the struct fields to reach it, separated by dots as default. The original fields will be removed.
```FLATTEN(filter, lookup_fields, output_fields[, parameters])```
+
- filter: optional
- lookup_fields: optional
-- output_fields: not required
+- output_fields: not required.
- parameters: optional
- prefix: `<String>` optional. Prefix string for flattened field names. Default is empty.
- depth: `<Integer>` optional. Number representing the nested levels to consider for flattening. Minimum 1. Default is `5`.
@@ -255,13 +277,14 @@ Output:
From unix timestamp function is used to convert the unix timestamp to date time string. The default time zone is UTC+0.
```FROM_UNIX_TIMESTAMP(filter, lookup_fields, output_fields[, parameters])```
+
- filter: optional
- lookup_fields: required
- output_fields: required
- parameters: optional
- precision: `<String>` optional. Default is `seconds`. Enum: `milliseconds`, `seconds`.
-#### Precision
+**Precision**
- `milliseconds` is used to convert the unix timestamp to milliseconds date time string. For example, `1619712000` will be converted to `2021-04-30 00:00:00.000`.
- `seconds` is used to convert the unix timestamp to seconds date time string. For example, `1619712000` will be converted to `2021-04-30 00:00:00`.
@@ -314,7 +337,7 @@ GeoIP lookup function is used to lookup the geoip information by ip address. You
- ISP: `<String>` optional.
- ORGANIZATION: `<String>` optional.
-#### Option
+**Option**
- `IP_TO_COUNTRY` is used to lookup the country or region information by ip address.
- `IP_TO_PROVINCE` is used to lookup the province or state information by ip address.
@@ -326,7 +349,7 @@ GeoIP lookup function is used to lookup the geoip information by ip address. You
- `IP_TO_JSON` is used to lookup the above information by ip address. The result is a json string.
- `IP_TO_OBJECT` is used to lookup the above information by ip address. The result is a `LocationResponse` object.
-#### GeoLocation Field Mapping
+**GeoLocation Field Mapping**
- `COUNTRY` is used to map the country information to the event field.
- `PROVINCE` is used to map the province information to the event field.
@@ -413,8 +436,8 @@ Rename function is used to rename or reformat(e.g. by replacing character unders
- parameters: required
- parent_fields: `<Array>` optional. Specify fields whose children will inherit the Rename fields and Rename expression operations.
- rename_fields: `Map<String, String>` required. The key is the original field name, and the value is the new field name.
- - current_field_name: `<String>` required. The original field name.
- - new_field_name: `<String>` required. The new field name.
+ - current_field_name: `<String>` required. The original field name.
+ - new_field_name: `<String>` required. The new field name.
- rename_expression: `<String>` optional. AviatorScript expression whose returned value will be used to rename fields.
```
@@ -427,9 +450,9 @@ Remove the prefix "tags_" from the field names and rename the field "timestamp_m
```yaml
- function: RENAME
-- parameters:
+ parameters:
rename_fields:
- - timestamp_ms: recv_time_ms
+ timestamp_ms: recv_time_ms
rename_expression: key=string.replace_all(key,'tags_',''); return key;
```
@@ -440,10 +463,10 @@ Rename the field `client_ip` to `source_ip`, including the fields under the `enc
```yaml
- function: RENAME
-- parameters:
+ parameters:
parent_fields: [encapsulation.ipv4]
rename_fields:
- - client_ip: source_ip
+ client_ip: source_ip
```
@@ -509,7 +532,7 @@ Unix timestamp converter function is used to convert the unix timestamp precisio
- parameters: required
- precision: `<String>` required. Enum: `milliseconds`, `seconds`, `minutes`. The minutes precision is used to generate Unix timestamp, round it to the minute level, and output it in seconds format.
- Example:
-_`__timestamp` Internal field, from source ingestion time or current unix timestamp.
+ `__timestamp` Internal field, from source ingestion time or current unix timestamp.
```yaml
- function: UNIX_TIMESTAMP_CONVERTER
@@ -518,4 +541,67 @@ _`__timestamp` Internal field, from source ingestion time or current unix timest
parameters:
precision: seconds
```
+### UUID
+Generate a version 4 (random) UUID in accordance with [RFC-9562](https://datatracker.ietf.org/doc/rfc9562/).
+
+```UUID(output_fields)```
+- filter: not required
+- lookup_fields: not required
+- output_fields: required
+- parameters: not required
+
+Example:
+
+```yaml
+- function: UUID
+ output_fields: [uuid]
+```
+Result: such as 3f0f8d7e-d89e-4b0a-9f2e-2eab5c99d062.
+
+### UUIDv5
+
+Generate a version 5 (namespaced) UUID in accordance with RFC-9562 for the given name and namespace. If namespace is not a valid UUID, this function will fail.
+Suitable for consistent identifiers across different systems. One of IP, DOMAIN, APP, or SUBSCRIBER to use a predefined namespace.
+- NAMESPACE_IP: `6ba7b890-9dad-11d1-80b4-00c04fd430c8`
+- NAMESPACE_DOMAIN: `6ba7b891-9dad-11d1-80b4-00c04fd430c8`
+- NAMESPACE_APP: `6ba7b892-9dad-11d1-80b4-00c04fd430c8`
+- NAMESPACE_SUBSCRIBER: `6ba7b893-9dad-11d1-80b4-00c04fd430c8`
+
+```UUIDV5(lookup_fields, output_fields[, parameters])```
+- filter: not required
+- lookup_fields: required
+- output_fields: required
+- parameters: required
+ - namespace: `<String>` required. The UUID namespace.
+
+Example:
+
+```yaml
+- function: UUIDv5
+ lookup_fields: [ client_ip, server_ip ] # Based on the client_ip and server_ip value as Name with separator "_".
+ output_fields: [ip_uuid]
+ parameters:
+ namespace: NAMESPACE_IP
+```
+
+Result: such as 2ed6657d-e927-568b-95e1-2665a8aea6a2.
+
+### UUIDv7
+
+Generate a version 7 (Unix-timestamp + random based variant) UUID in accordance with RFC-9562. Suitable for scenarios that require time ordering, such as database indexing and logging.
+
+```UUIDV7(output_fields)```
+- filter: not required
+- lookup_fields: not required
+- output_fields: required
+- parameters: not required
+
+Example:
+
+```yaml
+- function: UUIDv7
+ output_fields: [log_uuid]
+
+```
+Result: such as 2ed6657d-e927-568b-95e1-2665a8aea6a2. \ No newline at end of file
diff --git a/docs/processor/udtf.md b/docs/processor/udtf.md
index a6e8444..65a7840 100644
--- a/docs/processor/udtf.md
+++ b/docs/processor/udtf.md
@@ -29,8 +29,8 @@ The Unroll Function handles an array field—or an expression evaluating to an a
- parameters: optional
- regex: `<String>` optional. If lookup_fields is a string, the regex parameter is used to split the string into an array. The default value is a comma.
-#### Example
-
+Example
+
```yaml
functions:
- function: UNROLL
@@ -50,8 +50,8 @@ The JSON Unroll Function handles a JSON object, unrolls/explodes an array of obj
- path: `<String>` optional. Path to array to unroll, default is the root of the JSON object.
- new_path: `<String>` optional. Rename path to new_path, default is the same as path.
-#### Example
-
+Example
+
```yaml
functions:
- function: JSON_UNROLL
@@ -62,5 +62,53 @@ functions:
- new_path: tag
```
+### Path Unroll
+
+The PATH_UNROLL function processes a given file path, breaking it down into individual steps and transforming each step into a separate event while retaining top-level fields. At the final level, it outputs both the full file path and the file name.
+
+```PATH_UNROLL(filter, lookup_fields, output_fields[, parameters])```
+
+- filter: optional
+- lookup_fields: required
+- output_fields: required
+- parameters: optional
+ - separator: <String> optional. The delimiter used to split the path. Default is `/`.
+
+Example Usage:
+
+```yaml
+- function: PATH_UNROLL
+ lookup_fields: [ decoded_path, app]
+ output_fields: [ protocol_stack_id, app_name ]
+ parameters:
+ separator: "."
+```
+Input:
+
+```json
+{"decoded_path":"ETHERNET.IPv4.TCP.ssl","app":"wechat"}
+```
+When the input is processed, the following events are generated:
+```
+ #Event1: {"protocol_stack_id":"ETHERNET"}
+ #Event2: {"protocol_stack_id":"ETHERNET.IPv4"}
+ #Event3: {"protocol_stack_id":"ETHERNET.IPv4.TCP"}
+ #Event4: {"protocol_stack_id":"ETHERNET.IPv4.TCP.ssl"}
+ #Event5: {"app_name":"wechat","protocol_stack_id":"ETHERNET.IPv4.TCP.ssl.wechat"}
+```
+
+If decoded_path contains app value of `ETHERNET.IPv4.TCP.ssl`, the output will be as follows:
+```json
+{"decoded_path":"ETHERNET.IPv4.TCP.ssl","app":"ssl"}
+```
+In this case, the output will be:
+```
+ #Event1: {"protocol_stack_id":"ETHERNET"}
+ #Event2: {"protocol_stack_id":"ETHERNET.IPv4"}
+ #Event3: {"protocol_stack_id":"ETHERNET.IPv4.TCP"}
+ #Event4: {"protocol_stack_id":"ETHERNET.IPv4.TCP.ssl", "app_name":"ssl"}
+```
+
+
diff --git a/groot-bootstrap/pom.xml b/groot-bootstrap/pom.xml
index ab68e08..24a202a 100644
--- a/groot-bootstrap/pom.xml
+++ b/groot-bootstrap/pom.xml
@@ -66,6 +66,13 @@
<dependency>
<groupId>com.geedgenetworks</groupId>
+ <artifactId>connector-starrocks</artifactId>
+ <version>${revision}</version>
+ <scope>${scope}</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>com.geedgenetworks</groupId>
<artifactId>format-json</artifactId>
<version>${revision}</version>
<scope>${scope}</scope>
@@ -85,6 +92,13 @@
<scope>${scope}</scope>
</dependency>
+ <dependency>
+ <groupId>com.geedgenetworks</groupId>
+ <artifactId>format-csv</artifactId>
+ <version>${revision}</version>
+ <scope>${scope}</scope>
+ </dependency>
+
<!-- Idea debug dependencies -->
<dependency>
<groupId>org.apache.flink</groupId>
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/AES128GCM96Shade.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/AES128GCM96Shade.java
new file mode 100644
index 0000000..03ed1af
--- /dev/null
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/AES128GCM96Shade.java
@@ -0,0 +1,72 @@
+package com.geedgenetworks.bootstrap.command;
+
+import cn.hutool.core.util.RandomUtil;
+import com.geedgenetworks.common.crypto.CryptoShade;
+
+import javax.crypto.Cipher;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import java.nio.charset.StandardCharsets;
+import java.security.Key;
+import java.util.Base64;
+
+public class AES128GCM96Shade implements CryptoShade {
+ private static final String IDENTIFIER = "aes-128-gcm96";
+ private static final String ALGORITHM = "AES";
+ private static final String TRANSFORMATION = "AES/GCM/NoPadding";
+ private static final int GCM_TAG_LENGTH = 128;
+ private static final int GCM_NONCE_LENGTH = 12;
+ private static final byte[] NONCE = RandomUtil.randomBytes(GCM_NONCE_LENGTH);
+
+ private static final String[] SENSITIVE_OPTIONS =
+ new String[]{"secret_key", "connection.user", "connection.password", "kafka.sasl.jaas.config", "kafka.ssl.keystore.password", "kafka.ssl.truststore.password", "kafka.ssl.key.password"};
+
+ private static final Key SECURITY_KEY = new SecretKeySpec(".geedgenetworks.".getBytes(StandardCharsets.UTF_8), ALGORITHM);
+
+ @Override
+ public String[] sensitiveOptions() {
+ return SENSITIVE_OPTIONS;
+ }
+
+ @Override
+ public String getIdentifier() {
+ return IDENTIFIER;
+ }
+
+ @Override
+ public String encrypt(String content) {
+ String encryptedString = "";
+ try {
+ Cipher cipher = Cipher.getInstance(TRANSFORMATION);
+ GCMParameterSpec gcmSpec = new GCMParameterSpec(GCM_TAG_LENGTH, NONCE);
+ cipher.init(Cipher.ENCRYPT_MODE, SECURITY_KEY, gcmSpec);
+ byte[] encryptedBytes = cipher.doFinal(content.getBytes());
+ byte[] combinedBytes = new byte[GCM_NONCE_LENGTH + encryptedBytes.length];
+ System.arraycopy(NONCE, 0, combinedBytes, 0, GCM_NONCE_LENGTH);
+ System.arraycopy(encryptedBytes, 0, combinedBytes, GCM_NONCE_LENGTH, encryptedBytes.length);
+ encryptedString = Base64.getEncoder().encodeToString(combinedBytes);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ return encryptedString;
+ }
+
+ @Override
+ public String decrypt(String content) {
+ String decryptedString = "";
+ try {
+ byte[] combined = Base64.getDecoder().decode(content);
+ byte[] encryptedBytes = new byte[combined.length - GCM_NONCE_LENGTH];
+ System.arraycopy(combined, 0, NONCE, 0, GCM_NONCE_LENGTH);
+ System.arraycopy(combined, GCM_NONCE_LENGTH, encryptedBytes, 0, encryptedBytes.length);
+ Cipher cipher = Cipher.getInstance(TRANSFORMATION);
+ GCMParameterSpec gcmSpec = new GCMParameterSpec(GCM_TAG_LENGTH, NONCE);
+ cipher.init(Cipher.DECRYPT_MODE, SECURITY_KEY, gcmSpec);
+ byte[] decryptedBytes = cipher.doFinal(encryptedBytes);
+ decryptedString = new String(decryptedBytes);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ return decryptedString;
+ }
+}
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/AES256GCM96Shade.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/AES256GCM96Shade.java
new file mode 100644
index 0000000..efee134
--- /dev/null
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/AES256GCM96Shade.java
@@ -0,0 +1,72 @@
+package com.geedgenetworks.bootstrap.command;
+
+import cn.hutool.core.util.RandomUtil;
+import com.geedgenetworks.common.crypto.CryptoShade;
+
+import javax.crypto.Cipher;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import java.nio.charset.StandardCharsets;
+import java.security.*;
+import java.util.Base64;
+
+public class AES256GCM96Shade implements CryptoShade {
+ private static final String IDENTIFIER = "aes-256-gcm96";
+ private static final String ALGORITHM = "AES";
+ private static final String TRANSFORMATION = "AES/GCM/NoPadding";
+ private static final int GCM_TAG_LENGTH = 128;
+ private static final int GCM_NONCE_LENGTH = 12;
+ private static final byte[] NONCE = RandomUtil.randomBytes(GCM_NONCE_LENGTH);
+
+ private static final String[] SENSITIVE_OPTIONS =
+ new String[]{"secret_key", "connection.user", "connection.password", "kafka.sasl.jaas.config", "kafka.ssl.keystore.password", "kafka.ssl.truststore.password", "kafka.ssl.key.password"};
+
+ private static final Key SECURITY_KEY = new SecretKeySpec(".........geedgenetworks.........".getBytes(StandardCharsets.UTF_8), ALGORITHM);
+
+ @Override
+ public String[] sensitiveOptions() {
+ return SENSITIVE_OPTIONS;
+ }
+
+ @Override
+ public String getIdentifier() {
+ return IDENTIFIER;
+ }
+
+ @Override
+ public String encrypt(String content) {
+ String encryptedString = null;
+ try {
+ Cipher cipher = Cipher.getInstance(TRANSFORMATION);
+ GCMParameterSpec gcmSpec = new GCMParameterSpec(GCM_TAG_LENGTH, NONCE);
+ cipher.init(Cipher.ENCRYPT_MODE, SECURITY_KEY, gcmSpec);
+ byte[] encryptedBytes = cipher.doFinal(content.getBytes());
+ byte[] combinedBytes = new byte[GCM_NONCE_LENGTH + encryptedBytes.length];
+ System.arraycopy(NONCE, 0, combinedBytes, 0, GCM_NONCE_LENGTH);
+ System.arraycopy(encryptedBytes, 0, combinedBytes, GCM_NONCE_LENGTH, encryptedBytes.length);
+ encryptedString = Base64.getEncoder().encodeToString(combinedBytes);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ return encryptedString;
+ }
+
+ @Override
+ public String decrypt(String content) {
+ String decryptedString = null;
+ try {
+ byte[] combined = Base64.getDecoder().decode(content);
+ byte[] encryptedBytes = new byte[combined.length - GCM_NONCE_LENGTH];
+ System.arraycopy(combined, 0, NONCE, 0, GCM_NONCE_LENGTH);
+ System.arraycopy(combined, GCM_NONCE_LENGTH, encryptedBytes, 0, encryptedBytes.length);
+ Cipher cipher = Cipher.getInstance(TRANSFORMATION);
+ GCMParameterSpec gcmSpec = new GCMParameterSpec(GCM_TAG_LENGTH, NONCE);
+ cipher.init(Cipher.DECRYPT_MODE, SECURITY_KEY, gcmSpec);
+ byte[] decryptedBytes = cipher.doFinal(encryptedBytes);
+ decryptedString = new String(decryptedBytes);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ return decryptedString;
+ }
+}
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/AESConfigShade.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/AESShade.java
index 76b4944..91e05d0 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/AESConfigShade.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/AESShade.java
@@ -2,14 +2,15 @@ package com.geedgenetworks.bootstrap.command;
import cn.hutool.crypto.SecureUtil;
import cn.hutool.crypto.symmetric.SymmetricAlgorithm;
-import com.geedgenetworks.common.config.ConfigShade;
+import com.geedgenetworks.common.crypto.CryptoShade;
+
import java.nio.charset.StandardCharsets;
-public class AESConfigShade implements ConfigShade {
+public class AESShade implements CryptoShade {
private static final String IDENTIFIER = "aes";
private static final byte[] SECURITY_KEY =
- SecureUtil.generateKey(SymmetricAlgorithm.AES.getValue(), ".geedgenetworks.".getBytes(StandardCharsets.UTF_8)).getEncoded() ;
+ SecureUtil.generateKey(SymmetricAlgorithm.AES.getValue(), ".geedgenetworks.".getBytes(StandardCharsets.UTF_8)).getEncoded();
private static final String[] SENSITIVE_OPTIONS =
new String[] {"connection.user", "connection.password", "kafka.sasl.jaas.config","kafka.ssl.keystore.password","kafka.ssl.truststore.password","kafka.ssl.key.password"};
@@ -26,7 +27,7 @@ public class AESConfigShade implements ConfigShade {
@Override
public String encrypt(String content) {
- return SecureUtil.aes(SECURITY_KEY).encryptHex(content, StandardCharsets.UTF_8);
+ return SecureUtil.aes(SECURITY_KEY).encryptBase64(content, StandardCharsets.UTF_8);
}
@Override
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/Base64ConfigShade.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/Base64Shade.java
index 4ae2b5c..d07c372 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/Base64ConfigShade.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/Base64Shade.java
@@ -1,10 +1,10 @@
package com.geedgenetworks.bootstrap.command;
-import com.geedgenetworks.common.config.ConfigShade;
+import com.geedgenetworks.common.crypto.CryptoShade;
import java.nio.charset.StandardCharsets;
import java.util.Base64;
-public class Base64ConfigShade implements ConfigShade {
+public class Base64Shade implements CryptoShade {
private static final Base64.Encoder ENCODER = Base64.getEncoder();
private static final Base64.Decoder DECODER = Base64.getDecoder();
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/ConfDecryptCommand.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/ConfDecryptCommand.java
index 75f7819..b124c9d 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/ConfDecryptCommand.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/ConfDecryptCommand.java
@@ -5,12 +5,10 @@ import com.geedgenetworks.bootstrap.exception.CommandExecuteException;
import com.geedgenetworks.bootstrap.exception.ConfigCheckException;
import com.geedgenetworks.bootstrap.utils.ConfigBuilder;
import com.geedgenetworks.bootstrap.utils.ConfigFileUtils;
-import com.geedgenetworks.bootstrap.utils.ConfigShadeUtils;
import com.typesafe.config.*;
import lombok.extern.slf4j.Slf4j;
import java.nio.file.Path;
-import java.nio.file.Paths;
import java.util.Map;
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/ConfEncryptCommand.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/ConfEncryptCommand.java
index 676cba5..1d7be2e 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/ConfEncryptCommand.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/ConfEncryptCommand.java
@@ -5,12 +5,11 @@ import com.geedgenetworks.bootstrap.exception.CommandExecuteException;
import com.geedgenetworks.bootstrap.exception.ConfigCheckException;
import com.geedgenetworks.bootstrap.utils.ConfigBuilder;
import com.geedgenetworks.bootstrap.utils.ConfigFileUtils;
-import com.geedgenetworks.bootstrap.utils.ConfigShadeUtils;
+import com.geedgenetworks.bootstrap.utils.CryptoShadeUtils;
import com.typesafe.config.*;
import lombok.extern.slf4j.Slf4j;
import java.nio.file.Path;
-import java.nio.file.Paths;
import java.util.Map;
import static com.geedgenetworks.bootstrap.utils.ConfigFileUtils.checkConfigExist;
@@ -32,7 +31,7 @@ public class ConfEncryptCommand implements Command<ExecuteCommandArgs>{
checkConfigExist(configPath);
Map<String, Object> configMap = YamlUtil.loadByPath(configPath.toString());
Config config = ConfigBuilder.of(configMap, false);
- Config encryptConfig = ConfigShadeUtils.encryptConfig(config);
+ Config encryptConfig = CryptoShadeUtils.encryptConfig(config);
System.out.println(String.format(
"Encrypt config: %s", encryptConfig.root().render(ConfigRenderOptions.defaults().setOriginComments(false))));
log.info(
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/ExecuteCommandArgs.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/ExecuteCommandArgs.java
index 61ced82..5b36671 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/ExecuteCommandArgs.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/ExecuteCommandArgs.java
@@ -98,6 +98,7 @@ public class ExecuteCommandArgs extends CommandArgs {
static {
TARGET_TYPE_LIST.add(TargetType.LOCAL);
+ TARGET_TYPE_LIST.add(TargetType.TEST);
TARGET_TYPE_LIST.add(TargetType.REMOTE);
TARGET_TYPE_LIST.add(TargetType.YARN_SESSION);
TARGET_TYPE_LIST.add(TargetType.YARN_PER_JOB);
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/SM4GCM96Shade.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/SM4GCM96Shade.java
new file mode 100644
index 0000000..a6d27e4
--- /dev/null
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/SM4GCM96Shade.java
@@ -0,0 +1,73 @@
+package com.geedgenetworks.bootstrap.command;
+
+import cn.hutool.core.util.RandomUtil;
+import com.geedgenetworks.common.crypto.CryptoShade;
+
+import javax.crypto.Cipher;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import java.nio.charset.StandardCharsets;
+import java.security.Key;
+import java.util.Base64;
+
+public class SM4GCM96Shade implements CryptoShade {
+ private static final String IDENTIFIER = "sm4-gcm96";
+ private static final String ALGORITHM = "SM4";
+ private static final String TRANSFORMATION = "SM4/GCM/NoPadding";
+ private static final int GCM_TAG_LENGTH = 128;
+ private static final int GCM_NONCE_LENGTH = 12;
+ private static final byte[] NONCE = RandomUtil.randomBytes(GCM_NONCE_LENGTH);
+
+ private static final String[] SENSITIVE_OPTIONS =
+ new String[]{"connection.user", "connection.password", "kafka.sasl.jaas.config", "kafka.ssl.keystore.password", "kafka.ssl.truststore.password", "kafka.ssl.key.password"};
+
+ private static final Key SECURITY_KEY = new SecretKeySpec(".geedgenetworks.".getBytes(StandardCharsets.UTF_8), ALGORITHM);
+
+ @Override
+ public String[] sensitiveOptions() {
+ return SENSITIVE_OPTIONS;
+ }
+
+ @Override
+ public String getIdentifier() {
+ return IDENTIFIER;
+ }
+
+ @Override
+ public String encrypt(String content) {
+ String encryptedString = null;
+ try {
+
+ Cipher cipher = Cipher.getInstance(TRANSFORMATION);
+ GCMParameterSpec gcmSpec = new GCMParameterSpec(GCM_TAG_LENGTH, NONCE);
+ cipher.init(Cipher.ENCRYPT_MODE, SECURITY_KEY, gcmSpec);
+ byte[] encryptedBytes = cipher.doFinal(content.getBytes());
+ byte[] combinedBytes = new byte[GCM_NONCE_LENGTH + encryptedBytes.length];
+ System.arraycopy(NONCE, 0, combinedBytes, 0, GCM_NONCE_LENGTH);
+ System.arraycopy(encryptedBytes, 0, combinedBytes, GCM_NONCE_LENGTH, encryptedBytes.length);
+ encryptedString = Base64.getEncoder().encodeToString(combinedBytes);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ return encryptedString;
+ }
+
+ @Override
+ public String decrypt(String content) {
+ String decryptedString = null;
+ try {
+ byte[] combined = Base64.getDecoder().decode(content);
+ byte[] encryptedBytes = new byte[combined.length - GCM_NONCE_LENGTH];
+ System.arraycopy(combined, 0, NONCE, 0, GCM_NONCE_LENGTH);
+ System.arraycopy(combined, GCM_NONCE_LENGTH, encryptedBytes, 0, encryptedBytes.length);
+ Cipher cipher = Cipher.getInstance(TRANSFORMATION);
+ GCMParameterSpec gcmSpec = new GCMParameterSpec(GCM_TAG_LENGTH, NONCE);
+ cipher.init(Cipher.DECRYPT_MODE, SECURITY_KEY, gcmSpec);
+ byte[] decryptedBytes = cipher.doFinal(encryptedBytes);
+ decryptedString = new String(decryptedBytes);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ return decryptedString;
+ }
+}
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/SM4ConfigShade.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/SM4Shade.java
index 05d3e52..e274716 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/SM4ConfigShade.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/command/SM4Shade.java
@@ -3,11 +3,11 @@ package com.geedgenetworks.bootstrap.command;
import cn.hutool.crypto.KeyUtil;
import cn.hutool.crypto.SmUtil;
import cn.hutool.crypto.symmetric.SM4;
-import com.geedgenetworks.common.config.ConfigShade;
+import com.geedgenetworks.common.crypto.CryptoShade;
import java.nio.charset.StandardCharsets;
-public class SM4ConfigShade implements ConfigShade {
+public class SM4Shade implements CryptoShade {
private static final String IDENTIFIER = "sm4";
private static final String[] SENSITIVE_OPTIONS =
@@ -27,7 +27,7 @@ public class SM4ConfigShade implements ConfigShade {
@Override
public String encrypt(String content) {
- return SmUtil.sm4(SECURITY_KEY).encryptHex(content, StandardCharsets.UTF_8);
+ return SmUtil.sm4(SECURITY_KEY).encryptBase64(content, StandardCharsets.UTF_8);
}
@Override
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/enums/TargetType.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/enums/TargetType.java
index abf1c1f..bdc70d0 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/enums/TargetType.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/enums/TargetType.java
@@ -4,6 +4,7 @@ public enum TargetType {
LOCAL("local"),
REMOTE("remote"),
+ TEST("test"),
YARN_SESSION("yarn-session"),
YARN_PER_JOB("yarn-per-job"),
YARN_APPLICATION("yarn-application");
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractExecutor.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractExecutor.java
index 7a55ffe..f5b1a5d 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractExecutor.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractExecutor.java
@@ -14,30 +14,15 @@ import java.net.URLClassLoader;
import java.util.*;
import java.util.function.BiConsumer;
-public abstract class AbstractExecutor<K, V>
+public abstract class AbstractExecutor<K, V>
implements Executor<DataStream<Event>, JobRuntimeEnvironment> {
protected JobRuntimeEnvironment jobRuntimeEnvironment;
protected final Config operatorConfig;
protected final Map<K,V> operatorMap;
- protected final Map<String,Filter> filterMap = new HashMap<>();
- protected final Map<String, Split> splitMap = new HashMap<>();
- protected final Map<String, Processor> processorMap = new HashMap<>();
protected AbstractExecutor(List<URL> jarPaths, Config operatorConfig) {
this.operatorConfig = operatorConfig;
this.operatorMap = initialize(jarPaths, operatorConfig);
- ServiceLoader<Filter> filters = ServiceLoader.load(Filter.class);
- for (Filter filter : filters) {
- this.filterMap.put(filter.type(), filter);
- }
- ServiceLoader<Split> splits = ServiceLoader.load(Split.class);
- for (Split split : splits) {
- this.splitMap.put(split.type(), split);
- }
- ServiceLoader<Processor> processors = ServiceLoader.load(Processor.class);
- for (Processor processor : processors) {
- this.processorMap.put(processor.type(), processor);
- }
}
@Override
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractProcessorExecutor.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractProcessorExecutor.java
index b0a04cd..42a3a11 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractProcessorExecutor.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/AbstractProcessorExecutor.java
@@ -1,25 +1,19 @@
package com.geedgenetworks.bootstrap.execution;
-import com.alibaba.fastjson.JSONObject;
import com.geedgenetworks.bootstrap.exception.JobExecuteException;
import com.geedgenetworks.common.Event;
import com.geedgenetworks.common.config.*;
import com.geedgenetworks.common.exception.CommonErrorCode;
import com.geedgenetworks.common.exception.ConfigValidationException;
-import com.geedgenetworks.core.pojo.AggregateConfig;
import com.geedgenetworks.core.pojo.ProcessorConfig;
-import com.geedgenetworks.core.pojo.ProjectionConfig;
-import com.geedgenetworks.core.pojo.TableConfig;
-import com.geedgenetworks.core.processor.table.TableProcessor;
-import com.geedgenetworks.core.processor.aggregate.AggregateProcessor;
-import com.geedgenetworks.core.processor.projection.ProjectionProcessor;
+import com.geedgenetworks.core.processor.Processor;
import com.typesafe.config.Config;
import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
import java.net.URL;
import java.util.List;
import java.util.Map;
+import java.util.ServiceLoader;
public abstract class AbstractProcessorExecutor extends AbstractExecutor<String, ProcessorConfig> {
@@ -32,103 +26,33 @@ public abstract class AbstractProcessorExecutor extends AbstractExecutor<String,
public DataStream<Event> execute(DataStream<Event> dataStream, Node node) throws JobExecuteException {
ProcessorConfig processorConfig = operatorMap.get(node.getName());
- switch (processorConfig.getType()) {
- case "aggregate":
- dataStream = executeAggregateProcessor(dataStream, node, (AggregateConfig) processorConfig);
+ boolean found = false; // 标志变量
+ ServiceLoader<Processor> processors = ServiceLoader.load(Processor.class);
+ for (Processor processor : processors) {
+ if(processor.type().equals(processorConfig.getType())){
+ found = true;
+ if (node.getParallelism() > 0) {
+ processorConfig.setParallelism(node.getParallelism());
+ }
+ try {
+
+ dataStream = processor.processorFunction(
+ dataStream, processorConfig, jobRuntimeEnvironment.getStreamExecutionEnvironment().getConfig());
+ } catch (Exception e) {
+ throw new JobExecuteException("Create orderby pipeline instance failed!", e);
+ }
break;
- case "table":
- dataStream = executeTableProcessor(dataStream, node, (TableConfig) processorConfig);
- break;
- case "projection":
- dataStream = executeProjectionProcessor(dataStream, node, (ProjectionConfig) processorConfig);
- break;
- default:// 兼容历史版本
- dataStream = executeProjectionProcessor(dataStream, node, (ProjectionConfig) processorConfig);
- }
- return dataStream;
- }
- protected DataStream<Event> executeTableProcessor(DataStream<Event> dataStream, Node node, TableConfig tableConfig) throws JobExecuteException {
-
- TableProcessor tableProcessor;
- if (processorMap.containsKey(tableConfig.getType())) {
- tableProcessor = (TableProcessor) processorMap.get(tableConfig.getType());
- } else {
- Class cls;
- try {
- cls = Class.forName(tableConfig.getType());
- tableProcessor = (TableProcessor) cls.newInstance();
- } catch (ClassNotFoundException | InstantiationException | IllegalAccessException | RuntimeException e) {
- throw new JobExecuteException("get processing pipeline instance failed!", e);
- }
- }
- if (node.getParallelism() > 0) {
- tableConfig.setParallelism(node.getParallelism());
- }
- try {
-
- dataStream = tableProcessor.processorFunction(
- dataStream, tableConfig, jobRuntimeEnvironment.getStreamExecutionEnvironment().getConfig());
- } catch (Exception e) {
- throw new JobExecuteException("Create orderby pipeline instance failed!", e);
- }
- return dataStream;
- }
- protected DataStream<Event> executeAggregateProcessor(DataStream<Event> dataStream, Node node, AggregateConfig aggregateConfig) throws JobExecuteException {
-
- AggregateProcessor aggregateProcessor;
- if (processorMap.containsKey(aggregateConfig.getType())) {
- aggregateProcessor = (AggregateProcessor) processorMap.get(aggregateConfig.getType());
- } else {
- Class cls;
- try {
- cls = Class.forName(aggregateConfig.getType());
- aggregateProcessor = (AggregateProcessor) cls.newInstance();
- } catch (ClassNotFoundException | InstantiationException | IllegalAccessException | RuntimeException e) {
- throw new JobExecuteException("get processing pipeline instance failed!", e);
- }
- }
- if (node.getParallelism() > 0) {
- aggregateConfig.setParallelism(node.getParallelism());
- }
- try {
- dataStream =
- aggregateProcessor.processorFunction(
- dataStream, aggregateConfig, jobRuntimeEnvironment.getStreamExecutionEnvironment().getConfig());
- } catch (Exception e) {
- throw new JobExecuteException("Create aggregate pipeline instance failed!", e);
- }
- return dataStream;
- }
-
- protected DataStream<Event> executeProjectionProcessor(DataStream<Event> dataStream, Node node, ProjectionConfig projectionConfig) throws JobExecuteException {
-
- ProjectionProcessor projectionProcessor;
- if (processorMap.containsKey(projectionConfig.getType())) {
- projectionProcessor = (ProjectionProcessor) processorMap.get(projectionConfig.getType());
- } else {
- Class cls;
- try {
- cls = Class.forName(projectionConfig.getType());
- projectionProcessor = (ProjectionProcessor) cls.newInstance();
- } catch (ClassNotFoundException | InstantiationException | IllegalAccessException | RuntimeException e) {
- throw new JobExecuteException("get processing pipeline instance failed!", e);
}
}
- if (node.getParallelism() > 0) {
- projectionConfig.setParallelism(node.getParallelism());
- }
- try {
- dataStream =
- projectionProcessor.processorFunction(
- dataStream, projectionConfig,jobRuntimeEnvironment.getStreamExecutionEnvironment().getConfig());
- } catch (Exception e) {
- throw new JobExecuteException("Create processing pipeline instance failed!", e);
+ if (!found) {
+ throw new JobExecuteException("No matching processor found for type: " + processorConfig.getType());
}
return dataStream;
}
- protected ProcessorConfig checkProcessorConfig(String key, Map<String, Object> value, Config processorsConfig) {
- ProcessorConfig projectionConfig;
+ protected ProcessorConfig checkConfig(String key, Map<String, Object> value, Config processorsConfig) {
+ ProcessorConfig ProcessorConfig = new ProcessorConfig();
+ boolean found = false; // 标志变量
CheckResult result = CheckConfigUtil.checkAllExists(processorsConfig.getConfig(key),
ProjectionConfigOptions.TYPE.key());
if (!result.isSuccess()) {
@@ -136,84 +60,29 @@ public abstract class AbstractProcessorExecutor extends AbstractExecutor<String,
"Postprocessor: %s, Message: %s",
key, result.getMsg()));
}
- switch ((String) value.getOrDefault("type", "")) {
- case "projection":
- projectionConfig = checkProjectionProcessorConfig(key, value, processorsConfig);
+ ServiceLoader<Processor> processors = ServiceLoader.load(Processor.class);
+ for (Processor processor : processors) {
+ if(processor.type().equals(value.getOrDefault("type", "").toString())){
+ found = true;
+ try {
+ ProcessorConfig = processor.checkConfig(key, value, processorsConfig);
+
+ } catch (Exception e) {
+ throw new JobExecuteException("Create orderby pipeline instance failed!", e);
+ }
break;
- case "aggregate":
- projectionConfig = checkAggregateProcessorConfig(key, value, processorsConfig);
- break;
- case "table":
- projectionConfig = checkTableProcessorConfig(key, value, processorsConfig);
- break;
- default://兼容历史版本
- projectionConfig = checkProjectionProcessorConfig(key, value, processorsConfig);
+ }
}
- return projectionConfig;
- }
-
- protected ProcessorConfig checkProjectionProcessorConfig(String key, Map<String, Object> value, Config projectionProcessors) {
-
- CheckResult result = CheckConfigUtil.checkAtLeastOneExists(projectionProcessors.getConfig(key),
- ProjectionConfigOptions.OUTPUT_FIELDS.key(),
- ProjectionConfigOptions.REMOVE_FIELDS.key(),
- ProjectionConfigOptions.FUNCTIONS.key());
- if (!result.isSuccess()) {
- throw new ConfigValidationException(CommonErrorCode.CONFIG_VALIDATION_FAILED, String.format(
- "Processor: %s, At least one of [%s] should be specified.",
- key, String.join(",",
- ProjectionConfigOptions.OUTPUT_FIELDS.key(),
- ProjectionConfigOptions.REMOVE_FIELDS.key(),
- ProjectionConfigOptions.FUNCTIONS.key())));
+ if (!found) {
+ throw new JobExecuteException("No matching processor found for type: " + value.getOrDefault("type", "").toString());
}
-
- ProjectionConfig projectionConfig = new JSONObject(value).toJavaObject(ProjectionConfig.class);
- projectionConfig.setName(key);
-
- return projectionConfig;
+ return ProcessorConfig;
}
- protected AggregateConfig checkAggregateProcessorConfig(String key, Map<String, Object> value, Config aggregateProcessorsConfig) {
- CheckResult result = CheckConfigUtil.checkAllExists(aggregateProcessorsConfig.getConfig(key),
- AggregateConfigOptions.GROUP_BY_FIELDS.key(),
- AggregateConfigOptions.WINDOW_TYPE.key(),
- AggregateConfigOptions.FUNCTIONS.key(),
- AggregateConfigOptions.WINDOW_SIZE.key());
- if (!result.isSuccess()) {
- throw new ConfigValidationException(CommonErrorCode.CONFIG_VALIDATION_FAILED, String.format(
- "Aggregate processor: %s, At least one of [%s] should be specified.",
- key, String.join(",",
- AggregateConfigOptions.OUTPUT_FIELDS.key(),
- AggregateConfigOptions.REMOVE_FIELDS.key(),
- AggregateConfigOptions.FUNCTIONS.key())));
- }
-
- AggregateConfig aggregateConfig = new JSONObject(value).toJavaObject(AggregateConfig.class);
- aggregateConfig.setName(key);
- return aggregateConfig;
- }
- protected TableConfig checkTableProcessorConfig(String key, Map<String, Object> value, Config config) {
- CheckResult result = CheckConfigUtil.checkAtLeastOneExists(config.getConfig(key),
- TableConfigOptions.OUTPUT_FIELDS.key(),
- TableConfigOptions.REMOVE_FIELDS.key(),
- TableConfigOptions.FUNCTIONS.key());
- if (!result.isSuccess()) {
- throw new ConfigValidationException(CommonErrorCode.CONFIG_VALIDATION_FAILED, String.format(
- "Table processor: %s, At least one of [%s] should be specified.",
- key, String.join(",",
- TableConfigOptions.OUTPUT_FIELDS.key(),
- TableConfigOptions.REMOVE_FIELDS.key(),
- TableConfigOptions.FUNCTIONS.key())));
- }
-
- TableConfig tableConfig = new JSONObject(value).toJavaObject(TableConfig.class);
- tableConfig.setName(key);
- return tableConfig;
- }
}
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/FilterExecutor.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/FilterExecutor.java
index 66f0585..f3c81c2 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/FilterExecutor.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/FilterExecutor.java
@@ -8,10 +8,12 @@ import com.geedgenetworks.common.Event;
import com.geedgenetworks.common.config.CheckConfigUtil;
import com.geedgenetworks.common.config.CheckResult;
import com.geedgenetworks.common.config.FilterConfigOptions;
+import com.geedgenetworks.common.config.ProjectionConfigOptions;
import com.geedgenetworks.common.exception.CommonErrorCode;
import com.geedgenetworks.common.exception.ConfigValidationException;
import com.geedgenetworks.core.filter.Filter;
import com.geedgenetworks.core.pojo.FilterConfig;
+
import com.google.common.collect.Maps;
import com.typesafe.config.Config;
import lombok.extern.slf4j.Slf4j;
@@ -21,6 +23,7 @@ import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
import java.net.URL;
import java.util.List;
import java.util.Map;
+import java.util.ServiceLoader;
/**
* Initialize config and execute filter operator
@@ -37,18 +40,16 @@ public class FilterExecutor extends AbstractExecutor<String, FilterConfig> {
protected Map<String, FilterConfig> initialize(List<URL> jarPaths, Config operatorConfig) {
Map<String, FilterConfig> filterConfigMap = Maps.newHashMap();
if (operatorConfig.hasPath(Constants.FILTERS)) {
- Config filters = operatorConfig.getConfig(Constants.FILTERS);
- filters.root().unwrapped().forEach((key, value) -> {
- CheckResult result = CheckConfigUtil.checkAllExists(filters.getConfig(key),
+ Config filterConfig = operatorConfig.getConfig(Constants.FILTERS);
+ filterConfig.root().unwrapped().forEach((key, value) -> {
+ CheckResult result = CheckConfigUtil.checkAllExists(filterConfig.getConfig(key),
FilterConfigOptions.TYPE.key(), FilterConfigOptions.PROPERTIES.key());
if (!result.isSuccess()) {
throw new ConfigValidationException(CommonErrorCode.CONFIG_VALIDATION_FAILED, String.format(
"Filter: %s, Message: %s",
key, result.getMsg()));
}
- FilterConfig filterConfig = new JSONObject((Map<String, Object>) value).toJavaObject(FilterConfig.class);
- filterConfig.setName(key);
- filterConfigMap.put(key, filterConfig);
+ filterConfigMap.put(key, checkConfig(key, (Map<String, Object>) value, filterConfig));
});
}
@@ -58,30 +59,47 @@ public class FilterExecutor extends AbstractExecutor<String, FilterConfig> {
@Override
public DataStream<Event> execute(DataStream<Event> dataStream, Node node) throws JobExecuteException {
FilterConfig filterConfig = operatorMap.get(node.getName());
- String className = filterConfig.getType();
- Filter filter;
- if (filterMap.containsKey(filterConfig.getType())) {
-
- filter = filterMap.get(filterConfig.getType());
- } else {
- Class cls;
- try {
- cls = Class.forName(className);
- filter = (Filter) cls.newInstance();
- } catch (ClassNotFoundException | InstantiationException | IllegalAccessException | RuntimeException e) {
- throw new JobExecuteException("get filter instance failed!", e);
+ boolean found = false; // 标志变量
+ ServiceLoader<Filter> filters = ServiceLoader.load(Filter.class);
+ for (Filter filter : filters) {
+ if(filter.type().equals(filterConfig.getType())){
+ found = true;
+ if (node.getParallelism() > 0) {
+ filterConfig.setParallelism(node.getParallelism());
+ }
+ try {
+ dataStream =
+ filter.filterFunction(
+ dataStream, filterConfig);
+ } catch (Exception e) {
+ throw new JobExecuteException("Create filter instance failed!", e);
+ }
+ break;
}
}
- if (node.getParallelism() > 0) {
- filterConfig.setParallelism(node.getParallelism());
- }
- try {
- dataStream =
- filter.filterFunction(
- dataStream, filterConfig);
- } catch (Exception e) {
- throw new JobExecuteException("Create filter instance failed!", e);
+ if (!found) {
+ throw new JobExecuteException("No matching filter found for type: " + filterConfig.getType());
}
return dataStream;
}
+
+ protected FilterConfig checkConfig(String key, Map<String, Object> value, Config config) {
+ FilterConfig filterConfig = new FilterConfig();
+ boolean found = false; // 标志变量
+ ServiceLoader<Filter> filters = ServiceLoader.load(Filter.class);
+ for (Filter filter : filters) {
+ if(filter.type().equals(value.getOrDefault("type", "").toString())){
+ found = true;
+ try {
+ filterConfig = filter.checkConfig(key, value, config);
+ } catch (Exception e) {
+ throw new JobExecuteException("Create split pipeline instance failed!", e);
+ }
+ }
+ }
+ if (!found) {
+ throw new JobExecuteException("No matching filter found for type: " + value.getOrDefault("type", "").toString());
+ }
+ return filterConfig;
+ }
}
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/JobExecution.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/JobExecution.java
index f6e19eb..706fc18 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/JobExecution.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/JobExecution.java
@@ -6,14 +6,7 @@ import com.geedgenetworks.bootstrap.exception.JobExecuteException;
import com.geedgenetworks.bootstrap.main.GrootStreamRunner;
import com.geedgenetworks.common.Constants;
import com.geedgenetworks.common.Event;
-import com.geedgenetworks.common.config.CheckConfigUtil;
-import com.geedgenetworks.common.config.CheckResult;
import com.geedgenetworks.common.config.GrootStreamConfig;
-import com.geedgenetworks.common.config.SplitConfigOptions;
-import com.geedgenetworks.common.exception.CommonErrorCode;
-import com.geedgenetworks.common.exception.ConfigValidationException;
-import com.geedgenetworks.common.udf.RuleContext;
-import com.geedgenetworks.core.pojo.SplitConfig;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.typesafe.config.Config;
@@ -45,7 +38,7 @@ public class JobExecution {
private final Executor<DataStream<Event>, JobRuntimeEnvironment> postprocessingExecutor;
private final List<Node> nodes;
private final List<URL> jarPaths;
- private final Set<String> splitSet = new HashSet<>();
+ private final Map<String,String> nodeNameWithSplitTags = new HashMap<>();
public JobExecution(Config jobConfig, GrootStreamConfig grootStreamConfig) {
try {
@@ -209,12 +202,6 @@ public class JobExecution {
} else if (filters.containsKey(node.getName())) {
node.setType(ProcessorType.FILTER);
} else if (splits.containsKey(node.getName())) {
- splits.forEach((key, value) -> {
- SplitConfig splitConfig = new com.alibaba.fastjson.JSONObject((Map<String, Object>) value).toJavaObject(SplitConfig.class);
- for(RuleContext ruleContext:splitConfig.getRules()) {
- splitSet.add(ruleContext.getName());
- }
- });
node.setType(ProcessorType.SPLIT);
} else if (preprocessingPipelines.containsKey(node.getName())) {
node.setType(ProcessorType.PREPROCESSING);
@@ -233,7 +220,7 @@ public class JobExecution {
public void execute() throws JobExecuteException {
- if (!jobRuntimeEnvironment.isLocalMode()) {
+ if (!jobRuntimeEnvironment.isLocalMode() && !jobRuntimeEnvironment.isTestMode()) {
jobRuntimeEnvironment.registerPlugin(jarPaths);
}
List<Node> sourceNodes = nodes
@@ -268,39 +255,46 @@ public class JobExecution {
throw new JobExecuteException("Can't find downstream node " + downstreamNodeName);
});
if (node.getType().name().equals(ProcessorType.FILTER.name())) {
- if (splitSet.contains(node.getName())) {
- dataStream = filterExecutor.execute(((SingleOutputStreamOperator<Event>) dataStream).getSideOutput(new OutputTag<Event>(node.getName()) {
+ if (nodeNameWithSplitTags.containsKey(node.getName())) {
+ dataStream = filterExecutor.execute(((SingleOutputStreamOperator<Event>) dataStream).getSideOutput(new OutputTag<Event>(nodeNameWithSplitTags.get(node.getName())) {
}), node);
} else {
dataStream = filterExecutor.execute(dataStream, node);
}
} else if (node.getType().name().equals(ProcessorType.SPLIT.name())) {
+ if (node.getTags().size() == node.getDownstream().size()) {
+ for (int i = 0; i < node.getDownstream().size();i++) {
+ nodeNameWithSplitTags.put(node.getDownstream().get(i),node.getTags().get(i));
+ }
+ }
+ else {
+ throw new JobExecuteException("split node downstream size not equal tags size");
+ }
dataStream = splitExecutor.execute(dataStream, node);
-
} else if (node.getType().name().equals(ProcessorType.PREPROCESSING.name())) {
- if (splitSet.contains(node.getName())) {
- dataStream = preprocessingExecutor.execute(((SingleOutputStreamOperator<Event>) dataStream).getSideOutput(new OutputTag<Event>(node.getName()) {
+ if (nodeNameWithSplitTags.containsKey(node.getName())) {
+ dataStream = preprocessingExecutor.execute(((SingleOutputStreamOperator<Event>) dataStream).getSideOutput(new OutputTag<Event>(nodeNameWithSplitTags.get(node.getName())){
}), node);
} else {
dataStream = preprocessingExecutor.execute(dataStream, node);
}
} else if (node.getType().name().equals(ProcessorType.PROCESSING.name())) {
- if (splitSet.contains(node.getName())) {
- dataStream = processingExecutor.execute(((SingleOutputStreamOperator<Event>) dataStream).getSideOutput(new OutputTag<Event>(node.getName()) {
+ if (nodeNameWithSplitTags.containsKey(node.getName())) {
+ dataStream = processingExecutor.execute(((SingleOutputStreamOperator<Event>) dataStream).getSideOutput(new OutputTag<Event>(nodeNameWithSplitTags.get(node.getName())) {
}), node);
} else {
dataStream = processingExecutor.execute(dataStream, node);
}
} else if (node.getType().name().equals(ProcessorType.POSTPROCESSING.name())) {
- if (splitSet.contains(node.getName())) {
- dataStream = postprocessingExecutor.execute(((SingleOutputStreamOperator<Event>) dataStream).getSideOutput(new OutputTag<Event>(node.getName()) {
+ if (nodeNameWithSplitTags.containsKey(node.getName())) {
+ dataStream = postprocessingExecutor.execute(((SingleOutputStreamOperator<Event>) dataStream).getSideOutput(new OutputTag<Event>(nodeNameWithSplitTags.get(node.getName())) {
}), node);
} else {
dataStream = postprocessingExecutor.execute(dataStream, node);
}
} else if (node.getType().name().equals(ProcessorType.SINK.name())) {
- if (splitSet.contains(node.getName())) {
- dataStream = sinkExecutor.execute(((SingleOutputStreamOperator<Event>) dataStream).getSideOutput(new OutputTag<Event>(node.getName()) {
+ if (nodeNameWithSplitTags.containsKey(node.getName())) {
+ dataStream = sinkExecutor.execute(((SingleOutputStreamOperator<Event>) dataStream).getSideOutput(new OutputTag<Event>(nodeNameWithSplitTags.get(node.getName())) {
}), node);
} else {
dataStream = sinkExecutor.execute(dataStream, node);
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/JobRuntimeEnvironment.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/JobRuntimeEnvironment.java
index a4289ff..e23d446 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/JobRuntimeEnvironment.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/JobRuntimeEnvironment.java
@@ -93,7 +93,10 @@ public class JobRuntimeEnvironment implements RuntimeEnvironment{
return envConfig.hasPath(ExecutionConfigKeyName.ENV_TARGET_TYPE)
&& envConfig.getString(ExecutionConfigKeyName.ENV_TARGET_TYPE).equals(TargetType.LOCAL.getTarget());
}
-
+ public boolean isTestMode() {
+ return envConfig.hasPath(ExecutionConfigKeyName.ENV_TARGET_TYPE)
+ && envConfig.getString(ExecutionConfigKeyName.ENV_TARGET_TYPE).equals(TargetType.TEST.getTarget());
+ }
@Override
public void registerPlugin(List<URL> pluginPaths) {
pluginPaths.forEach(url -> log.info("Begin register plugins: {}", url));
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/Node.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/Node.java
index f86d106..66303c2 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/Node.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/Node.java
@@ -19,5 +19,6 @@ public class Node implements Serializable {
private ProcessorType type;
private List<String> downstream = Collections.emptyList();
private int parallelism;
+ private List<String> tags = Collections.emptyList();
}
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/PostprocessingExecutor.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/PostprocessingExecutor.java
index b9555b4..03e5bd5 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/PostprocessingExecutor.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/PostprocessingExecutor.java
@@ -30,7 +30,7 @@ public class PostprocessingExecutor extends AbstractProcessorExecutor {
if (operatorConfig.hasPath(Constants.POSTPROCESSING_PIPELINES)) {
Config postprocessors = operatorConfig.getConfig(Constants.POSTPROCESSING_PIPELINES);
postprocessors.root().unwrapped().forEach((key, value) -> {
- postprocessingConfigMap.put(key, checkProcessorConfig(key, (Map<String, Object>) value, postprocessors));
+ postprocessingConfigMap.put(key, checkConfig(key, (Map<String, Object>) value, postprocessors));
});
}
return postprocessingConfigMap;
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/PreprocessingExecutor.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/PreprocessingExecutor.java
index a7b9e5e..da8dc62 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/PreprocessingExecutor.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/PreprocessingExecutor.java
@@ -32,7 +32,7 @@ public class PreprocessingExecutor extends AbstractProcessorExecutor {
if (operatorConfig.hasPath(Constants.PREPROCESSING_PIPELINES)) {
Config preprocessors = operatorConfig.getConfig(Constants.PREPROCESSING_PIPELINES);
preprocessors.root().unwrapped().forEach((key, value) -> {
- preprocessingConfigMap.put(key, checkProcessorConfig(key, (Map<String, Object>) value, preprocessors));
+ preprocessingConfigMap.put(key, checkConfig(key, (Map<String, Object>) value, preprocessors));
});
}
return preprocessingConfigMap;
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/ProcessingExecutor.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/ProcessingExecutor.java
index f6788ed..cf6b496 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/ProcessingExecutor.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/ProcessingExecutor.java
@@ -30,7 +30,7 @@ public class ProcessingExecutor extends AbstractProcessorExecutor {
if (operatorConfig.hasPath(Constants.PROCESSING_PIPELINES)) {
Config processors = operatorConfig.getConfig(Constants.PROCESSING_PIPELINES);
processors.root().unwrapped().forEach((key, value) -> {
- processingConfigMap.put(key, checkProcessorConfig(key, (Map<String, Object>) value, processors));
+ processingConfigMap.put(key, checkConfig(key, (Map<String, Object>) value, processors));
});
}
return processingConfigMap;
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/SplitExecutor.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/SplitExecutor.java
index e549087..7fe93b5 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/SplitExecutor.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/execution/SplitExecutor.java
@@ -10,6 +10,8 @@ import com.geedgenetworks.common.config.SplitConfigOptions;
import com.geedgenetworks.common.exception.CommonErrorCode;
import com.geedgenetworks.common.exception.ConfigValidationException;
import com.geedgenetworks.common.udf.RuleContext;
+import com.geedgenetworks.core.filter.Filter;
+import com.geedgenetworks.core.pojo.FilterConfig;
import com.geedgenetworks.core.pojo.SplitConfig;
import com.geedgenetworks.core.split.Split;
import com.google.common.collect.Maps;
@@ -20,6 +22,7 @@ import org.apache.flink.streaming.api.datastream.DataStream;
import java.net.URL;
import java.util.List;
import java.util.Map;
+import java.util.ServiceLoader;
/**
@@ -58,31 +61,48 @@ public class SplitExecutor extends AbstractExecutor<String, SplitConfig> {
@Override
public DataStream<Event> execute(DataStream<Event> dataStream, Node node) throws JobExecuteException {
SplitConfig splitConfig = operatorMap.get(node.getName());
- String className = splitConfig.getType();
- Split split;
- if (splitMap.containsKey(splitConfig.getType())) {
-
- split = splitMap.get(splitConfig.getType());
- } else {
- Class cls;
- try {
- cls = Class.forName(className);
- split = (Split) cls.newInstance();
- } catch (ClassNotFoundException | InstantiationException | IllegalAccessException | RuntimeException e) {
- throw new JobExecuteException("get split instance failed!", e);
+ boolean found = false; // 标志变量
+ ServiceLoader<Split> splits = ServiceLoader.load(Split.class);
+ for (Split split : splits) {
+ found = true; // 标志变量
+ if(split.type().equals(splitConfig.getType())){
+ if (node.getParallelism() > 0) {
+ splitConfig.setParallelism(node.getParallelism());
+ }
+ try {
+ dataStream =
+ split.splitFunction(
+ dataStream, splitConfig);
+ } catch (Exception e) {
+ throw new JobExecuteException("Create split instance failed!", e);
+ }
+ break;
}
}
- if (node.getParallelism() > 0) {
- splitConfig.setParallelism(node.getParallelism());
- }
- try {
- dataStream =
- split.splitFunction(
- dataStream, splitConfig);
- } catch (Exception e) {
- throw new JobExecuteException("Create split instance failed!", e);
+ if (!found) {
+ throw new JobExecuteException("No matching split found for type: " + splitConfig.getType());
}
return dataStream;
}
+ protected SplitConfig checkConfig(String key, Map<String, Object> value, Config config) {
+ SplitConfig splitConfig = new SplitConfig();
+ boolean found = false; // 标志变量
+ ServiceLoader<Split> splits = ServiceLoader.load(Split.class);
+ for (Split split : splits) {
+ if(split.type().equals(value.getOrDefault("type", "").toString())){
+ found = true;
+ try {
+ splitConfig = split.checkConfig(key, value, config);
+ } catch (Exception e) {
+ throw new JobExecuteException("Create split pipeline instance failed!", e);
+ }
+ break;
+ }
+ }
+ if (!found) {
+ throw new JobExecuteException("No matching split found for type: " + value.getOrDefault("type", "").toString());
+ }
+ return splitConfig;
+ }
}
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/utils/ConfigBuilder.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/utils/ConfigBuilder.java
index 954c058..a83506d 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/utils/ConfigBuilder.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/utils/ConfigBuilder.java
@@ -25,7 +25,7 @@ public class ConfigBuilder {
Map<String, Object> configMap = YamlUtil.loadByPath(filePath.toString());
ConfigObject configObject = ConfigValueFactory.fromMap(configMap);
Config config = configObject.toConfig();
- return ConfigShadeUtils.decryptConfig(config);
+ return CryptoShadeUtils.decryptConfig(config);
}
public static Config of(@NonNull Map<String, Object> objectMap) {
@@ -36,7 +36,7 @@ public class ConfigBuilder {
ConfigObject configObject = ConfigValueFactory.fromMap(objectMap);
Config config = configObject.toConfig();
if (needDecrypt) {
- return ConfigShadeUtils.decryptConfig(config);
+ return CryptoShadeUtils.decryptConfig(config);
}
return config;
}
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/utils/ConfigShadeUtils.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/utils/CryptoShadeUtils.java
index 98db59c..94dda4d 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/utils/ConfigShadeUtils.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/utils/CryptoShadeUtils.java
@@ -3,7 +3,7 @@ package com.geedgenetworks.bootstrap.utils;
import com.alibaba.fastjson2.JSON;
import com.alibaba.fastjson2.JSONObject;
import com.geedgenetworks.common.Constants;
-import com.geedgenetworks.common.config.ConfigShade;
+import com.geedgenetworks.common.crypto.CryptoShade;
import com.geedgenetworks.common.config.TypesafeConfigUtils;
import com.google.common.base.Preconditions;
import com.typesafe.config.*;
@@ -11,38 +11,38 @@ import lombok.extern.slf4j.Slf4j;
import java.util.*;
import java.util.function.BiFunction;
-/** Config shade utilities */
+/** Crypto shade utilities */
@Slf4j
-public final class ConfigShadeUtils {
+public final class CryptoShadeUtils {
private static final String SHADE_IDENTIFIER_OPTION = "shade.identifier";
private static final String[] DEFAULT_SENSITIVE_OPTIONS =
new String[] {"password", "username", "auth"};
- private static final Map<String, ConfigShade> CONFIG_SHADES = new HashMap<>();
+ private static final Map<String, CryptoShade> CRYPTO_SHADES = new HashMap<>();
- private static final ConfigShade DEFAULT_SHADE = new DefaultConfigShade();
+ private static final CryptoShade DEFAULT_SHADE = new DefaultCryptoShade();
static {
- ServiceLoader<ConfigShade> serviceLoader = ServiceLoader.load(ConfigShade.class);
- Iterator<ConfigShade> it = serviceLoader.iterator();
+ ServiceLoader<CryptoShade> serviceLoader = ServiceLoader.load(CryptoShade.class);
+ Iterator<CryptoShade> it = serviceLoader.iterator();
it.forEachRemaining(
configShade -> {
- CONFIG_SHADES.put(configShade.getIdentifier(), configShade);
+ CRYPTO_SHADES.put(configShade.getIdentifier(), configShade);
});
- log.info("Load config shade: {}", CONFIG_SHADES.keySet());
+ log.info("Load config shade: {}", CRYPTO_SHADES.keySet());
}
public static String encryptOption(String identifier, String content) {
- ConfigShade configShade = CONFIG_SHADES.getOrDefault(identifier, DEFAULT_SHADE);
- return configShade.encrypt(content);
+ CryptoShade cryptoShade = CRYPTO_SHADES.getOrDefault(identifier, DEFAULT_SHADE);
+ return cryptoShade.encrypt(content);
}
public static String decryptOption(String identifier, String content) {
- ConfigShade configShade = CONFIG_SHADES.getOrDefault(identifier, DEFAULT_SHADE);
- return configShade.decrypt(content);
+ CryptoShade cryptoShade = CRYPTO_SHADES.getOrDefault(identifier, DEFAULT_SHADE);
+ return cryptoShade.decrypt(content);
}
public static Config decryptConfig(Config config) {
@@ -77,15 +77,15 @@ public final class ConfigShadeUtils {
@SuppressWarnings("unchecked")
private static Config processConfig(String identifier, Config config, boolean isDecrypted) {
- ConfigShade configShade = CONFIG_SHADES.getOrDefault(identifier, DEFAULT_SHADE);
+ CryptoShade cryptoShade = CRYPTO_SHADES.getOrDefault(identifier, DEFAULT_SHADE);
List<String> sensitiveOptions = new ArrayList<>(Arrays.asList(DEFAULT_SENSITIVE_OPTIONS));
- sensitiveOptions.addAll(Arrays.asList(configShade.sensitiveOptions()));
+ sensitiveOptions.addAll(Arrays.asList(cryptoShade.sensitiveOptions()));
BiFunction<String, Object, String> processFunction =
(key, value) -> {
if (isDecrypted) {
- return configShade.decrypt(value.toString());
+ return cryptoShade.decrypt(value.toString());
} else {
- return configShade.encrypt(value.toString());
+ return cryptoShade.encrypt(value.toString());
}
};
String jsonString = config.root().render(ConfigRenderOptions.concise());
@@ -116,7 +116,7 @@ public final class ConfigShadeUtils {
}
- private static class DefaultConfigShade implements ConfigShade {
+ private static class DefaultCryptoShade implements CryptoShade {
private static final String IDENTIFIER = "default";
@Override
diff --git a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/utils/EnvironmentUtil.java b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/utils/EnvironmentUtil.java
index 13db3d4..8028608 100644
--- a/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/utils/EnvironmentUtil.java
+++ b/groot-bootstrap/src/main/java/com/geedgenetworks/bootstrap/utils/EnvironmentUtil.java
@@ -1,8 +1,10 @@
package com.geedgenetworks.bootstrap.utils;
import com.geedgenetworks.bootstrap.execution.ExecutionConfigKeyName;
+import com.geedgenetworks.common.Constants;
import com.geedgenetworks.common.config.CheckResult;
import com.typesafe.config.Config;
+import com.typesafe.config.ConfigUtil;
import com.typesafe.config.ConfigValue;
import lombok.extern.slf4j.Slf4j;
import org.apache.flink.api.common.ExecutionConfig;
@@ -16,7 +18,7 @@ import java.util.concurrent.TimeUnit;
@Slf4j
public final class EnvironmentUtil {
- private EnvironmentUtil() {
+ private EnvironmentUtil() {
throw new UnsupportedOperationException("EnvironmentUtil is a utility class and cannot be instantiated");
}
@@ -30,10 +32,13 @@ public final class EnvironmentUtil {
configuration.setString(
PipelineOptions.CLASSPATHS.key(), pipeline.getString("classpaths"));
}
- if(pipeline.hasPath("object-reuse")) {
+ if (pipeline.hasPath("object-reuse")) {
configuration.setBoolean(PipelineOptions.OBJECT_REUSE.key(), pipeline.getBoolean("object-reuse"));
}
}
+ if (envConfig.hasPath(ConfigUtil.joinPath(Constants.SYSPROP_KMS_TYPE_CONFIG))) {
+ configuration.setString(Constants.SYSPROP_KMS_TYPE_CONFIG, envConfig.getString(ConfigUtil.joinPath(Constants.SYSPROP_KMS_TYPE_CONFIG)));
+ }
String prefixConf = "flink.";
if (!envConfig.isEmpty()) {
for (Map.Entry<String, ConfigValue> entryConfKey : envConfig.entrySet()) {
@@ -117,5 +122,4 @@ public final class EnvironmentUtil {
}
-
}
diff --git a/groot-bootstrap/src/main/resources/META-INF/services/com.geedgenetworks.common.config.ConfigShade b/groot-bootstrap/src/main/resources/META-INF/services/com.geedgenetworks.common.config.ConfigShade
deleted file mode 100644
index f490f28..0000000
--- a/groot-bootstrap/src/main/resources/META-INF/services/com.geedgenetworks.common.config.ConfigShade
+++ /dev/null
@@ -1,3 +0,0 @@
-com.geedgenetworks.bootstrap.command.Base64ConfigShade
-com.geedgenetworks.bootstrap.command.AESConfigShade
-com.geedgenetworks.bootstrap.command.SM4ConfigShade \ No newline at end of file
diff --git a/groot-bootstrap/src/main/resources/META-INF/services/com.geedgenetworks.common.crypto.CryptoShade b/groot-bootstrap/src/main/resources/META-INF/services/com.geedgenetworks.common.crypto.CryptoShade
new file mode 100644
index 0000000..273b40d
--- /dev/null
+++ b/groot-bootstrap/src/main/resources/META-INF/services/com.geedgenetworks.common.crypto.CryptoShade
@@ -0,0 +1,6 @@
+com.geedgenetworks.bootstrap.command.Base64Shade
+com.geedgenetworks.bootstrap.command.AESShade
+com.geedgenetworks.bootstrap.command.SM4Shade
+com.geedgenetworks.bootstrap.command.AES128GCM96Shade
+com.geedgenetworks.bootstrap.command.AES256GCM96Shade
+com.geedgenetworks.bootstrap.command.SM4GCM96Shade \ No newline at end of file
diff --git a/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/GrootStreamServerTest.java b/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/GrootStreamServerTest.java
deleted file mode 100644
index d7ed524..0000000
--- a/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/GrootStreamServerTest.java
+++ /dev/null
@@ -1,54 +0,0 @@
-package com.geedgenetworks.bootstrap.main.simple;
-
-import com.geedgenetworks.bootstrap.command.Command;
-import com.geedgenetworks.bootstrap.command.CommandArgs;
-import com.geedgenetworks.bootstrap.command.ExecuteCommandArgs;
-import com.geedgenetworks.bootstrap.enums.EngineType;
-import com.geedgenetworks.bootstrap.exception.ConfigCheckException;
-import com.geedgenetworks.bootstrap.exception.JobExecuteException;
-import com.geedgenetworks.bootstrap.utils.CommandLineUtils;
-import lombok.extern.slf4j.Slf4j;
-import org.apache.commons.lang3.exception.ExceptionUtils;
-
-@Slf4j
-public class GrootStreamServerTest {
- public static void main(String[] args) {
- ExecuteCommandArgs bootstrapCommandArgs = CommandLineUtils
- .parse(args, new ExecuteCommandArgs(), EngineType.FLINK13.getShellName(), true);
- run(bootstrapCommandArgs.buildCommand());
- }
-
- public static <T extends CommandArgs> void run(Command<T> command) throws JobExecuteException {
- try {
- command.execute();
- } catch (ConfigCheckException e) {
- outputConfigError(e);
- throw e;
- } catch (Exception e) {
- outputFatalError(e);
- throw e;
- }
- }
- private static void outputConfigError(Throwable throwable) {
- log.error(
- "\n\n===============================================================================\n\n");
- String errorMsg = throwable.getMessage();
- log.error("Config Error:\n");
- log.error("Reason: {} \n", errorMsg);
- log.error(
- "\n===============================================================================\n\n\n");
- }
-
-
- private static void outputFatalError(Throwable throwable) {
- log.error("\\n\\n===============================================================================\\n\\n");
- String errorMsg = throwable.getMessage();
- log.error("Fatal Error ,Reason is :{} \n", errorMsg);
- log.error("Exception StackTrace :{}", ExceptionUtils.getStackTrace(throwable));
- log.error("\\n\\n===============================================================================\\n\\n");
- }
-
-
-
-
-}
diff --git a/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobSplitWithAggTest.java b/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobAggTest.java
index 9fa81c0..e33998c 100644
--- a/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobSplitWithAggTest.java
+++ b/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobAggTest.java
@@ -3,8 +3,8 @@ package com.geedgenetworks.bootstrap.main.simple;
import cn.hutool.setting.yaml.YamlUtil;
import com.geedgenetworks.bootstrap.command.ExecuteCommandArgs;
import com.geedgenetworks.bootstrap.enums.EngineType;
-import com.geedgenetworks.bootstrap.exception.JobExecuteException;
import com.geedgenetworks.bootstrap.execution.ExecutionConfigKeyName;
+import com.geedgenetworks.bootstrap.execution.JobExecution;
import com.geedgenetworks.bootstrap.main.simple.collect.CollectSink;
import com.geedgenetworks.bootstrap.utils.CommandLineUtils;
import com.geedgenetworks.bootstrap.utils.ConfigFileUtils;
@@ -22,10 +22,12 @@ import org.junit.ClassRule;
import org.junit.Test;
import java.nio.file.Path;
+import java.util.List;
import java.util.Map;
+import java.util.Set;
-public class JobSplitWithAggTest {
+public class JobAggTest {
@ClassRule
public static MiniClusterWithClientResource flinkCluster =
@@ -39,7 +41,7 @@ public class JobSplitWithAggTest {
public void testSplitForAgg() {
CollectSink.values.clear();
- String[] args ={"--target", "remote", "-c", ".\\grootstream_job_split_agg_test.yaml"};
+ String[] args ={"--target", "test", "-c", ".\\grootstream_job_agg_test.yaml"};
ExecuteCommandArgs executeCommandArgs = CommandLineUtils
.parse(args, new ExecuteCommandArgs(), EngineType.FLINK13.getShellName(), true);
@@ -57,18 +59,21 @@ public class JobSplitWithAggTest {
ConfigValueFactory.fromAnyRef(executeCommandArgs.getTargetType().getTarget()));
- JobExecutionTest jobExecution = new JobExecutionTest(config, grootStreamConfig);
- jobExecution.getSingleOutputStreamOperator();
-
- try {
- jobExecution.getJobRuntimeEnvironment().getStreamExecutionEnvironment().execute();
- } catch (Exception e) {
- throw new JobExecuteException("Job executed error", e);
- }
-
- Assert.assertEquals(2, CollectSink.values.size());
- Assert.assertEquals("3", CollectSink.values.get(1).getExtractedFields().get("sessions").toString());
- Assert.assertEquals("3.0", CollectSink.values.get(1).getExtractedFields().get("pkts").toString());
+ JobExecution jobExecution = new JobExecution(config, grootStreamConfig);
+ jobExecution.execute();
+
+ Assert.assertEquals(4, CollectSink.values.size());
+ Assert.assertEquals("2", CollectSink.values.get(1).getExtractedFields().get("sessions").toString());
+ Assert.assertEquals("3.5", CollectSink.values.get(1).getExtractedFields().get("pkts").toString());
+ Assert.assertEquals("2", CollectSink.values.get(1).getExtractedFields().get("log_id_first").toString());
+ Assert.assertEquals("1", CollectSink.values.get(1).getExtractedFields().get("log_id_last").toString());
+ Assert.assertEquals("4", CollectSink.values.get(1).getExtractedFields().get("pktsmax").toString());
+ Assert.assertEquals("3", CollectSink.values.get(1).getExtractedFields().get("pktsmin").toString());
+ List<String> list = (List<String>) CollectSink.values.get(1).getExtractedFields().get("client_ip_list");
+ Set<String> set = (Set<String>) CollectSink.values.get(1).getExtractedFields().get("server_ip_set");
+ Assert.assertEquals(1, set.size());
+ Assert.assertEquals(2, list.size());
+ Assert.assertEquals("2", CollectSink.values.get(1).getExtractedFields().get("count").toString());
}
}
diff --git a/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobDosTest.java b/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobDosTest.java
new file mode 100644
index 0000000..ea3793e
--- /dev/null
+++ b/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobDosTest.java
@@ -0,0 +1,93 @@
+package com.geedgenetworks.bootstrap.main.simple;
+
+import cn.hutool.setting.yaml.YamlUtil;
+import com.geedgenetworks.bootstrap.command.ExecuteCommandArgs;
+import com.geedgenetworks.bootstrap.enums.EngineType;
+import com.geedgenetworks.bootstrap.execution.ExecutionConfigKeyName;
+import com.geedgenetworks.bootstrap.execution.JobExecution;
+import com.geedgenetworks.bootstrap.main.simple.collect.CollectSink;
+import com.geedgenetworks.bootstrap.utils.CommandLineUtils;
+import com.geedgenetworks.bootstrap.utils.ConfigFileUtils;
+import com.geedgenetworks.common.Constants;
+import com.geedgenetworks.common.config.ConfigProvider;
+import com.geedgenetworks.common.config.GrootStreamConfig;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigObject;
+import com.typesafe.config.ConfigUtil;
+import com.typesafe.config.ConfigValueFactory;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.test.util.MiniClusterWithClientResource;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.nio.file.Path;
+import java.util.Map;
+
+
+public class JobDosTest {
+
+ @ClassRule
+ public static MiniClusterWithClientResource flinkCluster =
+ new MiniClusterWithClientResource(
+ new MiniClusterResourceConfiguration.Builder()
+ .setNumberSlotsPerTaskManager(1)
+ .setNumberTaskManagers(1)
+ .build());
+
+ @Test
+ public void testSplit() {
+
+ CollectSink.values.clear();
+ String[] args ={"--target", "test", "-c", ".\\grootstream_job_dos_test.yaml"};
+ ExecuteCommandArgs executeCommandArgs = CommandLineUtils
+ .parse(args, new ExecuteCommandArgs(), EngineType.FLINK13.getShellName(), true);
+
+ executeCommandArgs.buildCommand();
+
+
+ GrootStreamConfig grootStreamConfig = ConfigProvider.locateAndGetGrootStreamConfig();
+ Path configFile = ConfigFileUtils.getConfigPath(executeCommandArgs);
+ // check config file exist
+ Map<String, Object> configMap = YamlUtil.loadByPath(configFile.toString());
+ ConfigObject configObject = ConfigValueFactory.fromMap(configMap);
+ Config config = configObject.toConfig();
+
+ config = config.withValue(ConfigUtil.joinPath(Constants.APPLICATION, Constants.APPLICATION_ENV, ExecutionConfigKeyName.ENV_TARGET_TYPE),
+ ConfigValueFactory.fromAnyRef(executeCommandArgs.getTargetType().getTarget()));
+
+
+ JobExecution jobExecution = new JobExecution(config, grootStreamConfig);
+ jobExecution.execute();
+ // Assert.assertEquals(7, CollectSink.values.size());
+
+
+ Assert.assertEquals(3, CollectSink.values.size());
+ Assert.assertEquals("200", CollectSink.values.get(1).getExtractedFields().get("sessions").toString());
+ Assert.assertEquals("2000", CollectSink.values.get(1).getExtractedFields().get("packets").toString());
+ Assert.assertEquals("20000", CollectSink.values.get(1).getExtractedFields().get("bytes").toString());
+ Assert.assertEquals("66.67", CollectSink.values.get(1).getExtractedFields().get("session_rate").toString());
+ Assert.assertEquals("666.67", CollectSink.values.get(1).getExtractedFields().get("packet_rate").toString());
+ Assert.assertEquals("53333.33", CollectSink.values.get(1).getExtractedFields().get("bit_rate").toString());
+
+ Assert.assertTrue( CollectSink.values.get(1).getExtractedFields().containsKey("log_id"));
+ Assert.assertTrue(CollectSink.values.get(1).getExtractedFields().containsKey("recv_time"));
+ Assert.assertEquals("1729476003", CollectSink.values.get(1).getExtractedFields().get("end_time").toString());
+ Assert.assertEquals("1729476000", CollectSink.values.get(1).getExtractedFields().get("start_time").toString());
+ Assert.assertEquals("1729476003", CollectSink.values.get(1).getExtractedFields().get("end_time").toString());
+ Assert.assertEquals("3", CollectSink.values.get(1).getExtractedFields().get("duration").toString());
+
+
+
+ Assert.assertEquals("1.2.2.2", CollectSink.values.get(1).getExtractedFields().get("destination_ip").toString());
+ Assert.assertEquals("1.1.1.1", CollectSink.values.get(1).getExtractedFields().get("source_ip").toString());
+
+ Assert.assertEquals("CN", CollectSink.values.get(1).getExtractedFields().get("source_country").toString());
+ Assert.assertEquals("1.1.1.1", CollectSink.values.get(1).getExtractedFields().get("source_ip").toString());
+ Assert.assertEquals("1.2.2.2", CollectSink.values.get(1).getExtractedFields().get("destination_ip").toString());
+ Assert.assertEquals("US", CollectSink.values.get(1).getExtractedFields().get("destination_country").toString());
+ Assert.assertEquals("123", CollectSink.values.get(1).getExtractedFields().get("rule_uuid").toString());
+
+ }
+
+}
diff --git a/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/SimpleJobTest.java b/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobEtlTest.java
index 90ff95d..80b7129 100644
--- a/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/SimpleJobTest.java
+++ b/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobEtlTest.java
@@ -3,8 +3,8 @@ package com.geedgenetworks.bootstrap.main.simple;
import cn.hutool.setting.yaml.YamlUtil;
import com.geedgenetworks.bootstrap.command.ExecuteCommandArgs;
import com.geedgenetworks.bootstrap.enums.EngineType;
-import com.geedgenetworks.bootstrap.exception.JobExecuteException;
import com.geedgenetworks.bootstrap.execution.ExecutionConfigKeyName;
+import com.geedgenetworks.bootstrap.execution.JobExecution;
import com.geedgenetworks.bootstrap.main.simple.collect.CollectSink;
import com.geedgenetworks.bootstrap.utils.CommandLineUtils;
import com.geedgenetworks.bootstrap.utils.ConfigFileUtils;
@@ -22,14 +22,13 @@ import org.junit.ClassRule;
import org.junit.Test;
import java.nio.file.Path;
-import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertTrue;
import static org.junit.jupiter.api.Assertions.assertEquals;
-public class SimpleJobTest {
+public class JobEtlTest {
@ClassRule
public static MiniClusterWithClientResource flinkCluster =
@@ -42,7 +41,7 @@ public class SimpleJobTest {
@Test
public void testEtl() {
- String[] args ={"--target", "remote", "-c", ".\\grootstream_job_etl_test.yaml"};
+ String[] args ={"--target", "test", "-c", ".\\grootstream_job_etl_test.yaml"};
ExecuteCommandArgs executeCommandArgs = CommandLineUtils
.parse(args, new ExecuteCommandArgs(), EngineType.FLINK13.getShellName(), true);
@@ -60,15 +59,8 @@ public class SimpleJobTest {
ConfigValueFactory.fromAnyRef(executeCommandArgs.getTargetType().getTarget()));
- JobExecutionTest jobExecution = new JobExecutionTest(config, grootStreamConfig);
- jobExecution.getSingleOutputStreamOperator();
-
- try {
- jobExecution.getJobRuntimeEnvironment().getStreamExecutionEnvironment().execute();
-
- } catch (Exception e) {
- throw new JobExecuteException("Job executed error", e);
- }
+ JobExecution jobExecution = new JobExecution(config, grootStreamConfig);
+ jobExecution.execute();
Assert.assertEquals(1, CollectSink.values.size());
Assert.assertEquals("BASE", CollectSink.values.get(0).getExtractedFields().get("decoded_as").toString());
Assert.assertEquals("google.com", CollectSink.values.get(0).getExtractedFields().get("server_domain").toString());
@@ -78,7 +70,16 @@ public class SimpleJobTest {
Assert.assertEquals("http://192.168.44.12:9098/hos/traffic_policy_capture_file_bucket/test", CollectSink.values.get(0).getExtractedFields().get("packet_capture_file").toString());
Assert.assertEquals("[2600:1015:b002::,255.255.255.255]", CollectSink.values.get(0).getExtractedFields().get("ip_string").toString());
Assert.assertEquals("hello", CollectSink.values.get(0).getExtractedFields().get("mail_attachment_name").toString());
+ Assert.assertEquals("aGVsbG8=", CollectSink.values.get(0).getExtractedFields().get("old_mail_attachment_name").toString());
+
Assert.assertEquals("MULTIPATH_ETHERNET", CollectSink.values.get(0).getExtractedFields().get("tunnels_schema_type").toString());
+ Assert.assertEquals(36, CollectSink.values.get(0).getExtractedFields().get("log_uuid").toString().length());
+ Assert.assertEquals(36, CollectSink.values.get(0).getExtractedFields().get("log_uuid_v7").toString().length());
+ Assert.assertEquals("dacad383-8355-5493-9e1e-20ef5cd8b8fd", CollectSink.values.get(0).getExtractedFields().get("ip_uuid").toString());
+
+ Assert.assertEquals("2024-01-18 17:01:57.095", CollectSink.values.get(0).getExtractedFields().get("start_time").toString());
+
+
}
@@ -86,7 +87,7 @@ public class SimpleJobTest {
public void testTransmission() {
CollectSink.values.clear();
- String[] args ={"--target", "remote", "-c", ".\\grootstream_job_transmission_test.yaml"};
+ String[] args ={"--target", "test", "-c", ".\\grootstream_job_transmission_test.yaml"};
ExecuteCommandArgs executeCommandArgs = CommandLineUtils
.parse(args, new ExecuteCommandArgs(), EngineType.FLINK13.getShellName(), true);
@@ -104,15 +105,8 @@ public class SimpleJobTest {
ConfigValueFactory.fromAnyRef(executeCommandArgs.getTargetType().getTarget()));
- JobExecutionTest jobExecution = new JobExecutionTest(config, grootStreamConfig);
- jobExecution.getSingleOutputStreamOperator();
-
- try {
- jobExecution.getJobRuntimeEnvironment().getStreamExecutionEnvironment().execute();
-
- } catch (Exception e) {
- throw new JobExecuteException("Job executed error", e);
- }
+ JobExecution jobExecution = new JobExecution(config, grootStreamConfig);
+ jobExecution.execute();
Assert.assertEquals(4, CollectSink.values.size());
}
diff --git a/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobExecutionTest.java b/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobExecutionTest.java
deleted file mode 100644
index 7b9544a..0000000
--- a/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobExecutionTest.java
+++ /dev/null
@@ -1,327 +0,0 @@
-package com.geedgenetworks.bootstrap.main.simple;
-
-import com.alibaba.fastjson2.JSONObject;
-import com.geedgenetworks.bootstrap.enums.ProcessorType;
-import com.geedgenetworks.bootstrap.exception.JobExecuteException;
-import com.geedgenetworks.bootstrap.execution.*;
-import com.geedgenetworks.bootstrap.main.GrootStreamRunner;
-import com.geedgenetworks.common.Constants;
-import com.geedgenetworks.common.config.GrootStreamConfig;
-import com.geedgenetworks.common.udf.RuleContext;
-import com.geedgenetworks.common.utils.ReflectionUtils;
-import com.geedgenetworks.common.Event;
-import com.geedgenetworks.core.pojo.SplitConfig;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.typesafe.config.Config;
-import com.typesafe.config.ConfigUtil;
-import com.typesafe.config.ConfigValueFactory;
-import lombok.Data;
-import lombok.extern.slf4j.Slf4j;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.util.OutputTag;
-
-import java.io.File;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.nio.file.Path;
-import java.util.*;
-import java.util.function.BiConsumer;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-@Slf4j
-@Data
-public class JobExecutionTest {
-
- protected final JobRuntimeEnvironment jobRuntimeEnvironment;
- private final Executor<DataStream<Event>, JobRuntimeEnvironment> sourceExecutor;
- private final Executor<DataStream<Event>, JobRuntimeEnvironment> filterExecutor;
- private final Executor<DataStream<Event>, JobRuntimeEnvironment> preprocessingExecutor;
- private final Executor<DataStream<Event>, JobRuntimeEnvironment> splitExecutor;
- private final Executor<DataStream<Event>, JobRuntimeEnvironment> processingExecutor;
- private final Executor<DataStream<Event>, JobRuntimeEnvironment> postprocessingExecutor;
- private final Executor<DataStream<Event>, JobRuntimeEnvironment> sinkExecutor;
- private final Set<String> splitSet = new HashSet<>();
- private final List<Node> nodes;
-
- private BiConsumer<ClassLoader, URL> ADD_URL_TO_CLASSLOADER =
- (classLoader, url) -> {
- if (classLoader.getClass().getName().endsWith("SafetyNetWrapperClassLoader")) {
- URLClassLoader c =
- (URLClassLoader) ReflectionUtils.getField(classLoader, "inner").get();
- ReflectionUtils.invoke(c, "addURL", url);
- } else if (classLoader instanceof URLClassLoader) {
- ReflectionUtils.invoke(classLoader, "addURL", url);
- } else {
- throw new RuntimeException(
- "Unsupported classloader: " + classLoader.getClass().getName());
- }
- };
- private final List<URL> jarPaths;
- public JobExecutionTest(Config config, GrootStreamConfig grootStreamConfig) {
- try {
- jarPaths = new ArrayList<>(Collections.singletonList(new File(StartBuilder.appBootstrapDir()
- .resolve(GrootStreamRunner.APP_JAR_NAME).toString())
- .toURI().toURL()));
- } catch (MalformedURLException e) {
- throw new JobExecuteException("load groot stream bootstrap jar error.", e);
- }
- registerPlugin(config.getConfig(Constants.APPLICATION));
-
- this.sourceExecutor = new SourceExecutor(jarPaths, config);
- this.sinkExecutor = new SinkExecutor(jarPaths, config);
- this.splitExecutor = new SplitExecutor(jarPaths, config);
- this.filterExecutor = new FilterExecutor(jarPaths, config);
- this.preprocessingExecutor = new PreprocessingExecutor(jarPaths, config);
- this.processingExecutor = new ProcessingExecutor(jarPaths, config);
- this.postprocessingExecutor = new PostprocessingExecutor(jarPaths, config);
- this.jobRuntimeEnvironment =
- JobRuntimeEnvironment.getInstance(this.registerPlugin(config, jarPaths), grootStreamConfig);
- this.splitExecutor.setRuntimeEnvironment(jobRuntimeEnvironment);
- this.sourceExecutor.setRuntimeEnvironment(jobRuntimeEnvironment);
- this.sinkExecutor.setRuntimeEnvironment(jobRuntimeEnvironment);
- this.filterExecutor.setRuntimeEnvironment(jobRuntimeEnvironment);
- this.preprocessingExecutor.setRuntimeEnvironment(jobRuntimeEnvironment);
- this.processingExecutor.setRuntimeEnvironment(jobRuntimeEnvironment);
- this.postprocessingExecutor.setRuntimeEnvironment(jobRuntimeEnvironment);
- this.nodes = buildJobNode(config);
-
- }
-
- private void registerPlugin(Config appConfig) {
- List<Path> thirdPartyJars = new ArrayList<>();
- Config envConfig = appConfig.getConfig(Constants.APPLICATION_ENV);
- if(envConfig.hasPath(ExecutionConfigKeyName.JARS)) {
- thirdPartyJars = new ArrayList<>(StartBuilder
- .getThirdPartyJars(envConfig.getString(ExecutionConfigKeyName.JARS)));
- }
- thirdPartyJars.addAll(StartBuilder.getConnectorJars());
- thirdPartyJars.addAll(StartBuilder.getPluginsJarDependencies());
-
- List<URL> jarDependencies = Stream.concat(thirdPartyJars.stream(), StartBuilder.getLibJars().stream())
- .map(Path::toUri)
- .map(uri -> {
- try {
- return uri.toURL();
- }catch (MalformedURLException e){
- throw new RuntimeException("the uri of jar illegal: " + uri, e);
- }
- })
- .collect(Collectors.toList());
- jarDependencies.forEach(url -> {
- ADD_URL_TO_CLASSLOADER.accept(Thread.currentThread().getContextClassLoader(), url);
- });
- jarPaths.addAll(jarDependencies);
-
- }
-
-
- private Config registerPlugin(Config config , List<URL> jars) {
- config = this.injectJarsToConfig(
- config, ConfigUtil.joinPath(Constants.APPLICATION, Constants.APPLICATION_ENV, ExecutionConfigKeyName.PIPELINE_JARS), jars);
- return this.injectJarsToConfig(
- config, ConfigUtil.joinPath(Constants.APPLICATION, Constants.APPLICATION_ENV, ExecutionConfigKeyName.PIPELINE_CLASSPATHS), jars);
- }
-
-
- private Config injectJarsToConfig(Config config, String path, List<URL> jars) {
- List<URL> validJars = new ArrayList<>();
- for (URL jarUrl : jars) {
- if (new File(jarUrl.getFile()).exists()) {
- validJars.add(jarUrl);
- log.info("Inject jar to config: {}", jarUrl);
- } else {
- log.warn("Remove invalid jar when inject jars into config: {}", jarUrl);
- }
- }
-
- if (config.hasPath(path)) {
- Set<URL> paths =
- Arrays.stream(config.getString(path).split(";"))
- .map(
- uri -> {
- try {
- return new URL(uri);
- } catch (MalformedURLException e) {
- throw new RuntimeException(
- "the uri of jar illegal:" + uri, e);
- }
- })
- .collect(Collectors.toSet());
- paths.addAll(validJars);
-
- config = config.withValue(
- path,
- ConfigValueFactory.fromAnyRef(
- paths.stream()
- .map(URL::toString)
- .distinct()
- .collect(Collectors.joining(";"))));
- } else {
- config =
- config.withValue(
- path,
- ConfigValueFactory.fromAnyRef(
- validJars.stream()
- .map(URL::toString)
- .distinct()
- .collect(Collectors.joining(";"))));
- }
- return config;
- }
-
- private List<Node> buildJobNode(Config config) {
-
-
- Map<String, Object> sources = Maps.newHashMap();
- Map<String, Object> sinks =Maps.newHashMap();
- Map<String, Object> filters = Maps.newHashMap();
- Map<String, Object> splits = Maps.newHashMap();
- Map<String, Object> preprocessingPipelines = Maps.newHashMap();
- Map<String, Object> processingPipelines = Maps.newHashMap();
- Map<String, Object> postprocessingPipelines = Maps.newHashMap();
-
- if (config.hasPath(Constants.SOURCES)) {
- sources = config.getConfig(Constants.SOURCES).root().unwrapped();
- }
- if (config.hasPath(Constants.SINKS)) {
- sinks =config.getConfig(Constants.SINKS).root().unwrapped();
- }
- if (config.hasPath(Constants.FILTERS)) {
- filters = config.getConfig(Constants.FILTERS).root().unwrapped();
- }
- if (config.hasPath(Constants.SPLITS)) {
- splits = config.getConfig(Constants.SPLITS).root().unwrapped();
- }
- if (config.hasPath(Constants.PREPROCESSING_PIPELINES)) {
- preprocessingPipelines = config.getConfig(Constants.PREPROCESSING_PIPELINES).root().unwrapped();
- }
- if (config.hasPath(Constants.PROCESSING_PIPELINES)) {
- processingPipelines = config.getConfig(Constants.PROCESSING_PIPELINES).root().unwrapped();
- }
- if (config.hasPath(Constants.POSTPROCESSING_PIPELINES)) {
- postprocessingPipelines = config.getConfig(Constants.POSTPROCESSING_PIPELINES).root().unwrapped();
- }
-
- List<? extends Config> topology = config.getConfig(Constants.APPLICATION).getConfigList(Constants.APPLICATION_TOPOLOGY);
-
- List<Node> nodes = Lists.newArrayList();
-
- topology.forEach(item -> {
- Node node = JSONObject.from(item.root().unwrapped()).toJavaObject(Node.class);
- nodes.add(node);
- });
-
- for(Node node : nodes) {
- if (sources.containsKey(node.getName())) {
- node.setType(ProcessorType.SOURCE);
- } else if (sinks.containsKey(node.getName())) {
- node.setType(ProcessorType.SINK);
- } else if (splits.containsKey(node.getName())) {
- splits.forEach((key, value) -> {
- SplitConfig splitConfig = new com.alibaba.fastjson.JSONObject((Map<String, Object>) value).toJavaObject(SplitConfig.class);
- for(RuleContext ruleContext:splitConfig.getRules()) {
- splitSet.add(ruleContext.getName());
- }
- });
- node.setType(ProcessorType.SPLIT);
- } else if (filters.containsKey(node.getName())) {
- node.setType(ProcessorType.FILTER);
- } else if (preprocessingPipelines.containsKey(node.getName())) {
- node.setType(ProcessorType.PREPROCESSING);
- } else if (processingPipelines.containsKey(node.getName())) {
- node.setType(ProcessorType.PROCESSING);
- } else if (postprocessingPipelines.containsKey(node.getName())) {
- node.setType(ProcessorType.POSTPROCESSING);
- } else {
- throw new JobExecuteException("unsupported process type " + node.getName());
- }
- }
-
- return nodes;
-
- }
-
-
- public DataStream<Event> getSingleOutputStreamOperator() throws JobExecuteException {
-
- List<Node> sourceNodes = nodes
- .stream().filter(v -> v.getType().name().equals(ProcessorType.SOURCE.name())).collect(Collectors.toList());
-
- DataStream<Event> singleOutputStreamOperator = null;
-
- for(Node sourceNode : sourceNodes) {
- singleOutputStreamOperator = sourceExecutor.execute(singleOutputStreamOperator, sourceNode);
- for (String nodeName : sourceNode.getDownstream()) {
- buildJobGraph(singleOutputStreamOperator, nodeName);
- }
- }
-
- return singleOutputStreamOperator;
-
-
- }
-
- private void buildJobGraph(DataStream<Event> dataStream, String downstreamNodeName) {
- Node node = getNode(downstreamNodeName).orElseGet(() -> {
- throw new JobExecuteException("can't find downstream node " + downstreamNodeName);
- });
- if (node.getType().name().equals(ProcessorType.FILTER.name())) {
- if (splitSet.contains(node.getName())) {
- dataStream = filterExecutor.execute(((SingleOutputStreamOperator<Event>) dataStream).getSideOutput(new OutputTag<Event>(node.getName()) {
- }), node);
- } else {
- dataStream = filterExecutor.execute(dataStream, node);
- }
- } else if (node.getType().name().equals(ProcessorType.SPLIT.name())) {
- dataStream = splitExecutor.execute(dataStream, node);
-
- } else if (node.getType().name().equals(ProcessorType.PREPROCESSING.name())) {
- if (splitSet.contains(node.getName())) {
- dataStream = preprocessingExecutor.execute(((SingleOutputStreamOperator<Event>) dataStream).getSideOutput(new OutputTag<Event>(node.getName()) {
- }), node);
- } else {
- dataStream = preprocessingExecutor.execute(dataStream, node);
- }
- } else if (node.getType().name().equals(ProcessorType.PROCESSING.name())) {
- if (splitSet.contains(node.getName())) {
- dataStream = processingExecutor.execute(((SingleOutputStreamOperator<Event>) dataStream).getSideOutput(new OutputTag<Event>(node.getName()) {
- }), node);
- } else {
- dataStream = processingExecutor.execute(dataStream, node);
- }
- } else if (node.getType().name().equals(ProcessorType.POSTPROCESSING.name())) {
- if (splitSet.contains(node.getName())) {
- dataStream = postprocessingExecutor.execute(((SingleOutputStreamOperator<Event>) dataStream).getSideOutput(new OutputTag<Event>(node.getName()) {
- }), node);
- } else {
- dataStream = postprocessingExecutor.execute(dataStream, node);
- }
- } else if (node.getType().name().equals(ProcessorType.SINK.name())) {
- if (splitSet.contains(node.getName())) {
- dataStream = sinkExecutor.execute(((SingleOutputStreamOperator<Event>) dataStream).getSideOutput(new OutputTag<Event>(node.getName()) {
- }), node);
- } else {
- dataStream = sinkExecutor.execute(dataStream, node);
- }
- } else {
- throw new JobExecuteException("unsupported process type " + node.getType().name());
- }
-
-
- for (String nodeName : node.getDownstream()) {
- buildJobGraph(dataStream, nodeName);
- }
-
-
- }
-
- private Optional<Node> getNode(String name) {
- return nodes.stream().filter(v-> v.getName().equals(name)).findFirst();
- }
-
-
-}
diff --git a/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobSplitTest.java b/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobSplitTest.java
index 2f6984b..352bad2 100644
--- a/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobSplitTest.java
+++ b/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/main/simple/JobSplitTest.java
@@ -5,6 +5,7 @@ import com.geedgenetworks.bootstrap.command.ExecuteCommandArgs;
import com.geedgenetworks.bootstrap.enums.EngineType;
import com.geedgenetworks.bootstrap.exception.JobExecuteException;
import com.geedgenetworks.bootstrap.execution.ExecutionConfigKeyName;
+import com.geedgenetworks.bootstrap.execution.JobExecution;
import com.geedgenetworks.bootstrap.main.simple.collect.CollectSink;
import com.geedgenetworks.bootstrap.utils.CommandLineUtils;
import com.geedgenetworks.bootstrap.utils.ConfigFileUtils;
@@ -42,7 +43,7 @@ public class JobSplitTest {
public void testSplit() {
CollectSink.values.clear();
- String[] args ={"--target", "remote", "-c", ".\\grootstream_job_split_test.yaml"};
+ String[] args ={"--target", "test", "-c", ".\\grootstream_job_split_test.yaml"};
ExecuteCommandArgs executeCommandArgs = CommandLineUtils
.parse(args, new ExecuteCommandArgs(), EngineType.FLINK13.getShellName(), true);
@@ -60,14 +61,8 @@ public class JobSplitTest {
ConfigValueFactory.fromAnyRef(executeCommandArgs.getTargetType().getTarget()));
- JobExecutionTest jobExecution = new JobExecutionTest(config, grootStreamConfig);
- jobExecution.getSingleOutputStreamOperator();
-
- try {
- jobExecution.getJobRuntimeEnvironment().getStreamExecutionEnvironment().execute();
- } catch (Exception e) {
- throw new JobExecuteException("Job executed error", e);
- }
+ JobExecution jobExecution = new JobExecution(config, grootStreamConfig);
+ jobExecution.execute();
Assert.assertEquals(7, CollectSink.values.size());
}
diff --git a/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/utils/ConfigShadeTest.java b/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/utils/ConfigShadeTest.java
deleted file mode 100644
index 17f56ce..0000000
--- a/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/utils/ConfigShadeTest.java
+++ /dev/null
@@ -1,81 +0,0 @@
-package com.geedgenetworks.bootstrap.utils;
-
-import cn.hutool.setting.yaml.YamlUtil;
-import com.alibaba.fastjson2.JSON;
-import com.alibaba.fastjson2.JSONObject;
-import com.geedgenetworks.common.Constants;
-import com.typesafe.config.Config;
-import com.typesafe.config.ConfigObject;
-import com.typesafe.config.ConfigRenderOptions;
-import com.typesafe.config.ConfigValueFactory;
-import lombok.extern.slf4j.Slf4j;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.nio.file.Paths;
-import java.util.Map;
-
-@Slf4j
-public class ConfigShadeTest {
-
- private static final String USERNAME = "grootstream";
-
- private static final String PASSWORD = "grootstream_password";
-
- @Test
- public void testParseConfig() throws URISyntaxException {
- URL resource = ConfigShadeTest.class.getResource("/inline_to_clickhouse.yaml");
- Assertions.assertNotNull(resource);
- Map<String, Object> configMap = YamlUtil.loadByPath(Paths.get(resource.toURI()).toString());
- ConfigObject configObject = ConfigValueFactory.fromMap(configMap);
- Config decryptConfig = ConfigShadeUtils.decryptConfig(configObject.toConfig());
- String jsonString = decryptConfig.root().render(ConfigRenderOptions.concise());
- JSONObject jsonObject = JSON.parseObject(jsonString);
- JSONObject sinkObject = jsonObject.getJSONObject(Constants.SINKS);
- log.info("Decrypt config: {}", decryptConfig.root().render(ConfigRenderOptions.concise()));
- Assertions.assertEquals(sinkObject.keySet().size(), 1);
- Assertions.assertNotNull(sinkObject);
- Assertions.assertNotNull(sinkObject.getJSONObject("clickhouse_sink"));
- Assertions.assertNotNull(sinkObject.getJSONObject("clickhouse_sink")
- .getJSONObject("properties"));
- Assertions.assertEquals(sinkObject.getJSONObject("clickhouse_sink")
- .getJSONObject("properties").isEmpty(), false);
- Assertions.assertEquals(sinkObject.getJSONObject("clickhouse_sink")
- .getJSONObject("properties").get("connection.user"),USERNAME);
- Assertions.assertNotNull(sinkObject.getJSONObject("clickhouse_sink")
- .getJSONObject("properties").get("connection.password"), PASSWORD);
- }
-
- @Test
- public void testDecryptAndEncrypt() {
- String encryptUsername = ConfigShadeUtils.encryptOption("base64", USERNAME);
- String decryptUsername = ConfigShadeUtils.decryptOption("base64", encryptUsername);
- String encryptPassword = ConfigShadeUtils.encryptOption("base64", PASSWORD);
- String decryptPassword = ConfigShadeUtils.decryptOption("base64", encryptPassword);
- Assertions.assertEquals("Z3Jvb3RzdHJlYW0=", encryptUsername);
- Assertions.assertEquals("Z3Jvb3RzdHJlYW1fcGFzc3dvcmQ=", encryptPassword);
- Assertions.assertEquals(decryptUsername, USERNAME);
- Assertions.assertEquals(decryptPassword, PASSWORD);
- encryptUsername = ConfigShadeUtils.encryptOption("aes", USERNAME);
- decryptUsername = ConfigShadeUtils.decryptOption("aes", encryptUsername);
- encryptPassword = ConfigShadeUtils.encryptOption("aes", PASSWORD);
- decryptPassword = ConfigShadeUtils.decryptOption("aes", encryptPassword);
- Assertions.assertEquals("ed986337dfdbe341be1d29702e6ae619", encryptUsername);
- Assertions.assertEquals("159c7da83d988a9ec041d10a6bfbe221bcbaed6b62d9cc1b04ff51e633ebd105", encryptPassword);
- Assertions.assertEquals(decryptUsername, USERNAME);
- Assertions.assertEquals(decryptPassword, PASSWORD);
- encryptUsername = ConfigShadeUtils.encryptOption("sm4", USERNAME);
- decryptUsername = ConfigShadeUtils.decryptOption("sm4", encryptUsername);
- Assertions.assertEquals("72ea74367a15cb96b0d1d42104149519", encryptUsername);
- Assertions.assertEquals(decryptUsername, USERNAME);
- encryptPassword = ConfigShadeUtils.encryptOption("sm4", PASSWORD);
- decryptPassword = ConfigShadeUtils.decryptOption("sm4", encryptPassword);
- Assertions.assertEquals("3876c7088d395bbbfa826e3648b6c9a022e7f80941c132313bde6dc8a7f2351f", encryptPassword);
- Assertions.assertEquals(decryptPassword, PASSWORD);
- System.out.println( ConfigShadeUtils.encryptOption("sm4", "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"admin\" password=\"galaxy2019\";"));
- System.out.println( ConfigShadeUtils.decryptOption("sm4", "f76480be84a8ee1b009504c6c56a5bed48239c348a468f94b4029a6a3148f51530b025d6dfa140af93b4c7c6fe0e3dce543773e779d272b5579555fbd3271e7fdbee088673a901b3f3b28e914a25f30a4a859d97594c5ea7d7c1dcefe8c62560baea32b6da0b767232ed8aca17af2dc6"));
- System.out.println( ConfigShadeUtils.encryptOption("aes", "testuser"));
- System.out.println( ConfigShadeUtils.encryptOption("aes", "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"olap\" password=\"galaxy2019\";"));
- }
-}
diff --git a/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/utils/CryptoShadeTest.java b/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/utils/CryptoShadeTest.java
new file mode 100644
index 0000000..f77ba44
--- /dev/null
+++ b/groot-bootstrap/src/test/java/com/geedgenetworks/bootstrap/utils/CryptoShadeTest.java
@@ -0,0 +1,106 @@
+package com.geedgenetworks.bootstrap.utils;
+
+import cn.hutool.setting.yaml.YamlUtil;
+import com.alibaba.fastjson2.JSON;
+import com.alibaba.fastjson2.JSONObject;
+import com.geedgenetworks.common.Constants;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigObject;
+import com.typesafe.config.ConfigRenderOptions;
+import com.typesafe.config.ConfigValueFactory;
+import lombok.extern.slf4j.Slf4j;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.file.Paths;
+import java.util.Map;
+
+@Slf4j
+public class CryptoShadeTest {
+
+ private static final String USERNAME = "grootstream";
+
+ private static final String PASSWORD = "grootstream_password";
+
+ @Test
+ public void testParseConfig() throws URISyntaxException {
+ URL resource = CryptoShadeTest.class.getResource("/inline_to_clickhouse.yaml");
+ Assertions.assertNotNull(resource);
+ Map<String, Object> configMap = YamlUtil.loadByPath(Paths.get(resource.toURI()).toString());
+ ConfigObject configObject = ConfigValueFactory.fromMap(configMap);
+ Config decryptConfig = CryptoShadeUtils.decryptConfig(configObject.toConfig());
+ String jsonString = decryptConfig.root().render(ConfigRenderOptions.concise());
+ JSONObject jsonObject = JSON.parseObject(jsonString);
+ JSONObject sinkObject = jsonObject.getJSONObject(Constants.SINKS);
+ log.info("Decrypt config: {}", decryptConfig.root().render(ConfigRenderOptions.concise()));
+ Assertions.assertEquals(sinkObject.keySet().size(), 1);
+ Assertions.assertNotNull(sinkObject);
+ Assertions.assertNotNull(sinkObject.getJSONObject("clickhouse_sink"));
+ Assertions.assertNotNull(sinkObject.getJSONObject("clickhouse_sink")
+ .getJSONObject("properties"));
+ Assertions.assertEquals(sinkObject.getJSONObject("clickhouse_sink")
+ .getJSONObject("properties").isEmpty(), false);
+ Assertions.assertEquals(sinkObject.getJSONObject("clickhouse_sink")
+ .getJSONObject("properties").get("connection.user"), USERNAME);
+ Assertions.assertNotNull(sinkObject.getJSONObject("clickhouse_sink")
+ .getJSONObject("properties").get("connection.password"), PASSWORD);
+ }
+
+ @Test
+ public void testDecryptAndEncrypt() {
+ String encryptUsername = CryptoShadeUtils.encryptOption("base64", USERNAME);
+ String decryptUsername = CryptoShadeUtils.decryptOption("base64", encryptUsername);
+ String encryptPassword = CryptoShadeUtils.encryptOption("base64", PASSWORD);
+ String decryptPassword = CryptoShadeUtils.decryptOption("base64", encryptPassword);
+ Assertions.assertEquals("Z3Jvb3RzdHJlYW0=", encryptUsername);
+ Assertions.assertEquals("Z3Jvb3RzdHJlYW1fcGFzc3dvcmQ=", encryptPassword);
+ Assertions.assertEquals(decryptUsername, USERNAME);
+ Assertions.assertEquals(decryptPassword, PASSWORD);
+
+ encryptUsername = CryptoShadeUtils.encryptOption("aes", USERNAME);
+ decryptUsername = CryptoShadeUtils.decryptOption("aes", encryptUsername);
+ encryptPassword = CryptoShadeUtils.encryptOption("aes", PASSWORD);
+ decryptPassword = CryptoShadeUtils.decryptOption("aes", encryptPassword);
+ Assertions.assertEquals("7ZhjN9/b40G+HSlwLmrmGQ==", encryptUsername);
+ Assertions.assertEquals("FZx9qD2Yip7AQdEKa/viIby67Wti2cwbBP9R5jPr0QU=", encryptPassword);
+ Assertions.assertEquals(decryptUsername, USERNAME);
+ Assertions.assertEquals(decryptPassword, PASSWORD);
+
+ encryptUsername = CryptoShadeUtils.encryptOption("sm4", USERNAME);
+ decryptUsername = CryptoShadeUtils.decryptOption("sm4", encryptUsername);
+ Assertions.assertEquals("cup0NnoVy5aw0dQhBBSVGQ==", encryptUsername);
+ Assertions.assertEquals(decryptUsername, USERNAME);
+ encryptPassword = CryptoShadeUtils.encryptOption("sm4", PASSWORD);
+ decryptPassword = CryptoShadeUtils.decryptOption("sm4", encryptPassword);
+ Assertions.assertEquals("OHbHCI05W7v6gm42SLbJoCLn+AlBwTIxO95tyKfyNR8=", encryptPassword);
+ Assertions.assertEquals(decryptPassword, PASSWORD);
+
+ System.out.println(CryptoShadeUtils.encryptOption("sm4", "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"admin\" password=\"galaxy2019\";"));
+ System.out.println(CryptoShadeUtils.decryptOption("sm4", "f76480be84a8ee1b009504c6c56a5bed48239c348a468f94b4029a6a3148f51530b025d6dfa140af93b4c7c6fe0e3dce543773e779d272b5579555fbd3271e7fdbee088673a901b3f3b28e914a25f30a4a859d97594c5ea7d7c1dcefe8c62560baea32b6da0b767232ed8aca17af2dc6"));
+ System.out.println(CryptoShadeUtils.encryptOption("aes", "testuser"));
+ System.out.println(CryptoShadeUtils.encryptOption("aes", "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"olap\" password=\"galaxy2019\";"));
+
+ encryptUsername = CryptoShadeUtils.encryptOption("sm4-gcm96", USERNAME);
+ decryptUsername = CryptoShadeUtils.decryptOption("sm4-gcm96", encryptUsername);
+ encryptPassword = CryptoShadeUtils.encryptOption("sm4-gcm96", PASSWORD);
+ decryptPassword = CryptoShadeUtils.decryptOption("sm4-gcm96", encryptPassword);
+ Assertions.assertEquals(decryptUsername, USERNAME);
+ Assertions.assertEquals(decryptPassword, PASSWORD);
+
+ encryptUsername = CryptoShadeUtils.encryptOption("aes-128-gcm96", USERNAME);
+ decryptUsername = CryptoShadeUtils.decryptOption("aes-128-gcm96", encryptUsername);
+ encryptPassword = CryptoShadeUtils.encryptOption("aes-128-gcm96", PASSWORD);
+ decryptPassword = CryptoShadeUtils.decryptOption("aes-128-gcm96", encryptPassword);
+ Assertions.assertEquals(decryptUsername, USERNAME);
+ Assertions.assertEquals(decryptPassword, PASSWORD);
+
+ encryptUsername = CryptoShadeUtils.encryptOption("aes-256-gcm96", USERNAME);
+ decryptUsername = CryptoShadeUtils.decryptOption("aes-256-gcm96", encryptUsername);
+ encryptPassword = CryptoShadeUtils.encryptOption("aes-256-gcm96", PASSWORD);
+ decryptPassword = CryptoShadeUtils.decryptOption("aes-256-gcm96", encryptPassword);
+ Assertions.assertEquals(decryptUsername, USERNAME);
+ Assertions.assertEquals(decryptPassword, PASSWORD);
+ }
+}
diff --git a/groot-bootstrap/src/test/resources/META-INF/services/com.geedgenetworks.common.config.ConfigShade b/groot-bootstrap/src/test/resources/META-INF/services/com.geedgenetworks.common.config.ConfigShade
deleted file mode 100644
index 6654db5..0000000
--- a/groot-bootstrap/src/test/resources/META-INF/services/com.geedgenetworks.common.config.ConfigShade
+++ /dev/null
@@ -1,2 +0,0 @@
-com.geedgenetworks.bootstrap.command.Base64ConfigShade
-com.geedgenetworks.bootstrap.command.AESConfigShade \ No newline at end of file
diff --git a/groot-bootstrap/src/test/resources/META-INF/services/com.geedgenetworks.common.crypto.CryptoShade b/groot-bootstrap/src/test/resources/META-INF/services/com.geedgenetworks.common.crypto.CryptoShade
new file mode 100644
index 0000000..273b40d
--- /dev/null
+++ b/groot-bootstrap/src/test/resources/META-INF/services/com.geedgenetworks.common.crypto.CryptoShade
@@ -0,0 +1,6 @@
+com.geedgenetworks.bootstrap.command.Base64Shade
+com.geedgenetworks.bootstrap.command.AESShade
+com.geedgenetworks.bootstrap.command.SM4Shade
+com.geedgenetworks.bootstrap.command.AES128GCM96Shade
+com.geedgenetworks.bootstrap.command.AES256GCM96Shade
+com.geedgenetworks.bootstrap.command.SM4GCM96Shade \ No newline at end of file
diff --git a/groot-bootstrap/src/test/resources/grootstream_job_split_agg_test.yaml b/groot-bootstrap/src/test/resources/grootstream_job_agg_test.yaml
index 5163642..36a9ad3 100644
--- a/groot-bootstrap/src/test/resources/grootstream_job_split_agg_test.yaml
+++ b/groot-bootstrap/src/test/resources/grootstream_job_agg_test.yaml
@@ -2,8 +2,8 @@ sources:
inline_source:
type : inline
fields: # [array of object] Field List, if not set, all fields(Map<String, Object>) will be output.
- properties:
- data: '[{"pkts":1,"sessions":1,"decoded_as":"DNS","log_id": 1, "recv_time":"1724925692000", "client_ip":"192.168.0.2","server_ip":"2600:1015:b002::"},{"pkts":2,"sessions":1,"decoded_as":"DNS","log_id": 1, "recv_time":"1724925692000", "client_ip":"192.168.0.2","server_ip":"2600:1015:b002::"},{"sessions":1,"decoded_as":"DNS","log_id": 1, "recv_time":"1724925692000", "client_ip":"192.168.0.2","pkts":3,"server_ip":"2600:1015:b002::"},{"sessions":1,"decoded_as":"DNS","log_id": 1, "recv_time":"1724936692000", "client_ip":"192.168.0.2","pkts":4,"server_ip":"2600:1015:b002::"},{"sessions":1,"decoded_as":"DNS","log_id": 1, "recv_time":"1724937692000", "client_ip":"192.168.0.2","pkts":5,"server_ip":"2600:1015:b002::"}]'
+ properties: # record 3,4 will be aggreated
+ data: '[{"pkts":1,"sessions":1,"log_id": 1, "recv_time":"1724925692000", "client_ip":"192.168.0.2","server_ip":"2600:1015:b002::"},{"pkts":1,"sessions":1,"decoded_as":null,"log_id": 1, "recv_time":"1724925692000", "client_ip":"192.168.0.1","server_ip":"2600:1015:b002::"},{"pkts":2,"sessions":1,"decoded_as":"HTTP","log_id": 2, "recv_time":"1724925692000", "client_ip":"192.168.0.2","server_ip":"2600:1015:b002::"},{"sessions":1,"decoded_as":"DNS","log_id": 2, "recv_time":"1724925692000", "client_ip":"192.168.0.2","pkts":3,"server_ip":"2600:1015:b002::"},{"sessions":1,"decoded_as":"DNS","log_id": 1, "recv_time":"1724936692000", "client_ip":"192.168.0.2","pkts":4,"server_ip":"2600:1015:b002::"},{"sessions":1,"decoded_as":"HTTP","log_id": 1, "recv_time":"1724937692000", "client_ip":"192.168.0.2","pkts":5,"server_ip":"2600:1015:b002::"}]'
interval.per.row: 1s # 可选
repeat.count: 1 # 可选
format: json
@@ -31,13 +31,26 @@ postprocessing_pipelines:
lookup_fields: [ sessions ]
- function: MEAN
lookup_fields: [ pkts ]
-
- table_processor:
- type: table
- functions:
- - function: JSON_UNROLL
- lookup_fields: [ encapsulation ]
- output_fields: [ new_name ]
+ - function: MAX
+ lookup_fields: [ pkts ]
+ output_fields: [ pktsmax ]
+ - function: MIN
+ lookup_fields: [ pkts ]
+ output_fields: [ pktsmin ]
+ - function: LONG_COUNT
+ output_fields: [ count ]
+ - function: COLLECT_LIST
+ lookup_fields: [ client_ip ]
+ output_fields: [ client_ip_list ]
+ - function: COLLECT_SET
+ lookup_fields: [ server_ip ]
+ output_fields: [ server_ip_set ]
+ - function: FIRST_VALUE
+ lookup_fields: [ log_id ]
+ output_fields: [ log_id_first ]
+ - function: LAST_VALUE
+ lookup_fields: [ log_id ]
+ output_fields: [ log_id_last ]
application: # [object] Application Configuration
env: # [object] Environment Variables
diff --git a/groot-bootstrap/src/test/resources/grootstream_job_dos_test.yaml b/groot-bootstrap/src/test/resources/grootstream_job_dos_test.yaml
new file mode 100644
index 0000000..92eaef2
--- /dev/null
+++ b/groot-bootstrap/src/test/resources/grootstream_job_dos_test.yaml
@@ -0,0 +1,130 @@
+sources:
+
+ inline_source:
+ type : inline
+ watermark_timestamp: timestamp_ms
+ watermark_timestamp_unit: ms
+ watermark_lag: 10
+ fields: # [array of object] Field List, if not set, all fields(Map<String, Object>) will be output.
+ properties:
+ data: '[{"sessions":100,"pkts":1000,"bytes":10000,"timestamp_ms":1729476000001,"vsys_id": 1,"duration": 60000,"rule_uuid": "123","client_ip":"1.1.1.2","server_ip":"1.2.2.2","client_country":"CN","server_country":"US"},{"sessions":100,"pkts":1000,"bytes":10000,"timestamp_ms":1729476000001,"vsys_id": 1,"duration": 60000,"rule_uuid": "123","client_ip":"1.1.1.1","server_ip":"1.2.2.2","client_country":"CN","server_country":"US"},{"timestamp_ms":1729476003000,"vsys_id": 1,"duration": 60000,"rule_uuid": "123","client_ip":"1.1.1.1","server_ip":"1.2.2.2","client_country":"CN1","server_country":"US1","sessions":100,"pkts":1000,"bytes":10000},{"timestamp_ms":1729477003000,"vsys_id": 1,"duration": 60000,"rule_uuid": "123","client_ip":"1.1.1.1","server_ip":"1.2.2.2","client_country":"CN1","server_country":"US1","sessions":100,"pkts":1000,"bytes":10000}]'
+ interval.per.row: 2s # 可选
+ repeat.count: 1 # 可选
+ format: json
+ json.ignore.parse.errors: false
+
+
+
+postprocessing_pipelines:
+
+ pre_etl_processor: # [object] Processing Pipeline
+ type: projection
+ functions: # [array of object] Function List
+ - function: CURRENT_UNIX_TIMESTAMP
+ output_fields: [ recv_time ]
+ parameters:
+ precision: seconds
+ aggregate_processor:
+ type: aggregate
+ group_by_fields: [vsys_id,rule_uuid,server_ip,client_ip]
+ window_type: tumbling_event_time # tumbling_event_time,sliding_processing_time,sliding_event_time
+ window_size: 600
+ functions:
+ - function: NUMBER_SUM
+ lookup_fields: [ sessions ]
+ - function: NUMBER_SUM
+ lookup_fields: [ bytes ]
+ - function: NUMBER_SUM
+ lookup_fields: [ pkts ]
+ output_fields: [ packets ]
+ - function: FIRST_VALUE
+ lookup_fields: [ client_country ]
+ - function: FIRST_VALUE
+ lookup_fields: [ server_country ]
+ - function: MIN
+ lookup_fields: [ timestamp_ms ]
+ output_fields: [ start_timestamp_ms ]
+ - function: MIN
+ lookup_fields: [ recv_time ]
+ - function: MAX
+ lookup_fields: [ timestamp_ms ]
+ output_fields: [ end_timestamp_ms ]
+ - function: FIRST_VALUE
+ lookup_fields: [ duration ]
+ post_etl_processor: # [object] Processing Pipeline
+ type: projection
+ remove_fields:
+ output_fields:
+ functions: # [array of object] Function List
+ - function: UNIX_TIMESTAMP_CONVERTER
+ lookup_fields: [ end_timestamp_ms ]
+ output_fields: [ end_time ]
+ parameters:
+ precision: seconds
+ - function: UNIX_TIMESTAMP_CONVERTER
+ lookup_fields: [ start_timestamp_ms ]
+ output_fields: [ start_time ]
+ parameters:
+ precision: seconds
+ - function: EVAL
+ output_fields: [ duration ]
+ parameters:
+ value_expression: "((end_time-start_time) > 0)? (end_time-start_time) : (duration/1000)"
+ - function: EVAL
+ output_fields: [ end_time ]
+ parameters:
+ value_expression: start_time + duration
+ - function: EVAL
+ output_fields: [ session_rate ]
+ parameters:
+ value_expression: math.round((double(sessions) / duration )*100)/100.0
+ - function: EVAL
+ output_fields: [ packet_rate ]
+ parameters:
+ value_expression: math.round((double(packets) / duration ) *100)/100.0
+ - function: EVAL
+ output_fields: [ bit_rate ]
+ parameters:
+ value_expression: math.round((double((bytes*8)) / duration) *100)/100.0
+ - function: RENAME
+ parameters:
+ rename_fields:
+ client_ip: source_ip
+ client_country: source_country
+ server_ip: destination_ip
+ server_country: destination_country
+ - function: SNOWFLAKE_ID
+ lookup_fields: ['']
+ output_fields: [log_id]
+ parameters:
+ data_center_id_num: 1
+
+sinks:
+
+ collect_sink:
+ type: collect
+ properties:
+ format: json
+
+application: # [object] Application Configuration
+
+ env: # [object] Environment Variables
+ name: groot-stream-job # [string] Job Name
+ parallelism: 1
+ properties:
+ k: v
+ pipeline:
+ object-reuse: true # [boolean] Object Reuse, default is false
+
+
+ topology:
+ - name: inline_source
+ downstream: [pre_etl_processor]
+ - name: pre_etl_processor
+ downstream: [aggregate_processor]
+ - name: aggregate_processor
+ downstream: [ post_etl_processor ]
+ - name: post_etl_processor
+ downstream: [ collect_sink]
+ - name: collect_sink
+
diff --git a/groot-bootstrap/src/test/resources/grootstream_job_etl_test.yaml b/groot-bootstrap/src/test/resources/grootstream_job_etl_test.yaml
index 9724e21..e3f5613 100644
--- a/groot-bootstrap/src/test/resources/grootstream_job_etl_test.yaml
+++ b/groot-bootstrap/src/test/resources/grootstream_job_etl_test.yaml
@@ -5,14 +5,14 @@ sources:
type : inline
fields: # [array of object] Field List, if not set, all fields(Map<String, Object>) will be output.
properties:
- data: '[{"encapsulation":"[{\"tunnels_schema_type\":\"MULTIPATH_ETHERNET\"}]","mail_attachment_name_charset":"GBK","mail_attachment_name":"aGVsbG8=","packet_capture_file":"test","ssl_sni":"www.google.com","decoded_as":"BASE","ssl_san":"www.google.com","__timestamp":1705568517095,"client_ip":"255.255.255.255","server_ip":"2600:1015:b002::"},{"decoded_as":"HTTP","log_id": 1, "recv_time":"111", "client_ip":"192.168.0.2","server_ip":"2600:1015:b002::"},{"decoded_as":"DNS","log_id": 1, "recv_time":"111", "client_ip":"192.168.0.2","server_ip":"2600:1015:b002::"},{"decoded_as":"SSL","log_id": 1, "recv_time":"111", "client_ip":"192.168.0.2","server_ip":"2600:1015:b002::"}]'
+ data: '[{"encapsulation":"[{\"tunnels_schema_type\":\"MULTIPATH_ETHERNET\"}]","mail_attachment_name_charset":"GBK","mail_attachment_name":"aGVsbG8=","packet_capture_file":"test","ssl_sni":"www.google.com","decoded_as":"BASE","ssl_san":"www.google.com","start_timestamp":1705568517095,"client_ip":"255.255.255.255","server_ip":"2600:1015:b002::"},{"decoded_as":"HTTP","log_id": 1, "recv_time":"111", "client_ip":"192.168.0.2","server_ip":"2600:1015:b002::"},{"decoded_as":"DNS","log_id": 1, "recv_time":"111", "client_ip":"192.168.0.2","server_ip":"2600:1015:b002::"},{"decoded_as":"SSL","log_id": 1, "recv_time":"111", "client_ip":"192.168.0.2","server_ip":"2600:1015:b002::"}]'
interval.per.row: 1s # 可选
repeat.count: 1 # 可选
format: json
json.ignore.parse.errors: false
filters:
schema_type_filter:
- type: com.geedgenetworks.core.filter.AviatorFilter
+ type: aviator
output_fields:
properties:
expression: event.decoded_as == 'SSL' || event.decoded_as == 'BASE'
@@ -20,7 +20,7 @@ filters:
preprocessing_pipelines:
preprocessing_processor: # [object] Preprocessing Pipeline
- type: com.geedgenetworks.core.processor.projection.ProjectionProcessorImpl
+ type: projection
output_fields:
properties:
key: value
@@ -36,7 +36,7 @@ preprocessing_pipelines:
processing_pipelines:
session_record_processor: # [object] Processing Pipeline
- type: com.geedgenetworks.core.processor.projection.ProjectionProcessorImpl
+ type: projection
remove_fields:
output_fields:
properties:
@@ -52,6 +52,14 @@ processing_pipelines:
value_field: mail_attachment_name
charset_field: mail_attachment_name_charset
+ - function: BASE64_ENCODE_TO_STRING
+ output_fields: [old_mail_attachment_name]
+ lookup_fields: [mail_attachment_name]
+ parameters:
+ input_type: string
+
+
+
- function: GEOIP_LOOKUP
lookup_fields: [ client_ip ]
output_fields: [ ]
@@ -101,6 +109,10 @@ processing_pipelines:
output_fields: [ ingestion_time ]
parameters:
value_expression: recv_time
+ - function: EVAL
+ output_fields: [internal_ip]
+ parameters:
+ value_expression: "(direction == 'Outbound')? client_ip : server_ip"
- function: DOMAIN
lookup_fields: [ http_host,ssl_sni,dtls_sni,quic_sni ]
output_fields: [ server_domain ]
@@ -141,6 +153,25 @@ processing_pipelines:
# tags: tags
rename_expression: key =string.replace_all(key,'encapsulation.0.','');return key;
+ - function: UUIDv5
+ lookup_fields: [ client_ip, server_ip ] # 基于 client_ip, server_ip的值组成UUIDv5 name 参数值与命名空间结合后,通过哈希生成唯一的 UUID。
+ output_fields: [ ip_uuid ]
+ parameters:
+ namespace: NAMESPACE_IP
+ - function: UUIDv7
+ output_fields: [ log_uuid_v7 ] # 生成基于时间戳和随机数的 UUID
+ - function: UUID
+ output_fields: [ log_uuid ]
+
+ - function: FROM_UNIX_TIMESTAMP
+ lookup_fields: [start_timestamp]
+ output_fields: [start_time]
+ parameters:
+ timezone: Asia/Shanghai
+ precision: milliseconds
+
+
+
sinks:
kafka_sink_a:
diff --git a/groot-bootstrap/src/test/resources/grootstream_job_split_test.yaml b/groot-bootstrap/src/test/resources/grootstream_job_split_test.yaml
index 9bb2900..01fc6dd 100644
--- a/groot-bootstrap/src/test/resources/grootstream_job_split_test.yaml
+++ b/groot-bootstrap/src/test/resources/grootstream_job_split_test.yaml
@@ -17,14 +17,14 @@ splits:
test_split:
type: split
rules:
- - name: table_processor
+ - tag: http_tag
expression: event.decoded_as == 'HTTP'
- - name: pre_etl_processor
+ - tag: dns_tag
expression: event.decoded_as == 'DNS'
postprocessing_pipelines:
pre_etl_processor: # [object] Processing Pipeline
- type: com.geedgenetworks.core.processor.projection.ProjectionProcessorImpl
+ type: projection
remove_fields: [fields,tags]
output_fields:
functions: # [array of object] Function List
@@ -83,8 +83,9 @@ application: # [object] Application Configuration
parallelism: 1 # [number] Operator-Level Parallelism.
downstream: [test_split,collect_sink]
- name: test_split
- parallelism: 1
+ tags: [http_tag,dns_tag]
downstream: [ table_processor,pre_etl_processor ]
+ parallelism: 1
- name: pre_etl_processor
parallelism: 1
downstream: [ collect_sink ]
diff --git a/groot-common/src/main/java/com/geedgenetworks/common/Constants.java b/groot-common/src/main/java/com/geedgenetworks/common/Constants.java
index b523591..27ce8fb 100644
--- a/groot-common/src/main/java/com/geedgenetworks/common/Constants.java
+++ b/groot-common/src/main/java/com/geedgenetworks/common/Constants.java
@@ -2,7 +2,7 @@ package com.geedgenetworks.common;
public final class Constants {
- public static final String DEFAULT_JOB_NAME="groot-stream-job";
+ public static final String DEFAULT_JOB_NAME = "groot-stream-job";
public static final String SOURCES = "sources";
public static final String FILTERS = "filters";
public static final String PREPROCESSING_PIPELINES = "preprocessing_pipelines";
@@ -14,7 +14,7 @@ public final class Constants {
public static final String PROPERTIES = "properties";
public static final String SPLITS = "splits";
- public static final String APPLICATION_ENV ="env";
+ public static final String APPLICATION_ENV = "env";
public static final String APPLICATION_TOPOLOGY = "topology";
public static final String JOB_NAME = "name";
public static final String GROOT_LOGO = "\n" +
@@ -49,6 +49,8 @@ public final class Constants {
public static final String SLIDING_PROCESSING_TIME = "sliding_processing_time";
public static final String SLIDING_EVENT_TIME = "sliding_event_time";
-
+ public static final String SYSPROP_KMS_TYPE_CONFIG = "kms.type";
+ public static final String SYSPROP_ENCRYPT_KMS_KEY_SCHEDULER_INTERVAL_NAME = "scheduler.encrypt.update.kms.key.minutes";
+ public static final String SYSPROP_ENCRYPT_SENSITIVE_FIELDS_SCHEDULER_INTERVAL_NAME = "scheduler.encrypt.update.sensitive.fields.minutes";
}
diff --git a/groot-common/src/main/java/com/geedgenetworks/common/Event.java b/groot-common/src/main/java/com/geedgenetworks/common/Event.java
index 4ab4aef..7733c66 100644
--- a/groot-common/src/main/java/com/geedgenetworks/common/Event.java
+++ b/groot-common/src/main/java/com/geedgenetworks/common/Event.java
@@ -8,6 +8,7 @@ import java.util.Map;
@Data
public class Event implements Serializable {
public static final String INTERNAL_TIMESTAMP_KEY = "__timestamp";
+ public static final String INTERNAL_HEADERS_KEY = "__headers";
public static final String WINDOW_START_TIMESTAMP = "__window_start_timestamp";
public static final String WINDOW_END_TIMESTAMP = "__window_end_timestamp";
diff --git a/groot-common/src/main/java/com/geedgenetworks/common/config/CheckConfigUtil.java b/groot-common/src/main/java/com/geedgenetworks/common/config/CheckConfigUtil.java
index 96df69c..1d4e819 100644
--- a/groot-common/src/main/java/com/geedgenetworks/common/config/CheckConfigUtil.java
+++ b/groot-common/src/main/java/com/geedgenetworks/common/config/CheckConfigUtil.java
@@ -12,6 +12,7 @@ public final class CheckConfigUtil {
private CheckConfigUtil() {}
public static CheckResult checkAllExists(Config config, String... params) {
+
List<String> missingParams =
Arrays.stream(params)
.filter(param -> !isValidParam(config, param))
@@ -20,11 +21,10 @@ public final class CheckConfigUtil {
if (!missingParams.isEmpty()) {
String errorMsg =
String.format(
- "please specify [%s] as non-empty.", String.join(",", missingParams));
+ "Please specify [%s] as non-empty.", String.join(",", missingParams));
return CheckResult.error(errorMsg);
- } else {
- return CheckResult.success();
}
+ return CheckResult.success();
}
/** check config if there was at least one usable */
@@ -33,48 +33,42 @@ public final class CheckConfigUtil {
return CheckResult.success();
}
- List<String> missingParams = new LinkedList<>();
- for (String param : params) {
- if (!isValidParam(config, param)) {
- missingParams.add(param);
- }
- }
+ List<String> missingParams = Arrays.stream(params)
+ .filter(param -> !isValidParam(config, param))
+ .collect(Collectors.toList());
if (missingParams.size() == params.length) {
String errorMsg =
String.format(
- "please specify at least one config of [%s] as non-empty.",
+ "Please specify at least one config of [%s] as non-empty.",
String.join(",", missingParams));
return CheckResult.error(errorMsg);
- } else {
- return CheckResult.success();
}
+ return CheckResult.success();
}
- public static boolean isValidParam(Config config, String param) {
- boolean isValidParam = true;
+ public static boolean isValidParam(Config config, String param) {
if (!config.hasPath(param)) {
- isValidParam = false;
- } else if (config.getAnyRef(param) instanceof List) {
- isValidParam = !((List<?>) config.getAnyRef(param)).isEmpty();
+ return false;
}
- return isValidParam;
+ Object value = config.getAnyRef(param);
+ return !(value instanceof List && ((List<?>) value).isEmpty());
}
/** merge all check result */
public static CheckResult mergeCheckResults(CheckResult... checkResults) {
+
List<CheckResult> notPassConfig =
Arrays.stream(checkResults)
.filter(item -> !item.isSuccess())
.collect(Collectors.toList());
if (notPassConfig.isEmpty()) {
return CheckResult.success();
- } else {
- String errMessage =
- notPassConfig.stream()
- .map(CheckResult::getMsg)
- .collect(Collectors.joining(","));
- return CheckResult.error(errMessage);
}
+ String errMessage =
+ notPassConfig.stream()
+ .map(CheckResult::getMsg)
+ .collect(Collectors.joining(","));
+ return CheckResult.error(errMessage);
}
}
diff --git a/groot-common/src/main/java/com/geedgenetworks/common/config/CheckResult.java b/groot-common/src/main/java/com/geedgenetworks/common/config/CheckResult.java
index 5bf0196..e8e47f3 100644
--- a/groot-common/src/main/java/com/geedgenetworks/common/config/CheckResult.java
+++ b/groot-common/src/main/java/com/geedgenetworks/common/config/CheckResult.java
@@ -1,8 +1,13 @@
package com.geedgenetworks.common.config;
+import lombok.Data;
+
+@Data
public class CheckResult {
private static final CheckResult SUCCESS = new CheckResult(true, "");
+
private boolean success;
+
private String msg;
private CheckResult(boolean success, String msg) {
@@ -10,71 +15,16 @@ public class CheckResult {
this.msg = msg;
}
+ /** @return a successful instance of CheckResult */
public static CheckResult success() {
return SUCCESS;
}
+ /**
+ * @param msg the error message
+ * @return an error instance of CheckResult
+ */
public static CheckResult error(String msg) {
return new CheckResult(false, msg);
}
-
- public boolean isSuccess() {
- return this.success;
- }
-
- public String getMsg() {
- return this.msg;
- }
-
- public void setSuccess(boolean success) {
- this.success = success;
- }
-
- public void setMsg(String msg) {
- this.msg = msg;
- }
-
- public boolean equals(Object o) {
- if (o == this) {
- return true;
- } else if (!(o instanceof CheckResult)) {
- return false;
- } else {
- CheckResult other = (CheckResult)o;
- if (!other.canEqual(this)) {
- return false;
- } else if (this.isSuccess() != other.isSuccess()) {
- return false;
- } else {
- Object this$msg = this.getMsg();
- Object other$msg = other.getMsg();
- if (this$msg == null) {
- if (other$msg != null) {
- return false;
- }
- } else if (!this$msg.equals(other$msg)) {
- return false;
- }
-
- return true;
- }
- }
- }
-
- protected boolean canEqual(Object other) {
- return other instanceof CheckResult;
- }
-
- public int hashCode() {
- int PRIME = 59;
- int result = 1;
- result = result * PRIME + (this.isSuccess() ? 79 : 97);
- Object $msg = this.getMsg();
- result = result * PRIME + ($msg == null ? 43 : $msg.hashCode());
- return result;
- }
-
- public String toString() {
- return "CheckResult(success=" + this.isSuccess() + ", msg=" + this.getMsg() + ")";
- }
}
diff --git a/groot-common/src/main/java/com/geedgenetworks/common/config/CheckUDFContextUtil.java b/groot-common/src/main/java/com/geedgenetworks/common/config/CheckUDFContextUtil.java
new file mode 100644
index 0000000..f1170be
--- /dev/null
+++ b/groot-common/src/main/java/com/geedgenetworks/common/config/CheckUDFContextUtil.java
@@ -0,0 +1,107 @@
+package com.geedgenetworks.common.config;
+
+import com.geedgenetworks.common.udf.UDFContext;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public final class CheckUDFContextUtil {
+
+ private CheckUDFContextUtil() {}
+
+ // Check if all the params are present in the UDFContext
+ public static CheckResult checkAllExists(UDFContext context, String... params) {
+ List<String> invalidParams = Arrays.stream(params)
+ .filter(param -> isInvalidParam(context, param))
+ .collect(Collectors.toList());
+
+ if (!invalidParams.isEmpty()) {
+ String errorMsg = java.lang.String.format("Please specify [%s] as non-empty.", java.lang.String.join(",", invalidParams));
+ return CheckResult.error(errorMsg);
+ }
+ return CheckResult.success();
+ }
+
+ // Check if at least one of the params is present in the UDFContext
+ public static CheckResult checkAtLeastOneExists(UDFContext context, String... params) {
+ if (params.length == 0) {
+ return CheckResult.success();
+ }
+
+ List<String> invalidParams = Arrays.stream(params)
+ .filter(param -> isInvalidParam(context, param))
+ .collect(Collectors.toList());
+
+ if (invalidParams.size() == params.length) {
+ String errorMsg = java.lang.String.format("Please specify at least one config of [%s] as non-empty.", java.lang.String.join(",", invalidParams));
+ return CheckResult.error(errorMsg);
+ }
+ return CheckResult.success();
+ }
+
+
+
+ // Check Array/Map Object has only one item
+ public static CheckResult checkCollectionSingleItemExists (UDFContext context, String param) {
+ if (context == null) {
+ return CheckResult.error("UDFContext is null");
+ }
+
+ if (UDFContextConfigOptions.LOOKUP_FIELDS.key().equals(param)) {
+ return context.getLookupFields() != null && context.getLookupFields().size() == 1 ? CheckResult.success() : CheckResult.error("Lookup fields should have only one item");
+ } else if (UDFContextConfigOptions.OUTPUT_FIELDS.key().equals(param)) {
+ return context.getOutputFields() != null && context.getOutputFields().size() == 1 ? CheckResult.success() : CheckResult.error("Output fields should have only one item");
+ } else if (UDFContextConfigOptions.PARAMETERS.key().equals(param)) {
+ return context.getParameters() != null && context.getParameters().size() == 1 ? CheckResult.success() : CheckResult.error("Parameters should have only one item");
+ } else {
+ return CheckResult.error("Invalid param");
+ }
+
+ }
+
+ // Check Parameters contains keys
+ public static CheckResult checkParametersContainsKeys(UDFContext context, String... keys) {
+ if (context == null) {
+ return CheckResult.error("UDFContext is null");
+ }
+
+ if (context.getParameters() == null) {
+ return CheckResult.error("Parameters is null");
+ }
+
+ List<String> missingKeys = Arrays.stream(keys)
+ .filter(key -> !context.getParameters().containsKey(key))
+ .collect(Collectors.toList());
+
+ if (!missingKeys.isEmpty()) {
+ String errorMsg = java.lang.String.format("Please specify [%s] as non-empty.", java.lang.String.join(",", missingKeys));
+ return CheckResult.error(errorMsg);
+ }
+ return CheckResult.success();
+ }
+
+ public static boolean isInvalidParam(UDFContext context, String param) {
+ if (context == null) {
+ return true;
+ }
+
+ if (UDFContextConfigOptions.NAME.key().equals(param)) {
+ return context.getName() == null;
+ } else if (UDFContextConfigOptions.LOOKUP_FIELDS.key().equals(param)) {
+ return context.getLookupFields() == null || context.getLookupFields().isEmpty();
+ } else if (UDFContextConfigOptions.OUTPUT_FIELDS.key().equals(param)) {
+ return context.getOutputFields() == null || context.getOutputFields().isEmpty();
+ } else if (UDFContextConfigOptions.FILTER.key().equals(param)) {
+ return context.getFilter() == null;
+ } else if (UDFContextConfigOptions.PARAMETERS.key().equals(param)) {
+ return context.getParameters() == null || context.getParameters().isEmpty();
+ } else if (UDFContextConfigOptions.FUNCTION.key().equals(param)) {
+ return context.getFunction() == null;
+ } else {
+ return true;
+ }
+
+ }
+
+
+}
diff --git a/groot-common/src/main/java/com/geedgenetworks/common/config/CommonConfig.java b/groot-common/src/main/java/com/geedgenetworks/common/config/CommonConfig.java
index 4fdf0c6..aeda71d 100644
--- a/groot-common/src/main/java/com/geedgenetworks/common/config/CommonConfig.java
+++ b/groot-common/src/main/java/com/geedgenetworks/common/config/CommonConfig.java
@@ -15,13 +15,28 @@ import static com.google.common.base.Preconditions.checkNotNull;
public class CommonConfig implements Serializable {
private List<KnowledgeBaseConfig> knowledgeBaseConfig = CommonConfigOptions.KNOWLEDGE_BASE.defaultValue();
+
+ private Map<String,KmsConfig> kmsConfig = CommonConfigOptions.KMS.defaultValue();
+
+ private SSLConfig sslConfig = CommonConfigOptions.SSL.defaultValue();
+
private Map<String,String> propertiesConfig = CommonConfigOptions.PROPERTIES.defaultValue();
public void setKnowledgeBaseConfig(List<KnowledgeBaseConfig> knowledgeBaseConfig) {
- checkNotNull(knowledgeBaseConfig, CommonConfigOptions.KNOWLEDGE_BASE + "knowledgeConfig should not be null");
+ checkNotNull(knowledgeBaseConfig, CommonConfigOptions.KNOWLEDGE_BASE + " knowledgeConfig should not be null");
this.knowledgeBaseConfig = knowledgeBaseConfig;
}
+ public void setKmsConfig(Map<String,KmsConfig> kmsConfig) {
+ checkNotNull(kmsConfig, CommonConfigOptions.KMS + " kmsConfig should not be null");
+ this.kmsConfig = kmsConfig;
+ }
+
+ public void setSslConfig(SSLConfig sslConfig) {
+ checkNotNull(sslConfig, CommonConfigOptions.SSL + " sslConfig should not be null");
+ this.sslConfig = sslConfig;
+ }
+
diff --git a/groot-common/src/main/java/com/geedgenetworks/common/config/CommonConfigDomProcessor.java b/groot-common/src/main/java/com/geedgenetworks/common/config/CommonConfigDomProcessor.java
index 785b4bb..b3b17e8 100644
--- a/groot-common/src/main/java/com/geedgenetworks/common/config/CommonConfigDomProcessor.java
+++ b/groot-common/src/main/java/com/geedgenetworks/common/config/CommonConfigDomProcessor.java
@@ -1,8 +1,6 @@
package com.geedgenetworks.common.config;
import com.hazelcast.internal.config.AbstractDomConfigProcessor;
-import com.hazelcast.logging.ILogger;
-import com.hazelcast.logging.Logger;
import lombok.extern.slf4j.Slf4j;
import org.w3c.dom.Node;
@@ -16,6 +14,7 @@ import static com.hazelcast.internal.config.DomConfigHelper.*;
@Slf4j
public class CommonConfigDomProcessor extends AbstractDomConfigProcessor {
private final GrootStreamConfig config;
+
CommonConfigDomProcessor(boolean domLevel3, GrootStreamConfig config) {
super(domLevel3);
this.config = config;
@@ -26,12 +25,16 @@ public class CommonConfigDomProcessor extends AbstractDomConfigProcessor {
final CommonConfig commonConfig = config.getCommonConfig();
for (Node node : childElements(rootNode)) {
String name = cleanNodeName(node);
- if (CommonConfigOptions.KNOWLEDGE_BASE.key().equals(name)) {
- commonConfig.setKnowledgeBaseConfig(parseKnowledgeBaseConfig(node));
+ if (CommonConfigOptions.KNOWLEDGE_BASE.key().equals(name)) {
+ commonConfig.setKnowledgeBaseConfig(parseKnowledgeBaseConfig(node));
+ } else if (CommonConfigOptions.KMS.key().equals(name)) {
+ commonConfig.setKmsConfig(parseKmsConfig(node));
+ } else if (CommonConfigOptions.SSL.key().equals(name)) {
+ commonConfig.setSslConfig(parseSSLConfig(node));
} else if (CommonConfigOptions.PROPERTIES.key().equals(name)) {
- commonConfig.setPropertiesConfig(parsePropertiesConfig(node));
+ commonConfig.setPropertiesConfig(parsePropertiesConfig(node));
} else {
- log.warn("Unrecognized configuration element: " + name);
+ log.warn("Unrecognized Groot Stream configuration element: {}", name);
}
}
@@ -39,12 +42,12 @@ public class CommonConfigDomProcessor extends AbstractDomConfigProcessor {
private Map<String, String> parsePropertiesConfig(Node properties) {
- Map<String, String> propertiesMap = new HashMap<>();
- for (Node node : childElements(properties)) {
- String name = cleanNodeName(node);
- propertiesMap.put(name,getTextContent(node));
- }
- return propertiesMap;
+ Map<String, String> propertiesMap = new HashMap<>();
+ for (Node node : childElements(properties)) {
+ String name = cleanNodeName(node);
+ propertiesMap.put(name, getTextContent(node));
+ }
+ return propertiesMap;
}
@@ -57,11 +60,11 @@ public class CommonConfigDomProcessor extends AbstractDomConfigProcessor {
return knowledgeConfigList;
}
- private KnowledgeBaseConfig parseKnowledgeBaseConfigAsObject(Node kbNode) {
+
+ private KnowledgeBaseConfig parseKnowledgeBaseConfigAsObject(Node kbNode) {
KnowledgeBaseConfig knowledgeBaseConfig = new KnowledgeBaseConfig();
for (Node node : childElements(kbNode)) {
String name = cleanNodeName(node);
-
if (CommonConfigOptions.KNOWLEDGE_BASE_NAME.key().equals(name)) {
knowledgeBaseConfig.setName(getTextContent(node));
} else if (CommonConfigOptions.KNOWLEDGE_BASE_FS_TYPE.key().equals(name)) {
@@ -72,19 +75,71 @@ public class CommonConfigDomProcessor extends AbstractDomConfigProcessor {
knowledgeBaseConfig.setFiles(parseKnowledgeBaseFilesConfig(node));
} else if (CommonConfigOptions.KNOWLEDGE_BASE_PROPERTIES.key().equals(name)) {
knowledgeBaseConfig.setProperties(parseKnowledgeBasePropertiesConfig(node));
+ } else {
+ log.warn("Unrecognized KB configuration element: {}", name);
}
- else{
- log.warn("Unrecognized configuration element: " + name);
- }
+
}
return knowledgeBaseConfig;
}
+ private SSLConfig parseSSLConfig(Node sslRootNode) {
+ SSLConfig sslConfig = new SSLConfig();
+ for (Node node : childElements(sslRootNode)) {
+ String name = cleanNodeName(node);
+ if (CommonConfigOptions.SKIP_VERIFICATION.key().equals(name)) {
+ sslConfig.setSkipVerification(getBooleanValue(getTextContent(node)));
+ } else if (CommonConfigOptions.CA_CERTIFICATE_PATH.key().equals(name)) {
+ sslConfig.setCaCertificatePath(getTextContent(node));
+ } else if (CommonConfigOptions.CERTIFICATE_PATH.key().equals(name)) {
+ sslConfig.setCertificatePath(getTextContent(node));
+ } else if (CommonConfigOptions.PRIVATE_KEY_PATH.key().equals(name)) {
+ sslConfig.setPrivateKeyPath(getTextContent(node));
+ } else {
+ log.warn("Unrecognized SSL configuration element: {}", name);
+ }
+ }
+ return sslConfig;
+ }
+
+ private Map<String, KmsConfig> parseKmsConfig(Node kmsRootNode) {
+ Map<String, KmsConfig> kmsConfigMap = new HashMap<>();
+ for (Node node : childElements(kmsRootNode)) {
+ String name = cleanNodeName(node);
+ kmsConfigMap.put(name, parseKmsConfigAsObject(node));
+ }
+ return kmsConfigMap;
+ }
+
+ private KmsConfig parseKmsConfigAsObject(Node kmsNode) {
+ KmsConfig kmsConfig = new KmsConfig();
+ for (Node node : childElements(kmsNode)) {
+ String name = cleanNodeName(node);
+ if (CommonConfigOptions.KMS_TYPE.key().equals(name)) {
+ kmsConfig.setType(getTextContent(node));
+ } else if (CommonConfigOptions.KMS_URL.key().equals(name)) {
+ kmsConfig.setUrl(getTextContent(node));
+ } else if (CommonConfigOptions.KMS_USERNAME.key().equals(name)) {
+ kmsConfig.setUsername(getTextContent(node));
+ } else if (CommonConfigOptions.KMS_PASSWORD.key().equals(name)) {
+ kmsConfig.setPassword(getTextContent(node));
+ } else if (CommonConfigOptions.KMS_DEFAULT_KEY_PATH.key().equals(name)) {
+ kmsConfig.setDefaultKeyPath(getTextContent(node));
+ } else if (CommonConfigOptions.KMS_PLUGIN_KEY_PATH.key().equals(name)) {
+ kmsConfig.setPluginKeyPath(getTextContent(node));
+ } else {
+ log.warn("Unrecognized KMS configuration element: {}", name);
+ }
+ }
+ return kmsConfig;
+ }
+
+
private Map<String, String> parseKnowledgeBasePropertiesConfig(Node properties) {
Map<String, String> propertiesMap = new HashMap<>();
for (Node node : childElements(properties)) {
String name = cleanNodeName(node);
- propertiesMap.put(name,getTextContent(node));
+ propertiesMap.put(name, getTextContent(node));
}
return propertiesMap;
}
diff --git a/groot-common/src/main/java/com/geedgenetworks/common/config/CommonConfigOptions.java b/groot-common/src/main/java/com/geedgenetworks/common/config/CommonConfigOptions.java
index a3f3468..167fcba 100644
--- a/groot-common/src/main/java/com/geedgenetworks/common/config/CommonConfigOptions.java
+++ b/groot-common/src/main/java/com/geedgenetworks/common/config/CommonConfigOptions.java
@@ -12,28 +12,28 @@ public class CommonConfigOptions {
public static final Option<Map<String, String>> KNOWLEDGE_BASE_PROPERTIES =
Options.key("properties")
.mapType()
- .defaultValue(new HashMap<String,String>())
- .withDescription("The properties of knowledgebase");
+ .defaultValue(new HashMap<String, String>())
+ .withDescription("The properties of knowledge base");
public static final Option<String> KNOWLEDGE_BASE_NAME =
Options.key("name")
.stringType()
.defaultValue("")
- .withDescription("The name of knowledgebase.");
+ .withDescription("The name of knowledge base.");
public static final Option<String> KNOWLEDGE_BASE_FS_TYPE =
Options.key("fs_type")
.stringType()
.defaultValue("")
- .withDescription("The filesystem type of knowledgebase.");
+ .withDescription("The filesystem type of knowledge base.");
public static final Option<String> KNOWLEDGE_BASE_FS_PATH =
Options.key("fs_path")
.stringType()
.defaultValue("")
- .withDescription("The filesystem path of knowledgebase.");
+ .withDescription("The filesystem path of knowledge base.");
public static final Option<List<String>> KNOWLEDGE_BASE_FILES =
Options.key("files")
.listType()
.defaultValue(new ArrayList<String>())
- .withDescription("The files of knowledgebase.");
+ .withDescription("The files of knowledge base.");
public static final Option<String> KNOWLEDGE_BASE_STORAGE_FS_TYPE = Options.key("fs_type")
.stringType()
@@ -47,7 +47,8 @@ public class CommonConfigOptions {
public static final Option<List<KnowledgeBaseConfig>> KNOWLEDGE_BASE =
Options.key("knowledge_base")
- .type(new TypeReference<List<KnowledgeBaseConfig>>() {})
+ .type(new TypeReference<List<KnowledgeBaseConfig>>() {
+ })
.noDefaultValue()
.withDescription("The knowledge base configuration.");
@@ -55,13 +56,68 @@ public class CommonConfigOptions {
Options.key("properties")
.mapType()
.noDefaultValue()
- .withDescription("The general properties of grootstream");
+ .withDescription("The general properties of groot stream");
- public static final Option<String> ZOOKEEPER_QUORUM =
- Options.key("quorum")
- .stringType()
- .defaultValue("")
- .withDescription("The quorum of zookeeper.");
+ public static final Option<Map<String, KmsConfig>> KMS =
+ Options.key("kms")
+ .type(new TypeReference<Map<String, KmsConfig>>() {
+ })
+ .noDefaultValue()
+ .withDescription("The kms configuration.");
+
+ public static final Option<String> KMS_TYPE = Options.key("type")
+ .stringType()
+ .defaultValue("local")
+ .withDescription("The type of KMS.");
+ public static final Option<String> KMS_URL = Options.key("url")
+ .stringType()
+ .defaultValue("")
+ .withDescription("The access url of KMS.");
+ public static final Option<String> KMS_USERNAME = Options.key("username")
+ .stringType()
+ .defaultValue("")
+ .withDescription("The access username of KMS.");
+
+ public static final Option<String> KMS_PASSWORD = Options.key("password")
+ .stringType()
+ .defaultValue("")
+ .withDescription("The access username of KMS.");
+
+ public static final Option<String> KMS_DEFAULT_KEY_PATH = Options.key("default_key_path")
+ .stringType()
+ .defaultValue("")
+ .withDescription("The default key path of KMS.");
+
+ public static final Option<String> KMS_PLUGIN_KEY_PATH = Options.key("plugin_key_path")
+ .stringType()
+ .defaultValue("")
+ .withDescription("The plugin key path of KMS.");
+
+ public static final Option<SSLConfig> SSL = Options.key("ssl")
+ .type(new TypeReference<SSLConfig>() {
+ })
+ .noDefaultValue()
+ .withDescription("The ssl configuration.");
+
+ public static final Option<Boolean> SKIP_VERIFICATION = Options.key("skip_verification")
+ .booleanType()
+ .defaultValue(false)
+ .withDescription("The skip certificate of the configuration.");
+
+ public static final Option<String> CA_CERTIFICATE_PATH = Options.key("ca_certificate_path")
+ .stringType()
+ .defaultValue("")
+ .withDescription("The ca certificate file path of the configuration.");
+
+ public static final Option<String> CERTIFICATE_PATH = Options.key("certificate_path")
+ .stringType()
+ .defaultValue("")
+ .withDescription("The certificate file path of the configuration.");
+
+ public static final Option<String> PRIVATE_KEY_PATH = Options.key("private_key_path")
+ .stringType()
+ .defaultValue("")
+ .withDescription("The private key file path of the configuration.");
}
diff --git a/groot-common/src/main/java/com/geedgenetworks/common/config/KmsConfig.java b/groot-common/src/main/java/com/geedgenetworks/common/config/KmsConfig.java
new file mode 100644
index 0000000..f0e213f
--- /dev/null
+++ b/groot-common/src/main/java/com/geedgenetworks/common/config/KmsConfig.java
@@ -0,0 +1,15 @@
+package com.geedgenetworks.common.config;
+
+import lombok.Data;
+
+import java.io.Serializable;
+
+@Data
+public class KmsConfig implements Serializable {
+ private String type = CommonConfigOptions.KMS_TYPE.defaultValue();
+ private String url = CommonConfigOptions.KMS_URL.defaultValue();
+ private String username = CommonConfigOptions.KMS_USERNAME.defaultValue();
+ private String password = CommonConfigOptions.KMS_PASSWORD.defaultValue();
+ private String defaultKeyPath = CommonConfigOptions.KMS_DEFAULT_KEY_PATH.defaultValue();
+ private String pluginKeyPath = CommonConfigOptions.KMS_PLUGIN_KEY_PATH.defaultValue();
+}
diff --git a/groot-common/src/main/java/com/geedgenetworks/common/config/KnowledgeBaseConfig.java b/groot-common/src/main/java/com/geedgenetworks/common/config/KnowledgeBaseConfig.java
index b8e0160..baf4aee 100644
--- a/groot-common/src/main/java/com/geedgenetworks/common/config/KnowledgeBaseConfig.java
+++ b/groot-common/src/main/java/com/geedgenetworks/common/config/KnowledgeBaseConfig.java
@@ -1,15 +1,10 @@
package com.geedgenetworks.common.config;
-
-import com.geedgenetworks.utils.StringUtil;
import lombok.Data;
import java.io.Serializable;
-import java.util.Arrays;
import java.util.List;
import java.util.Map;
-import static com.google.common.base.Preconditions.checkArgument;
-
@Data
public class KnowledgeBaseConfig implements Serializable {
private String name = CommonConfigOptions.KNOWLEDGE_BASE_NAME.defaultValue();
@@ -18,18 +13,4 @@ public class KnowledgeBaseConfig implements Serializable {
private Map<String, String> properties = CommonConfigOptions.KNOWLEDGE_BASE_PROPERTIES.defaultValue();
private List<String> files = CommonConfigOptions.KNOWLEDGE_BASE_FILES.defaultValue();
- public void setFsType(String fsType) {
- this.fsType = fsType;
- }
-
- public void setFsPath(String fsPath) {
- this.fsPath = fsPath;
- }
-
- public void setFiles(List<String> files) {
- this.files = files;
- }
-
-
-
}
diff --git a/groot-common/src/main/java/com/geedgenetworks/common/config/SSLConfig.java b/groot-common/src/main/java/com/geedgenetworks/common/config/SSLConfig.java
new file mode 100644
index 0000000..874c163
--- /dev/null
+++ b/groot-common/src/main/java/com/geedgenetworks/common/config/SSLConfig.java
@@ -0,0 +1,16 @@
+package com.geedgenetworks.common.config;
+
+import lombok.Data;
+
+import java.io.Serializable;
+
+@Data
+public class SSLConfig implements Serializable {
+ private Boolean skipVerification = CommonConfigOptions.SKIP_VERIFICATION.defaultValue();
+
+ private String caCertificatePath = CommonConfigOptions.CA_CERTIFICATE_PATH.defaultValue();
+
+ private String certificatePath = CommonConfigOptions.CERTIFICATE_PATH.defaultValue();
+
+ private String privateKeyPath = CommonConfigOptions.PRIVATE_KEY_PATH.defaultValue();
+}
diff --git a/groot-common/src/main/java/com/geedgenetworks/common/config/UDFContextConfigOptions.java b/groot-common/src/main/java/com/geedgenetworks/common/config/UDFContextConfigOptions.java
new file mode 100644
index 0000000..ac36b02
--- /dev/null
+++ b/groot-common/src/main/java/com/geedgenetworks/common/config/UDFContextConfigOptions.java
@@ -0,0 +1,54 @@
+package com.geedgenetworks.common.config;
+
+import java.util.List;
+import java.util.Map;
+import com.alibaba.fastjson2.TypeReference;
+public interface UDFContextConfigOptions {
+ Option<String> NAME = Options.key("name")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("The name of the function.");
+
+ Option<List<String>> LOOKUP_FIELDS = Options.key("lookup_fields")
+ .listType()
+ .noDefaultValue()
+ .withDescription("The fields to be looked up.");
+
+ Option<List<String>> OUTPUT_FIELDS = Options.key("output_fields")
+ .listType()
+ .noDefaultValue()
+ .withDescription("The fields to be outputted.");
+
+ Option<String> FILTER = Options.key("filter")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("The filter expression to be applied.");
+
+ Option<Map<String, Object>> PARAMETERS = Options.key("parameters")
+ .type(new TypeReference<Map<String, Object>>() {})
+ .noDefaultValue()
+ .withDescription("The parameters for the function.");
+
+ Option<String> PARAMETERS_KB_NAME = Options.key("kb_name")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("The name of the knowledge base.");
+
+ Option<String> PARAMETERS_OPTION = Options.key("option")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("The option for the function.");
+
+ Option<Map<String, String>> PARAMETERS_GEOLOCATION_FIELD_MAPPING = Options.key("geolocation_field_mapping")
+ .mapType()
+ .noDefaultValue()
+ .withDescription("The geolocation field mapping.");
+
+
+ Option<String> FUNCTION = Options.key("function")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("The function to be executed.");
+
+
+}
diff --git a/groot-common/src/main/java/com/geedgenetworks/common/config/ConfigShade.java b/groot-common/src/main/java/com/geedgenetworks/common/crypto/CryptoShade.java
index 2943bc8..985f4df 100644
--- a/groot-common/src/main/java/com/geedgenetworks/common/config/ConfigShade.java
+++ b/groot-common/src/main/java/com/geedgenetworks/common/crypto/CryptoShade.java
@@ -1,10 +1,10 @@
-package com.geedgenetworks.common.config;
+package com.geedgenetworks.common.crypto;
-public interface ConfigShade {
+public interface CryptoShade {
/**
* The unique identifier of the current interface, used it to select the correct {@link
- * ConfigShade}
+ * CryptoShade}
*/
String getIdentifier();
diff --git a/groot-common/src/main/java/com/geedgenetworks/common/exception/CommonErrorCode.java b/groot-common/src/main/java/com/geedgenetworks/common/exception/CommonErrorCode.java
index e4d9f59..5298810 100644
--- a/groot-common/src/main/java/com/geedgenetworks/common/exception/CommonErrorCode.java
+++ b/groot-common/src/main/java/com/geedgenetworks/common/exception/CommonErrorCode.java
@@ -2,12 +2,13 @@ package com.geedgenetworks.common.exception;
public enum CommonErrorCode implements GrootStreamErrorCodeSupplier {
- UNSUPPORTED_OPERATION("GROOT-STREAM-COMMON-0001", "Unsupported operation exception"),
- ILLEGAL_ARGUMENT("GROOT-STREAM-COMMON-0002", "Illegal argument exception"),
- SYNTAX_ERROR("GROOT-STREAM-COMMON-0003", "Syntax Error"),
- FILE_OPERATION_ERROR("GROOT-STREAM-COMMON-0004", "File operation failed, such as (read,list,write,move,copy,sync) etc..."),
-
- CONFIG_VALIDATION_FAILED("GROOT-STREAM-COMMON-0005", "Configuration item validate failed"),
+ UNSUPPORTED_OPERATION("GROOT-STREAM-COMMON-0001", "Unsupported operation."),
+ ILLEGAL_ARGUMENT("GROOT-STREAM-COMMON-0002", "Illegal argument."),
+ SYNTAX_ERROR("GROOT-STREAM-COMMON-0003", "Syntax error."),
+ FILE_OPERATION_ERROR("GROOT-STREAM-COMMON-0004", "File operation failed (e.g., read, list, write, move, copy, sync)."),
+ CONFIG_VALIDATION_FAILED("GROOT-STREAM-COMMON-0005", "Configuration item validation failed."),
+ JSON_OPERATION_FAILED(
+ "GROOT-STREAM-COMMON-0006", "JSON convert/parse operation failed."),
;
private final String code;
diff --git a/groot-common/src/main/java/com/geedgenetworks/common/udf/RuleContext.java b/groot-common/src/main/java/com/geedgenetworks/common/udf/RuleContext.java
index ead0ecd..6aa9e3d 100644
--- a/groot-common/src/main/java/com/geedgenetworks/common/udf/RuleContext.java
+++ b/groot-common/src/main/java/com/geedgenetworks/common/udf/RuleContext.java
@@ -10,7 +10,7 @@ import java.io.Serializable;
@Data
public class RuleContext implements Serializable {
- private String name;
+ private String tag;
private String expression;
private Expression compiledExpression;
private OutputTag<Event> outputTag ;
diff --git a/groot-common/src/main/java/com/geedgenetworks/common/udf/ScalarFunction.java b/groot-common/src/main/java/com/geedgenetworks/common/udf/ScalarFunction.java
index 2aab34b..2723652 100644
--- a/groot-common/src/main/java/com/geedgenetworks/common/udf/ScalarFunction.java
+++ b/groot-common/src/main/java/com/geedgenetworks/common/udf/ScalarFunction.java
@@ -1,5 +1,9 @@
package com.geedgenetworks.common.udf;
import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.config.CheckUDFContextUtil;
+import com.geedgenetworks.common.config.UDFContextConfigOptions;
+import com.geedgenetworks.common.exception.CommonErrorCode;
+import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
import org.apache.flink.api.common.functions.RuntimeContext;
import java.io.Serializable;
@@ -13,4 +17,15 @@ public interface ScalarFunction extends Serializable {
void close();
+ default void checkConfig(UDFContext udfContext) {
+ if (udfContext == null) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "UDFContext cannot be null");
+ }
+
+ if (!CheckUDFContextUtil.checkAtLeastOneExists(udfContext, UDFContextConfigOptions.LOOKUP_FIELDS.key(), UDFContextConfigOptions.OUTPUT_FIELDS.key(), UDFContextConfigOptions.FILTER.key()).isSuccess()) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "At least one of the config should be specified.");
+ }
+
+ }
+
}
diff --git a/groot-common/src/main/java/com/geedgenetworks/common/udf/UDFContext.java b/groot-common/src/main/java/com/geedgenetworks/common/udf/UDFContext.java
index 4062924..ea98226 100644
--- a/groot-common/src/main/java/com/geedgenetworks/common/udf/UDFContext.java
+++ b/groot-common/src/main/java/com/geedgenetworks/common/udf/UDFContext.java
@@ -1,63 +1,22 @@
package com.geedgenetworks.common.udf;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import lombok.Data;
+
import java.io.Serializable;
import java.util.List;
import java.util.Map;
+@Data
public class UDFContext implements Serializable {
private String name;
- private List<String> lookup_fields;
- private List<String> output_fields;
+ @JsonProperty("lookup_fields")
+ private List<String> lookupFields;
+ @JsonProperty("output_fields")
+ private List<String> outputFields;
private String filter;
private Map<String, Object> parameters;
private String function;
- public String getName() {
- return name;
- }
-
- public void setName(String name) {
- this.name = name;
- }
-
- public List<String> getLookup_fields() {
- return lookup_fields;
- }
-
- public void setLookup_fields(List<String> lookup_fields) {
- this.lookup_fields = lookup_fields;
- }
-
- public List<String> getOutput_fields() {
- return output_fields;
- }
-
- public void setOutput_fields(List<String> output_fields) {
- this.output_fields = output_fields;
- }
-
- public String getFilter() {
- return filter;
- }
-
- public void setFilter(String filter) {
- this.filter = filter;
- }
-
- public Map<String, Object> getParameters() {
- return parameters;
- }
-
- public void setParameters(Map<String, Object> parameters) {
- this.parameters = parameters;
- }
-
- public String getFunction() {
- return function;
- }
-
- public void setFunction(String function) {
- this.function = function;
- }
}
diff --git a/groot-common/src/main/resources/grootstream.yaml b/groot-common/src/main/resources/grootstream.yaml
index 1a9a974..26752e3 100644
--- a/groot-common/src/main/resources/grootstream.yaml
+++ b/groot-common/src/main/resources/grootstream.yaml
@@ -11,6 +11,24 @@ grootstream:
files:
- 64af7077-eb9b-4b8f-80cf-2ceebc89bea9
- 004390bc-3135-4a6f-a492-3662ecb9e289
+
+ kms:
+ local:
+ type: local
+ vault:
+ type: vault
+ url: https://192.168.40.223:8200
+ username: tsg_olap
+ password: tsg_olap
+ default_key_path: tsg_olap/transit
+ plugin_key_path: tsg_olap/plugin/gmsm
+
+ ssl:
+ skip_verification: true
+ ca_certificate_path: ./config/ssl/root.pem
+ certificate_path: ./config/ssl/worker.pem
+ private_key_path: ./config/ssl/worker.key
+
properties:
hos.path: http://192.168.44.12:9098/hos
hos.bucket.name.traffic_file: traffic_file_bucket
diff --git a/groot-common/src/main/resources/udf.plugins b/groot-common/src/main/resources/udf.plugins
index 7544cc7..3d6a353 100644
--- a/groot-common/src/main/resources/udf.plugins
+++ b/groot-common/src/main/resources/udf.plugins
@@ -3,17 +3,23 @@ com.geedgenetworks.core.udf.CurrentUnixTimestamp
com.geedgenetworks.core.udf.DecodeBase64
com.geedgenetworks.core.udf.Domain
com.geedgenetworks.core.udf.Drop
+com.geedgenetworks.core.udf.EncodeBase64
+com.geedgenetworks.core.udf.Encrypt
com.geedgenetworks.core.udf.Eval
+com.geedgenetworks.core.udf.Flatten
com.geedgenetworks.core.udf.FromUnixTimestamp
com.geedgenetworks.core.udf.GenerateStringArray
com.geedgenetworks.core.udf.GeoIpLookup
+com.geedgenetworks.core.udf.Hmac
com.geedgenetworks.core.udf.JsonExtract
com.geedgenetworks.core.udf.PathCombine
com.geedgenetworks.core.udf.Rename
com.geedgenetworks.core.udf.SnowflakeId
com.geedgenetworks.core.udf.StringJoiner
com.geedgenetworks.core.udf.UnixTimestampConverter
-com.geedgenetworks.core.udf.Flatten
+com.geedgenetworks.core.udf.uuid.UUID
+com.geedgenetworks.core.udf.uuid.UUIDv5
+com.geedgenetworks.core.udf.uuid.UUIDv7
com.geedgenetworks.core.udf.udaf.NumberSum
com.geedgenetworks.core.udf.udaf.CollectList
com.geedgenetworks.core.udf.udaf.CollectSet
@@ -27,4 +33,7 @@ com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogram
com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogramQuantile
com.geedgenetworks.core.udf.udaf.HdrHistogram.HdrHistogramQuantiles
com.geedgenetworks.core.udf.udtf.JsonUnroll
-com.geedgenetworks.core.udf.udtf.Unroll \ No newline at end of file
+com.geedgenetworks.core.udf.udtf.Unroll
+com.geedgenetworks.core.udf.udtf.PathUnroll
+com.geedgenetworks.core.udf.udaf.Max
+com.geedgenetworks.core.udf.udaf.Min \ No newline at end of file
diff --git a/groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/EventKafkaDeserializationSchema.java b/groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/EventKafkaDeserializationSchema.java
index aae6678..bd95dd6 100644
--- a/groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/EventKafkaDeserializationSchema.java
+++ b/groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/EventKafkaDeserializationSchema.java
@@ -9,9 +9,15 @@ import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema;
import org.apache.flink.streaming.connectors.kafka.RuntimeContextAware;
import org.apache.flink.util.Collector;
import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.header.Header;
+import org.apache.kafka.common.header.Headers;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
public class EventKafkaDeserializationSchema implements KafkaDeserializationSchema<Event>, RuntimeContextAware {
private static final Logger LOG = LoggerFactory.getLogger(EventKafkaDeserializationSchema.class);
private final DeserializationSchema<Event> valueDeserialization;
@@ -46,6 +52,12 @@ public class EventKafkaDeserializationSchema implements KafkaDeserializationSche
Event event = valueDeserialization.deserialize(record.value());
if(event != null){
event.getExtractedFields().put(Event.INTERNAL_TIMESTAMP_KEY, record.timestamp());
+ Headers headers = record.headers();
+ Map<String, String> headersMap = new HashMap<>();
+ for (Header header : headers) {
+ headersMap.put(header.key(), new String(header.value(), StandardCharsets.UTF_8));
+ }
+ event.getExtractedFields().put(Event.INTERNAL_HEADERS_KEY, headersMap);
out.collect(event);
internalMetrics.incrementOutEvents();
return;
diff --git a/groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/KafkaConnectorOptionsUtil.java b/groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/KafkaConnectorOptionsUtil.java
index 48734ea..7924f8d 100644
--- a/groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/KafkaConnectorOptionsUtil.java
+++ b/groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/KafkaConnectorOptionsUtil.java
@@ -1,11 +1,13 @@
package com.geedgenetworks.connectors.kafka;
+import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
public class KafkaConnectorOptionsUtil {
public static final String PROPERTIES_PREFIX = "kafka.";
+ public static final String HEADERS_PREFIX = "headers.";
public static Properties getKafkaProperties(Map<String, String> tableOptions) {
final Properties kafkaProperties = new Properties();
@@ -23,6 +25,21 @@ public class KafkaConnectorOptionsUtil {
return kafkaProperties;
}
+ public static Map<String, String> getKafkaHeaders(Map<String, String> tableOptions) {
+ final Map<String, String> headers = new HashMap<>();
+
+ tableOptions.keySet().stream()
+ .filter(key -> key.startsWith(HEADERS_PREFIX))
+ .forEach(
+ key -> {
+ final String value = tableOptions.get(key);
+ final String subKey = key.substring((HEADERS_PREFIX).length());
+ headers.put(subKey, value);
+ });
+
+ return headers;
+ }
+
/**
* Decides if the table options contains Kafka client properties that start with prefix
* 'properties'.
diff --git a/groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/KafkaSinkProvider.java b/groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/KafkaSinkProvider.java
index 964aa94..496e6a3 100644
--- a/groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/KafkaSinkProvider.java
+++ b/groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/KafkaSinkProvider.java
@@ -10,6 +10,7 @@ import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSink;
import org.apache.flink.streaming.connectors.kafka.GrootFlinkKafkaProducer;
+import java.util.Map;
import java.util.Optional;
import java.util.Properties;
@@ -19,6 +20,7 @@ public class KafkaSinkProvider implements SinkProvider {
private final String topic;
private final Properties properties;
+ private final Map<String, String> headers;
private final boolean logFailuresOnly;
private final RateLimitingStrategy rateLimitingStrategy;
@@ -27,6 +29,7 @@ public class KafkaSinkProvider implements SinkProvider {
EncodingFormat valueEncodingFormat,
String topic,
Properties properties,
+ Map<String, String> headers,
boolean logFailuresOnly,
RateLimitingStrategy rateLimitingStrategy
) {
@@ -34,6 +37,7 @@ public class KafkaSinkProvider implements SinkProvider {
this.valueSerialization = valueEncodingFormat.createRuntimeEncoder(dataType);
this.topic = topic;
this.properties = properties;
+ this.headers = headers;
this.logFailuresOnly = logFailuresOnly;
this.rateLimitingStrategy = rateLimitingStrategy;
}
@@ -48,6 +52,7 @@ public class KafkaSinkProvider implements SinkProvider {
);
kafkaProducer.setLogFailuresOnly(logFailuresOnly);
kafkaProducer.setRateLimitingStrategy(rateLimitingStrategy);
+ kafkaProducer.setHeaders(headers);
return dataStream.addSink(kafkaProducer);
}
}
diff --git a/groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/KafkaTableFactory.java b/groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/KafkaTableFactory.java
index fbbaed2..394e618 100644
--- a/groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/KafkaTableFactory.java
+++ b/groot-connectors/connector-kafka/src/main/java/com/geedgenetworks/connectors/kafka/KafkaTableFactory.java
@@ -18,8 +18,7 @@ import org.apache.flink.util.Preconditions;
import java.util.*;
import static com.geedgenetworks.connectors.kafka.KafkaConnectorOptions.*;
-import static com.geedgenetworks.connectors.kafka.KafkaConnectorOptionsUtil.PROPERTIES_PREFIX;
-import static com.geedgenetworks.connectors.kafka.KafkaConnectorOptionsUtil.getKafkaProperties;
+import static com.geedgenetworks.connectors.kafka.KafkaConnectorOptionsUtil.*;
public class KafkaTableFactory implements SourceTableFactory, SinkTableFactory {
public static final String IDENTIFIER = "kafka";
@@ -51,7 +50,7 @@ public class KafkaTableFactory implements SourceTableFactory, SinkTableFactory {
// 获取valueEncodingFormat
EncodingFormat valueEncodingFormat = helper.discoverEncodingFormat(EncodingFormatFactory.class, FactoryUtil.FORMAT);
- helper.validateExcept(PROPERTIES_PREFIX); // 校验参数,排除properties.参数
+ helper.validateExcept(PROPERTIES_PREFIX, HEADERS_PREFIX); // 校验参数,排除properties.参数
StructType dataType = context.getDataType();
ReadableConfig config = context.getConfiguration();
@@ -59,8 +58,9 @@ public class KafkaTableFactory implements SourceTableFactory, SinkTableFactory {
String topic = config.get(TOPIC).get(0);
boolean logFailuresOnly = config.get(LOG_FAILURES_ONLY);
final Properties properties = getKafkaProperties(context.getOptions());
+ Map<String, String> headers = getKafkaHeaders(context.getOptions());
- return new KafkaSinkProvider(dataType, valueEncodingFormat, topic, properties, logFailuresOnly, getRateLimitingStrategy(config));
+ return new KafkaSinkProvider(dataType, valueEncodingFormat, topic, properties, headers, logFailuresOnly, getRateLimitingStrategy(config));
}
@Override
diff --git a/groot-connectors/connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/GrootFlinkKafkaProducer.java b/groot-connectors/connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/GrootFlinkKafkaProducer.java
index 09e8190..3b7e0c5 100644
--- a/groot-connectors/connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/GrootFlinkKafkaProducer.java
+++ b/groot-connectors/connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/GrootFlinkKafkaProducer.java
@@ -61,12 +61,15 @@ import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.errors.InvalidTxnStateException;
import org.apache.kafka.common.errors.ProducerFencedException;
+import org.apache.kafka.common.header.Header;
+import org.apache.kafka.common.header.internals.RecordHeader;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.Nullable;
import java.io.IOException;
+import java.nio.charset.StandardCharsets;
import java.time.Duration;
import java.util.*;
import java.util.concurrent.BlockingDeque;
@@ -257,6 +260,8 @@ public class GrootFlinkKafkaProducer<IN>
private final Map<String, KafkaMetricMutableWrapper> previouslyCreatedMetrics = new HashMap<>();
private RateLimitingStrategy rateLimitingStrategy;
private boolean rateLimitingEnable;
+ private Map<String, String> headers;
+ private Header[] recordHeaders;
private transient InternalMetrics internalMetrics;
@@ -769,6 +774,10 @@ public class GrootFlinkKafkaProducer<IN>
this.rateLimitingEnable = rateLimitingStrategy != null && rateLimitingStrategy.rateLimited();
}
+ public void setHeaders(Map<String, String> headers) {
+ this.headers = headers;
+ }
+
/**
* Disables the propagation of exceptions thrown when committing presumably timed out Kafka
* transactions during recovery of the job. If a Kafka transaction is timed out, a commit will
@@ -796,6 +805,17 @@ public class GrootFlinkKafkaProducer<IN>
rateLimitingStrategy = rateLimitingStrategy.withMaxRate(subTaskMaxRate);
LOG.error("rateLimitingStrategy: {}", rateLimitingStrategy);
}
+ if(headers != null && !headers.isEmpty()){
+ recordHeaders = new Header[headers.size()];
+ int i = 0;
+ for (Map.Entry<String, String> entry : headers.entrySet()) {
+ recordHeaders[i++] = new RecordHeader(entry.getKey(), entry.getValue().getBytes(StandardCharsets.UTF_8));
+ }
+ checkState(i == headers.size());
+ }else{
+ recordHeaders = new Header[0];
+ }
+
internalMetrics = new InternalMetrics(getRuntimeContext());
if (logFailuresOnly) {
callback =
@@ -929,6 +949,9 @@ public class GrootFlinkKafkaProducer<IN>
record = kafkaSchema.serialize(next, context.timestamp());
}
+ for (int i = 0; i < recordHeaders.length; i++) {
+ record.headers().add(recordHeaders[i]);
+ }
if (!rateLimitingEnable) {
pendingRecords.incrementAndGet();
transaction.producer.send(record, callback);
diff --git a/groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/FakerUtils.java b/groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/FakerUtils.java
index 5101fa1..0a36100 100644
--- a/groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/FakerUtils.java
+++ b/groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/FakerUtils.java
@@ -165,7 +165,8 @@ public class FakerUtils {
private static Faker<?> parseSequenceFaker(JSONObject obj) {
long start = obj.getLongValue("start", 0L);
long step = obj.getLongValue("step", 1L);
- return new SequenceFaker(start, step);
+ int batch = obj.getIntValue("batch", 1);
+ return new SequenceFaker(start, step, batch);
}
private static Faker<?> parseStringFaker(JSONObject obj) {
diff --git a/groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/SequenceFaker.java b/groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/SequenceFaker.java
index 0005234..867f138 100644
--- a/groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/SequenceFaker.java
+++ b/groot-connectors/connector-mock/src/main/java/com/geedgenetworks/connectors/mock/faker/SequenceFaker.java
@@ -3,22 +3,34 @@ package com.geedgenetworks.connectors.mock.faker;
public class SequenceFaker extends Faker<Long> {
private final long start;
private final long step;
+ private final int batch;
private long value;
+ private int cnt;
public SequenceFaker(long start) {
- this(start, 1);
+ this(start, 1, 1);
}
public SequenceFaker(long start, long step) {
+ this(start, step, 1);
+ }
+
+ public SequenceFaker(long start, long step, int batch) {
this.start = start;
this.step = step;
+ this.batch = batch;
this.value = start;
+ this.cnt = 0;
}
@Override
public Long geneValue() throws Exception {
Long rst = value;
- value += step;
+ cnt++;
+ if(cnt == batch){
+ cnt = 0;
+ value += step;
+ }
return rst;
}
diff --git a/groot-connectors/connector-starrocks/pom.xml b/groot-connectors/connector-starrocks/pom.xml
new file mode 100644
index 0000000..095ee6d
--- /dev/null
+++ b/groot-connectors/connector-starrocks/pom.xml
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>com.geedgenetworks</groupId>
+ <artifactId>groot-connectors</artifactId>
+ <version>${revision}</version>
+ </parent>
+
+ <artifactId>connector-starrocks</artifactId>
+ <name>Groot : Connectors : StarRocks </name>
+
+ <dependencies>
+ <dependency>
+ <groupId>com.starrocks</groupId>
+ <artifactId>flink-connector-starrocks</artifactId>
+ <version>1.2.9_flink-1.13_2.12</version>
+ </dependency>
+ </dependencies>
+
+</project> \ No newline at end of file
diff --git a/groot-connectors/connector-starrocks/src/main/java/com/geedgenetworks/connectors/starrocks/StarRocksTableFactory.java b/groot-connectors/connector-starrocks/src/main/java/com/geedgenetworks/connectors/starrocks/StarRocksTableFactory.java
new file mode 100644
index 0000000..fc41481
--- /dev/null
+++ b/groot-connectors/connector-starrocks/src/main/java/com/geedgenetworks/connectors/starrocks/StarRocksTableFactory.java
@@ -0,0 +1,85 @@
+package com.geedgenetworks.connectors.starrocks;
+
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.core.connector.sink.SinkProvider;
+import com.geedgenetworks.core.factories.FactoryUtil;
+import com.geedgenetworks.core.factories.SinkTableFactory;
+import com.starrocks.connector.flink.table.sink.EventStarRocksDynamicSinkFunctionV2;
+import com.starrocks.connector.flink.table.sink.SinkFunctionFactory;
+import com.starrocks.connector.flink.table.sink.StarRocksSinkOptions;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.util.Preconditions;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public class StarRocksTableFactory implements SinkTableFactory {
+ public static final String IDENTIFIER = "starrocks";
+ @Override
+ public String factoryIdentifier() {
+ return IDENTIFIER;
+ }
+
+ @Override
+ public SinkProvider getSinkProvider(Context context) {
+ final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
+ helper.validateExcept(CONNECTION_INFO_PREFIX);
+
+ final boolean logFailuresOnly = context.getConfiguration().get(LOG_FAILURES_ONLY);
+ StarRocksSinkOptions.Builder builder = StarRocksSinkOptions.builder();
+ context.getOptions().forEach((key, value) -> {
+ if(key.startsWith(CONNECTION_INFO_PREFIX)){
+ builder.withProperty(key.substring(CONNECTION_INFO_PREFIX.length()), value);
+ }
+ });
+ builder.withProperty("sink.properties.format", "json");
+ final StarRocksSinkOptions options = builder.build();
+ SinkFunctionFactory.detectStarRocksFeature(options);
+ Preconditions.checkArgument(options.isSupportTransactionStreamLoad());
+ final SinkFunction<Event> sinkFunction = new EventStarRocksDynamicSinkFunctionV2(options, logFailuresOnly);
+ return new SinkProvider() {
+ @Override
+ public DataStreamSink<?> consumeDataStream(DataStream<Event> dataStream) {
+ /*DataStream<String> ds = dataStream.flatMap(new FlatMapFunction<Event, String>() {
+ @Override
+ public void flatMap(Event value, Collector<String> out) throws Exception {
+ try {
+ out.collect(JSON.toJSONString(value.getExtractedFields()));
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ });
+ SinkFunction<String> sink = StarRocksSink.sink(options);
+ return ds.addSink(sink);
+ */
+ return dataStream.addSink(sinkFunction);
+ }
+ };
+ }
+
+ @Override
+ public Set<ConfigOption<?>> requiredOptions() {
+ return new HashSet<>();
+ }
+
+ @Override
+ public Set<ConfigOption<?>> optionalOptions() {
+ final Set<ConfigOption<?>> options = new HashSet<>();
+ options.add(LOG_FAILURES_ONLY);
+ return options;
+ }
+
+ public static final String CONNECTION_INFO_PREFIX = "connection.";
+
+ public static final ConfigOption<Boolean> LOG_FAILURES_ONLY =
+ ConfigOptions.key("log.failures.only")
+ .booleanType()
+ .defaultValue(true)
+ .withDescription("Optional flag to whether the sink should fail on errors, or only log them;\n"
+ + "If this is set to true, then exceptions will be only logged, if set to false, exceptions will be eventually thrown, true by default.");
+}
diff --git a/groot-connectors/connector-starrocks/src/main/java/com/starrocks/connector/flink/table/sink/EventStarRocksDynamicSinkFunctionV2.java b/groot-connectors/connector-starrocks/src/main/java/com/starrocks/connector/flink/table/sink/EventStarRocksDynamicSinkFunctionV2.java
new file mode 100644
index 0000000..71a9467
--- /dev/null
+++ b/groot-connectors/connector-starrocks/src/main/java/com/starrocks/connector/flink/table/sink/EventStarRocksDynamicSinkFunctionV2.java
@@ -0,0 +1,318 @@
+package com.starrocks.connector.flink.table.sink;
+
+import com.alibaba.fastjson2.JSON;
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.core.metrics.InternalMetrics;
+import com.starrocks.connector.flink.manager.StarRocksSinkBufferEntity;
+import com.starrocks.connector.flink.manager.StarRocksStreamLoadListener;
+import com.starrocks.connector.flink.tools.EnvUtils;
+import com.starrocks.connector.flink.tools.JsonWrapper;
+import com.starrocks.data.load.stream.LabelGeneratorFactory;
+import com.starrocks.data.load.stream.StreamLoadSnapshot;
+import com.starrocks.data.load.stream.properties.StreamLoadProperties;
+import com.starrocks.data.load.stream.v2.StreamLoadManagerV2;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.operators.util.SimpleVersionedListState;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+
+public class EventStarRocksDynamicSinkFunctionV2 extends StarRocksDynamicSinkFunctionBase<Event> {
+
+ private static final long serialVersionUID = 1L;
+ private static final Logger log = LoggerFactory.getLogger(StarRocksDynamicSinkFunctionV2.class);
+
+ private static final int NESTED_ROW_DATA_HEADER_SIZE = 256;
+
+ private final StarRocksSinkOptions sinkOptions;
+ private final boolean logFailuresOnly;
+ private final StreamLoadProperties properties;
+ private StreamLoadManagerV2 sinkManager;
+
+ private transient volatile ListState<StarrocksSnapshotState> snapshotStates;
+
+ private transient long restoredCheckpointId;
+
+ private transient List<ExactlyOnceLabelGeneratorSnapshot> restoredGeneratorSnapshots;
+
+ private transient Map<Long, List<StreamLoadSnapshot>> snapshotMap;
+
+ private transient StarRocksStreamLoadListener streamLoadListener;
+
+ // Only valid when using exactly-once and label prefix is set
+ @Nullable
+ private transient ExactlyOnceLabelGeneratorFactory exactlyOnceLabelFactory;
+
+ @Deprecated
+ private transient ListState<Map<String, StarRocksSinkBufferEntity>> legacyState;
+ @Deprecated
+ private transient List<StarRocksSinkBufferEntity> legacyData;
+ private transient JsonWrapper jsonWrapper;
+ private transient InternalMetrics internalMetrics;
+
+ public EventStarRocksDynamicSinkFunctionV2(StarRocksSinkOptions sinkOptions, boolean logFailuresOnly) {
+ Preconditions.checkArgument(sinkOptions.getSemantic() == StarRocksSinkSemantic.AT_LEAST_ONCE);
+ this.sinkOptions = sinkOptions;
+ this.logFailuresOnly = logFailuresOnly;
+ this.properties = sinkOptions.getProperties(null);
+ this.sinkManager = new StreamLoadManagerV2(sinkOptions.getProperties(null),
+ sinkOptions.getSemantic() == StarRocksSinkSemantic.AT_LEAST_ONCE);
+ }
+
+ @Override
+ public void invoke(Event value, Context context) throws Exception {
+ internalMetrics.incrementInEvents(1);
+ String json;
+ try {
+ json = JSON.toJSONString(value.getExtractedFields());
+ } catch (Exception e) {
+ internalMetrics.incrementErrorEvents(1);
+ log.error("json convert error", e);
+ return;
+ }
+ try {
+ sinkManager.write(null, sinkOptions.getDatabaseName(), sinkOptions.getTableName(), json);
+ } catch (Exception e) {
+ internalMetrics.incrementErrorEvents(1);
+ if (logFailuresOnly) {
+ log.error("write error", e);
+ resetSinkManager();
+ } else {
+ throw e;
+ }
+ }
+ }
+
+ private void resetSinkManager(){
+ try {
+ StreamLoadSnapshot snapshot = sinkManager.snapshot();
+ sinkManager.abort(snapshot);
+ } catch (Exception ex) {
+ log.error("write error", ex);
+ }
+ sinkManager.close();
+
+ this.sinkManager = new StreamLoadManagerV2(this.properties,
+ sinkOptions.getSemantic() == StarRocksSinkSemantic.AT_LEAST_ONCE);
+ sinkManager.setStreamLoadListener(streamLoadListener);
+
+ LabelGeneratorFactory labelGeneratorFactory;
+ String labelPrefix = sinkOptions.getLabelPrefix();
+ if (labelPrefix == null ||
+ sinkOptions.getSemantic() == StarRocksSinkSemantic.AT_LEAST_ONCE ||
+ !sinkOptions.isEnableExactlyOnceLabelGen()) {
+ labelGeneratorFactory = new LabelGeneratorFactory.DefaultLabelGeneratorFactory(
+ labelPrefix == null ? "flink" : labelPrefix);
+ } else {
+ labelGeneratorFactory = exactlyOnceLabelFactory;
+ }
+ sinkManager.setLabelGeneratorFactory(labelGeneratorFactory);
+
+ sinkManager.init();
+ }
+
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ internalMetrics = new InternalMetrics(getRuntimeContext());
+ this.streamLoadListener = new EventStreamLoadListener(getRuntimeContext(), sinkOptions, internalMetrics);
+ sinkManager.setStreamLoadListener(streamLoadListener);
+
+ LabelGeneratorFactory labelGeneratorFactory;
+ String labelPrefix = sinkOptions.getLabelPrefix();
+ if (labelPrefix == null ||
+ sinkOptions.getSemantic() == StarRocksSinkSemantic.AT_LEAST_ONCE ||
+ !sinkOptions.isEnableExactlyOnceLabelGen()) {
+ labelGeneratorFactory = new LabelGeneratorFactory.DefaultLabelGeneratorFactory(
+ labelPrefix == null ? "flink" : labelPrefix);
+ } else {
+ this.exactlyOnceLabelFactory = new ExactlyOnceLabelGeneratorFactory(
+ labelPrefix,
+ getRuntimeContext().getNumberOfParallelSubtasks(),
+ getRuntimeContext().getIndexOfThisSubtask(),
+ restoredCheckpointId);
+ exactlyOnceLabelFactory.restore(restoredGeneratorSnapshots);
+ labelGeneratorFactory = exactlyOnceLabelFactory;
+ }
+ sinkManager.setLabelGeneratorFactory(labelGeneratorFactory);
+
+ sinkManager.init();
+
+ if (sinkOptions.getSemantic() == StarRocksSinkSemantic.EXACTLY_ONCE) {
+ openForExactlyOnce();
+ }
+
+ log.info("Open sink function v2. {}", EnvUtils.getGitInformation());
+ }
+
+ private void openForExactlyOnce() throws Exception {
+ if (sinkOptions.isAbortLingeringTxns()) {
+ LingeringTransactionAborter aborter = new LingeringTransactionAborter(
+ sinkOptions.getLabelPrefix(),
+ restoredCheckpointId,
+ getRuntimeContext().getIndexOfThisSubtask(),
+ sinkOptions.getAbortCheckNumTxns(),
+ sinkOptions.getDbTables(),
+ restoredGeneratorSnapshots,
+ sinkManager.getStreamLoader());
+ aborter.execute();
+ }
+
+ notifyCheckpointComplete(Long.MAX_VALUE);
+ }
+
+ private JsonWrapper getOrCreateJsonWrapper() {
+ if (jsonWrapper == null) {
+ this.jsonWrapper = new JsonWrapper();
+ }
+
+ return jsonWrapper;
+ }
+
+ public void finish() {
+ sinkManager.flush();
+ }
+
+ @Override
+ public void close() {
+ log.info("Close sink function");
+ try {
+ sinkManager.flush();
+ } catch (Exception e) {
+ log.error("Failed to flush when closing", e);
+ throw e;
+ } finally {
+ StreamLoadSnapshot snapshot = sinkManager.snapshot();
+ sinkManager.abort(snapshot);
+ sinkManager.close();
+ }
+ }
+
+ @Override
+ public void snapshotState(FunctionSnapshotContext functionSnapshotContext) throws Exception {
+ sinkManager.flush();
+ if (sinkOptions.getSemantic() != StarRocksSinkSemantic.EXACTLY_ONCE) {
+ return;
+ }
+
+ StreamLoadSnapshot snapshot = sinkManager.snapshot();
+
+ if (sinkManager.prepare(snapshot)) {
+ snapshotMap.put(functionSnapshotContext.getCheckpointId(), Collections.singletonList(snapshot));
+
+ snapshotStates.clear();
+ List<ExactlyOnceLabelGeneratorSnapshot> labelSnapshots = exactlyOnceLabelFactory == null ? null
+ : exactlyOnceLabelFactory.snapshot(functionSnapshotContext.getCheckpointId());
+ snapshotStates.add(StarrocksSnapshotState.of(snapshotMap, labelSnapshots));
+ } else {
+ sinkManager.abort(snapshot);
+ throw new RuntimeException("Snapshot state failed by prepare");
+ }
+
+ if (legacyState != null) {
+ legacyState.clear();
+ }
+ }
+
+ @Override
+ public void initializeState(FunctionInitializationContext functionInitializationContext) throws Exception {
+ log.info("Initialize state");
+ if (sinkOptions.getSemantic() != StarRocksSinkSemantic.EXACTLY_ONCE) {
+ return;
+ }
+
+ ListStateDescriptor<byte[]> descriptor =
+ new ListStateDescriptor<>(
+ "starrocks-sink-transaction",
+ TypeInformation.of(new TypeHint<byte[]>() {})
+ );
+
+ ListState<byte[]> listState = functionInitializationContext.getOperatorStateStore().getListState(descriptor);
+ snapshotStates = new SimpleVersionedListState<>(listState, new StarRocksVersionedSerializer(getOrCreateJsonWrapper()));
+
+ // old version
+ ListStateDescriptor<Map<String, StarRocksSinkBufferEntity>> legacyDescriptor =
+ new ListStateDescriptor<>(
+ "buffered-rows",
+ TypeInformation.of(new TypeHint<Map<String, StarRocksSinkBufferEntity>>(){})
+ );
+ legacyState = functionInitializationContext.getOperatorStateStore().getListState(legacyDescriptor);
+ this.restoredCheckpointId = 0;
+ this.restoredGeneratorSnapshots = new ArrayList<>();
+ this.snapshotMap = new ConcurrentHashMap<>();
+ if (functionInitializationContext.isRestored()) {
+ for (StarrocksSnapshotState state : snapshotStates.get()) {
+ for (Map.Entry<Long, List<StreamLoadSnapshot>> entry : state.getData().entrySet()) {
+ snapshotMap.compute(entry.getKey(), (k, v) -> {
+ if (v == null) {
+ return new ArrayList<>(entry.getValue());
+ }
+ v.addAll(entry.getValue());
+ return v;
+ });
+ }
+
+ if (state.getLabelSnapshots() != null) {
+ List<ExactlyOnceLabelGeneratorSnapshot> labelSnapshots = state.getLabelSnapshots();
+ restoredGeneratorSnapshots.addAll(labelSnapshots);
+ long checkpointId = labelSnapshots.isEmpty() ? -1 : labelSnapshots.get(0).getCheckpointId();
+ restoredCheckpointId = Math.max(restoredCheckpointId, checkpointId);
+ }
+ }
+
+ legacyData = new ArrayList<>();
+ for (Map<String, StarRocksSinkBufferEntity> entry : legacyState.get()) {
+ legacyData.addAll(entry.values());
+ }
+ log.info("There are {} items from legacy state", legacyData.size());
+ }
+ }
+
+ @Override
+ public void notifyCheckpointComplete(long checkpointId) throws Exception {
+ if (sinkOptions.getSemantic() != StarRocksSinkSemantic.EXACTLY_ONCE) {
+ return;
+ }
+
+ boolean succeed = true;
+ List<Long> commitCheckpointIds = snapshotMap.keySet().stream()
+ .filter(cpId -> cpId <= checkpointId)
+ .sorted(Long::compare)
+ .collect(Collectors.toList());
+
+ for (Long cpId : commitCheckpointIds) {
+ try {
+ for (StreamLoadSnapshot snapshot : snapshotMap.get(cpId)) {
+ if (!sinkManager.commit(snapshot)) {
+ succeed = false;
+ break;
+ }
+ }
+
+ if (!succeed) {
+ throw new RuntimeException(String.format("Failed to commit some transactions for snapshot %s, " +
+ "please check taskmanager logs for details", cpId));
+ }
+ } catch (Exception e) {
+ log.error("Failed to notify checkpoint complete, checkpoint id : {}", checkpointId, e);
+ throw new RuntimeException("Failed to notify checkpoint complete for checkpoint id " + checkpointId, e);
+ }
+
+ snapshotMap.remove(cpId);
+ }
+
+ // set legacyState to null to avoid clear it in latter snapshotState
+ legacyState = null;
+ }
+
+} \ No newline at end of file
diff --git a/groot-connectors/connector-starrocks/src/main/java/com/starrocks/connector/flink/table/sink/EventStreamLoadListener.java b/groot-connectors/connector-starrocks/src/main/java/com/starrocks/connector/flink/table/sink/EventStreamLoadListener.java
new file mode 100644
index 0000000..337109b
--- /dev/null
+++ b/groot-connectors/connector-starrocks/src/main/java/com/starrocks/connector/flink/table/sink/EventStreamLoadListener.java
@@ -0,0 +1,28 @@
+package com.starrocks.connector.flink.table.sink;
+
+import com.geedgenetworks.core.metrics.InternalMetrics;
+import com.starrocks.connector.flink.manager.StarRocksStreamLoadListener;
+import com.starrocks.data.load.stream.StreamLoadResponse;
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+public class EventStreamLoadListener extends StarRocksStreamLoadListener {
+ private transient InternalMetrics internalMetrics;
+ public EventStreamLoadListener(RuntimeContext context, StarRocksSinkOptions sinkOptions, InternalMetrics internalMetrics) {
+ super(context, sinkOptions);
+ this.internalMetrics = internalMetrics;
+ }
+
+ @Override
+ public void flushSucceedRecord(StreamLoadResponse response) {
+ super.flushSucceedRecord(response);
+ if (response.getFlushRows() != null) {
+ internalMetrics.incrementOutEvents(response.getFlushRows());
+ }
+ }
+
+ @Override
+ public void flushFailedRecord() {
+ super.flushFailedRecord();
+ internalMetrics.incrementErrorEvents(1);
+ }
+}
diff --git a/groot-connectors/connector-starrocks/src/main/resources/META-INF/services/com.geedgenetworks.core.factories.Factory b/groot-connectors/connector-starrocks/src/main/resources/META-INF/services/com.geedgenetworks.core.factories.Factory
new file mode 100644
index 0000000..c04c5dc
--- /dev/null
+++ b/groot-connectors/connector-starrocks/src/main/resources/META-INF/services/com.geedgenetworks.core.factories.Factory
@@ -0,0 +1 @@
+com.geedgenetworks.connectors.starrocks.StarRocksTableFactory
diff --git a/groot-connectors/pom.xml b/groot-connectors/pom.xml
index 1747fb3..cf5381c 100644
--- a/groot-connectors/pom.xml
+++ b/groot-connectors/pom.xml
@@ -17,6 +17,7 @@
<module>connector-ipfix-collector</module>
<module>connector-file</module>
<module>connector-mock</module>
+ <module>connector-starrocks</module>
</modules>
<dependencies>
<dependency>
diff --git a/groot-core/pom.xml b/groot-core/pom.xml
index e723fa5..184e148 100644
--- a/groot-core/pom.xml
+++ b/groot-core/pom.xml
@@ -12,6 +12,21 @@
<name>Groot : Core </name>
<dependencies>
+ <dependency>
+ <groupId>com.typesafe</groupId>
+ <artifactId>config</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>com.fasterxml.uuid</groupId>
+ <artifactId>java-uuid-generator</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>com.uber</groupId>
+ <artifactId>h3</artifactId>
+ <version>4.1.1</version>
+ </dependency>
<dependency>
<groupId>org.mock-server</groupId>
@@ -109,6 +124,16 @@
<scope>provided</scope>
</dependency>
+ <dependency>
+ <groupId>io.github.jopenlibs</groupId>
+ <artifactId>vault-java-driver</artifactId>
+ <version>6.2.0</version>
+ </dependency>
+ <dependency>
+ <groupId>org.bouncycastle</groupId>
+ <artifactId>bcpkix-jdk18on</artifactId>
+ <version>1.78.1</version>
+ </dependency>
</dependencies>
<build>
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/connector/format/MapDeserialization.java b/groot-core/src/main/java/com/geedgenetworks/core/connector/format/MapDeserialization.java
new file mode 100644
index 0000000..7887097
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/connector/format/MapDeserialization.java
@@ -0,0 +1,8 @@
+package com.geedgenetworks.core.connector.format;
+
+import java.io.IOException;
+import java.util.Map;
+
+public interface MapDeserialization {
+ Map<String, Object> deserializeToMap(byte[] bytes) throws IOException;
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/factories/FactoryUtil.java b/groot-core/src/main/java/com/geedgenetworks/core/factories/FactoryUtil.java
index 6b93dab..a120ca5 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/factories/FactoryUtil.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/factories/FactoryUtil.java
@@ -159,6 +159,16 @@ public final class FactoryUtil {
return discoverFactory(Thread.currentThread().getContextClassLoader(), factoryClass, connector);
}
+ public static <T extends DecodingFormatFactory> T discoverDecodingFormatFactory(
+ Class<T> factoryClass, String type) {
+ return discoverFactory(Thread.currentThread().getContextClassLoader(), factoryClass, type);
+ }
+
+ public static <T extends EncodingFormatFactory> T discoverEncodingFormatFactory(
+ Class<T> factoryClass, String type) {
+ return discoverFactory(Thread.currentThread().getContextClassLoader(), factoryClass, type);
+ }
+
private static <T> T readOption(ReadableConfig options, ConfigOption<T> option) {
try {
return options.get(option);
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/filter/AviatorFilter.java b/groot-core/src/main/java/com/geedgenetworks/core/filter/AviatorFilter.java
index d8b8bc4..06693c9 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/filter/AviatorFilter.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/filter/AviatorFilter.java
@@ -1,11 +1,14 @@
package com.geedgenetworks.core.filter;
+import com.alibaba.fastjson.JSONObject;
import com.geedgenetworks.common.Event;
import com.geedgenetworks.core.pojo.FilterConfig;
+import com.typesafe.config.Config;
import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-public class AviatorFilter implements Filter {
+import java.util.Map;
+
+public class AviatorFilter implements Filter<FilterConfig> {
@Override
public DataStream<Event> filterFunction(
@@ -29,4 +32,12 @@ public class AviatorFilter implements Filter {
return "aviator";
}
+ @Override
+ public FilterConfig checkConfig(String name, Map<String, Object> configProperties, Config typeSafeConfig) {
+
+ FilterConfig filterConfig = new JSONObject(configProperties).toJavaObject(FilterConfig.class);
+ filterConfig.setName(name);
+ return filterConfig;
+ }
+
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/filter/Filter.java b/groot-core/src/main/java/com/geedgenetworks/core/filter/Filter.java
index f8b50eb..41daf3d 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/filter/Filter.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/filter/Filter.java
@@ -3,13 +3,20 @@ package com.geedgenetworks.core.filter;
import com.geedgenetworks.common.Event;
import com.geedgenetworks.core.pojo.FilterConfig;
+import com.typesafe.config.Config;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-public interface Filter {
+import java.io.Serializable;
+import java.util.Map;
+
+public interface Filter<T extends FilterConfig> extends Serializable {
DataStream<Event> filterFunction(
- DataStream<Event> singleOutputStreamOperator, FilterConfig FilterConfig)
+ DataStream<Event> singleOutputStreamOperator, T FilterConfig)
throws Exception;
String type();
+
+ T checkConfig(String name, Map<String, Object> configProperties, Config typeSafeConfig);
+
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/pojo/KmsKey.java b/groot-core/src/main/java/com/geedgenetworks/core/pojo/KmsKey.java
new file mode 100644
index 0000000..2690254
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/pojo/KmsKey.java
@@ -0,0 +1,19 @@
+package com.geedgenetworks.core.pojo;
+
+
+import lombok.Data;
+
+@Data
+public class KmsKey {
+
+ private byte[] keyData;
+ private int keyVersion;
+
+ public KmsKey() {
+ }
+
+ public KmsKey(byte[] keyData, int keyVersion) {
+ this.keyData = keyData;
+ this.keyVersion = keyVersion;
+ }
+} \ No newline at end of file
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/Processor.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/Processor.java
index 3852414..1c9ba6f 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/processor/Processor.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/Processor.java
@@ -1,15 +1,22 @@
package com.geedgenetworks.core.processor;
import com.geedgenetworks.common.Event;
+import com.geedgenetworks.core.pojo.ProcessorConfig;
+import com.typesafe.config.Config;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-public interface Processor<T> {
+import java.io.Serializable;
+import java.util.Map;
+
+public interface Processor<T extends ProcessorConfig> extends Serializable {
DataStream<Event> processorFunction(
DataStream<Event> singleOutputStreamOperator,
T processorConfig, ExecutionConfig config)
throws Exception;
String type();
+
+ T checkConfig(String name, Map<String, Object> configProperties, Config typeSafeConfig);
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AbstractFirstAggregation.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AbstractFirstAggregation.java
index 3632ba7..ce77ee8 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AbstractFirstAggregation.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AbstractFirstAggregation.java
@@ -6,6 +6,7 @@ import com.alibaba.fastjson.JSON;
import com.geedgenetworks.common.Accumulator;
import com.geedgenetworks.common.Constants;
import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.KeybyEntity;
import com.geedgenetworks.common.exception.CommonErrorCode;
import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
import com.geedgenetworks.common.udf.AggregateFunction;
@@ -139,12 +140,10 @@ public class AbstractFirstAggregation extends ProcessFunction<Event, Accumulator
return (long) (ThreadLocalRandom.current().nextDouble() * (double) windowSize);
}
- public Accumulator createAccumulator() {
+ public Accumulator createAccumulator(Map<String, Object> keysMap) {
- Map<String, Object> map = new HashMap<>();
Accumulator accumulator = new Accumulator();
-
- accumulator.setMetricsFields(map);
+ accumulator.setMetricsFields(keysMap);
for (UdfEntity udfEntity : functions) {
udfEntity.getAggregateFunction().initAccumulator(accumulator);
}
@@ -152,17 +151,21 @@ public class AbstractFirstAggregation extends ProcessFunction<Event, Accumulator
}
- public String getKey(Event value, List<String> keys) {
+ public KeybyEntity getKey(Event value, List<String> keys) {
+ KeybyEntity keybyEntity = new KeybyEntity(new HashMap<>());
StringBuilder stringBuilder = new StringBuilder();
for (String key : keys) {
-
- if (value.getExtractedFields().containsKey(key)) {
- stringBuilder.append(value.getExtractedFields().get(key).toString());
- } else {
+ Object object = value.getExtractedFields().get(key);
+ if(object==null){
stringBuilder.append(",");
+ }else{
+ keybyEntity.getKeys().put(key, object);
+ stringBuilder.append(object);
}
+
}
- return SecureUtil.md5(stringBuilder.toString());
+ keybyEntity.setKeysToString(SecureUtil.md5(stringBuilder.toString()));
+ return keybyEntity;
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorImpl.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorImpl.java
index c261fb6..4712d36 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorImpl.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorImpl.java
@@ -1,9 +1,15 @@
package com.geedgenetworks.core.processor.aggregate;
+import com.alibaba.fastjson.JSONObject;
import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.config.AggregateConfigOptions;
+import com.geedgenetworks.common.config.CheckConfigUtil;
+import com.geedgenetworks.common.config.CheckResult;
import com.geedgenetworks.common.exception.CommonErrorCode;
+import com.geedgenetworks.common.exception.ConfigValidationException;
import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
import com.geedgenetworks.core.pojo.AggregateConfig;
+import com.typesafe.config.Config;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
@@ -13,6 +19,8 @@ import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindo
import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
import org.apache.flink.streaming.api.windowing.time.Time;
+import java.util.Map;
+
import static com.geedgenetworks.common.Constants.*;
public class AggregateProcessorImpl implements AggregateProcessor {
@@ -97,4 +105,25 @@ public class AggregateProcessorImpl implements AggregateProcessor {
return "aggregate";
}
+ @Override
+ public AggregateConfig checkConfig(String name, Map<String, Object> configProperties, Config typeSafeConfig) {
+ CheckResult result = CheckConfigUtil.checkAllExists(typeSafeConfig.getConfig(name),
+ AggregateConfigOptions.GROUP_BY_FIELDS.key(),
+ AggregateConfigOptions.WINDOW_TYPE.key(),
+ AggregateConfigOptions.FUNCTIONS.key(),
+ AggregateConfigOptions.WINDOW_SIZE.key());
+ if (!result.isSuccess()) {
+ throw new ConfigValidationException(CommonErrorCode.CONFIG_VALIDATION_FAILED, String.format(
+ "Aggregate processor: %s, At least one of [%s] should be specified.",
+ name, String.join(",",
+ AggregateConfigOptions.OUTPUT_FIELDS.key(),
+ AggregateConfigOptions.REMOVE_FIELDS.key(),
+ AggregateConfigOptions.FUNCTIONS.key())));
+ }
+
+ AggregateConfig aggregateConfig = new JSONObject(configProperties).toJavaObject(AggregateConfig.class);
+ aggregateConfig.setName(name);
+ return aggregateConfig;
+ }
+
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/FirstAggregationEventTime.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/FirstAggregationEventTime.java
index 156c0ed..5adc6d1 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/FirstAggregationEventTime.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/FirstAggregationEventTime.java
@@ -3,6 +3,7 @@ package com.geedgenetworks.core.processor.aggregate;
import com.geedgenetworks.common.Accumulator;
import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.KeybyEntity;
import com.geedgenetworks.core.pojo.AggregateConfig;
import lombok.extern.slf4j.Slf4j;
import org.apache.flink.configuration.Configuration;
@@ -33,7 +34,7 @@ public class FirstAggregationEventTime extends AbstractFirstAggregation {
Long timestamp;
internalMetrics.incrementInEvents();
try {
- String key = getKey(value, groupByFields);
+ KeybyEntity keybyEntity = getKey(value, groupByFields);
while ((timestamp = eventTimeTimersQueue.peek()) != null && timestamp <= ctx.timerService().currentWatermark()) {
eventTimeTimersQueue.poll();
onTimer(timestamp, out);
@@ -41,15 +42,15 @@ public class FirstAggregationEventTime extends AbstractFirstAggregation {
long windowEnd = assignWindowEnd(ctx.timerService().currentWatermark());
if (!windows.containsKey(windowEnd)) {
Map<String, Accumulator> map = new HashMap<>();
- map.put(key, createAccumulator());
+ map.put(keybyEntity.getKeysToString(), createAccumulator(keybyEntity.getKeys()));
windows.put(windowEnd, map);
eventTimeTimersQueue.add(windowEnd);
} else {
- if (!windows.get(windowEnd).containsKey(key)) {
- windows.get(windowEnd).put(key, createAccumulator());
+ if (!windows.get(windowEnd).containsKey(keybyEntity.getKeysToString())) {
+ windows.get(windowEnd).put(keybyEntity.getKeysToString(), createAccumulator(keybyEntity.getKeys()));
}
}
- add(value, windows.get(windowEnd).get(key));
+ add(value, windows.get(windowEnd).get(keybyEntity.getKeysToString()));
} catch (Exception e) {
log.error("Error in pre-aggregate processElement", e);
internalMetrics.incrementErrorEvents();
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/FirstAggregationProcessingTime.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/FirstAggregationProcessingTime.java
index e98daa5..01c346f 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/FirstAggregationProcessingTime.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/FirstAggregationProcessingTime.java
@@ -3,6 +3,7 @@ package com.geedgenetworks.core.processor.aggregate;
import com.geedgenetworks.common.Accumulator;
import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.KeybyEntity;
import com.geedgenetworks.core.pojo.AggregateConfig;
import lombok.extern.slf4j.Slf4j;
import org.apache.flink.configuration.Configuration;
@@ -32,7 +33,7 @@ public class FirstAggregationProcessingTime extends AbstractFirstAggregation {
Long timestamp;
internalMetrics.incrementInEvents();
try {
- String key = getKey(value, groupByFields);
+ KeybyEntity keybyEntity = getKey(value, groupByFields);
while ((timestamp = processingTimeTimersQueue.peek()) != null && timestamp <= ctx.timerService().currentProcessingTime()) {
processingTimeTimersQueue.poll();
onTimer(timestamp, out);
@@ -40,15 +41,15 @@ public class FirstAggregationProcessingTime extends AbstractFirstAggregation {
long windowEnd = assignWindowEnd(ctx.timerService().currentProcessingTime());
if (!windows.containsKey(windowEnd)) {
Map<String, Accumulator> map = new HashMap<>();
- map.put(key, createAccumulator());
+ map.put(keybyEntity.getKeysToString(), createAccumulator(keybyEntity.getKeys()));
windows.put(windowEnd, map);
processingTimeTimersQueue.add(windowEnd);
} else {
- if (!windows.get(windowEnd).containsKey(key)) {
- windows.get(windowEnd).put(key, createAccumulator());
+ if (!windows.get(windowEnd).containsKey(keybyEntity.getKeysToString())) {
+ windows.get(windowEnd).put(keybyEntity.getKeysToString(), createAccumulator(keybyEntity.getKeys()));
}
}
- add(value, windows.get(windowEnd).get(key));
+ add(value, windows.get(windowEnd).get(keybyEntity.getKeysToString()));
} catch (Exception e) {
log.error("Error in pre-aggregate processElement", e);
internalMetrics.incrementErrorEvents();
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/KeySelector.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/KeySelector.java
index da09690..a6fb294 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/KeySelector.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/KeySelector.java
@@ -22,12 +22,12 @@ public class KeySelector implements org.apache.flink.api.java.functions.KeySelec
KeybyEntity keybyEntity = new KeybyEntity(new HashMap<>());
StringBuilder stringBuilder = new StringBuilder();
for (String key : keys) {
-
- if (value.getExtractedFields().containsKey(key)) {
- keybyEntity.getKeys().put(key, value.getExtractedFields().get(key));
- stringBuilder.append(value.getExtractedFields().get(key).toString());
- } else {
+ Object object = value.getExtractedFields().get(key);
+ if(object==null){
stringBuilder.append(",");
+ }else{
+ keybyEntity.getKeys().put(key, object);
+ stringBuilder.append(object);
}
}
String hashedKey = SecureUtil.md5(stringBuilder.toString());
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/PreKeySelector.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/PreKeySelector.java
index 6e43184..4b21ba7 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/PreKeySelector.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/PreKeySelector.java
@@ -22,12 +22,12 @@ public class PreKeySelector implements org.apache.flink.api.java.functions.KeySe
KeybyEntity keybyEntity = new KeybyEntity(new HashMap<>());
StringBuilder stringBuilder = new StringBuilder();
for (String key : keys) {
-
- if (value.getMetricsFields().containsKey(key)) {
- keybyEntity.getKeys().put(key, value.getMetricsFields().get(key));
- stringBuilder.append(value.getMetricsFields().get(key).toString());
- } else {
+ Object object = value.getMetricsFields().get(key);
+ if(object==null){
stringBuilder.append(",");
+ }else{
+ keybyEntity.getKeys().put(key, object);
+ stringBuilder.append(object);
}
}
String hashedKey = SecureUtil.md5(stringBuilder.toString());
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/ProjectionProcessorImpl.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/ProjectionProcessorImpl.java
index d87e7e2..7b35566 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/ProjectionProcessorImpl.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/ProjectionProcessorImpl.java
@@ -1,13 +1,22 @@
package com.geedgenetworks.core.processor.projection;
+import com.alibaba.fastjson.JSONObject;
import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.config.CheckConfigUtil;
+import com.geedgenetworks.common.config.CheckResult;
+import com.geedgenetworks.common.config.ProjectionConfigOptions;
+import com.geedgenetworks.common.exception.CommonErrorCode;
+import com.geedgenetworks.common.exception.ConfigValidationException;
import com.geedgenetworks.core.pojo.ProjectionConfig;
+import com.typesafe.config.Config;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
import org.apache.flink.util.OutputTag;
+import java.util.Map;
+
public class ProjectionProcessorImpl implements ProjectionProcessor {
@@ -28,4 +37,25 @@ public class ProjectionProcessorImpl implements ProjectionProcessor {
public String type() {
return "projection";
}
+
+ @Override
+ public ProjectionConfig checkConfig(String name, Map<String, Object> configProperties, Config typeSafeConfig) {
+ CheckResult result = CheckConfigUtil.checkAtLeastOneExists(typeSafeConfig.getConfig(name),
+ ProjectionConfigOptions.OUTPUT_FIELDS.key(),
+ ProjectionConfigOptions.REMOVE_FIELDS.key(),
+ ProjectionConfigOptions.FUNCTIONS.key());
+ if (!result.isSuccess()) {
+ throw new ConfigValidationException(CommonErrorCode.CONFIG_VALIDATION_FAILED, String.format(
+ "Processor: %s, At least one of [%s] should be specified.",
+ name, String.join(",",
+ ProjectionConfigOptions.OUTPUT_FIELDS.key(),
+ ProjectionConfigOptions.REMOVE_FIELDS.key(),
+ ProjectionConfigOptions.FUNCTIONS.key())));
+ }
+
+ ProjectionConfig projectionConfig = new JSONObject(configProperties).toJavaObject(ProjectionConfig.class);
+ projectionConfig.setName(name);
+
+ return projectionConfig;
+ }
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessorImpl.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessorImpl.java
index 6ddc616..84454cf 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessorImpl.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessorImpl.java
@@ -1,7 +1,14 @@
package com.geedgenetworks.core.processor.table;
+import com.alibaba.fastjson.JSONObject;
import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.config.CheckConfigUtil;
+import com.geedgenetworks.common.config.CheckResult;
+import com.geedgenetworks.common.config.TableConfigOptions;
+import com.geedgenetworks.common.exception.CommonErrorCode;
+import com.geedgenetworks.common.exception.ConfigValidationException;
import com.geedgenetworks.core.pojo.TableConfig;
+import com.typesafe.config.Config;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
@@ -30,4 +37,24 @@ public class TableProcessorImpl implements TableProcessor {
public String type() {
return "table";
}
+
+ @Override
+ public TableConfig checkConfig(String name, Map<String, Object> configProperties, Config typeSafeConfig) {
+ CheckResult result = CheckConfigUtil.checkAtLeastOneExists(typeSafeConfig.getConfig(name),
+ TableConfigOptions.OUTPUT_FIELDS.key(),
+ TableConfigOptions.REMOVE_FIELDS.key(),
+ TableConfigOptions.FUNCTIONS.key());
+ if (!result.isSuccess()) {
+ throw new ConfigValidationException(CommonErrorCode.CONFIG_VALIDATION_FAILED, String.format(
+ "Table processor: %s, At least one of [%s] should be specified.",
+ name, String.join(",",
+ TableConfigOptions.OUTPUT_FIELDS.key(),
+ TableConfigOptions.REMOVE_FIELDS.key(),
+ TableConfigOptions.FUNCTIONS.key())));
+ }
+
+ TableConfig tableConfig = new JSONObject(configProperties).toJavaObject(TableConfig.class);
+ tableConfig.setName(name);
+ return tableConfig;
+ }
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/split/Split.java b/groot-core/src/main/java/com/geedgenetworks/core/split/Split.java
index 37e7b44..4e4e387 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/split/Split.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/split/Split.java
@@ -2,15 +2,21 @@ package com.geedgenetworks.core.split;
import com.geedgenetworks.common.Event;
import com.geedgenetworks.core.pojo.SplitConfig;
+import com.typesafe.config.Config;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import java.io.Serializable;
+import java.util.Map;
import java.util.Set;
-public interface Split {
+public interface Split<T extends SplitConfig> extends Serializable {
DataStream<Event> splitFunction(
- DataStream<Event> dataStream, SplitConfig splitConfig)
+ DataStream<Event> dataStream, T splitConfig)
throws Exception;
String type();
+
+ T checkConfig(String name, Map<String, Object> configProperties, Config typeSafeConfig);
+
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/split/SplitFunction.java b/groot-core/src/main/java/com/geedgenetworks/core/split/SplitFunction.java
index f07b568..07d4f9f 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/split/SplitFunction.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/split/SplitFunction.java
@@ -43,7 +43,7 @@ public class SplitFunction extends ProcessFunction<Event, Event> {
instance.setFunctionMissing(null);
Expression compiledExp = instance.compile(expression, true);
rule.setCompiledExpression(compiledExp);
- OutputTag<Event> outputTag = new OutputTag<>(rule.getName()){};
+ OutputTag<Event> outputTag = new OutputTag<>(rule.getTag()){};
rule.setOutputTag(outputTag);
}
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/split/SplitOperator.java b/groot-core/src/main/java/com/geedgenetworks/core/split/SplitOperator.java
index f6d2c8c..48ef92d 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/split/SplitOperator.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/split/SplitOperator.java
@@ -1,10 +1,15 @@
package com.geedgenetworks.core.split;
+import com.alibaba.fastjson.JSONObject;
import com.geedgenetworks.common.Event;
+import com.geedgenetworks.core.pojo.FilterConfig;
import com.geedgenetworks.core.pojo.SplitConfig;
+import com.typesafe.config.Config;
import org.apache.flink.streaming.api.datastream.DataStream;
-public class SplitOperator implements Split {
+import java.util.Map;
+
+public class SplitOperator implements Split<SplitConfig> {
@Override
public DataStream<Event> splitFunction(
@@ -26,4 +31,11 @@ public class SplitOperator implements Split {
return "split";
}
+ @Override
+ public SplitConfig checkConfig(String name, Map<String, Object> configProperties, Config config) {
+ SplitConfig splitConfig = new JSONObject(configProperties).toJavaObject(SplitConfig.class);
+ splitConfig.setName(name);
+ return splitConfig;
+ }
+
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/AsnLookup.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/AsnLookup.java
index bdb3698..ac282b3 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/AsnLookup.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/AsnLookup.java
@@ -2,8 +2,7 @@ package com.geedgenetworks.core.udf;
import com.alibaba.fastjson2.JSON;
import com.geedgenetworks.common.Constants;
-import com.geedgenetworks.common.config.CommonConfig;
-import com.geedgenetworks.common.config.KnowledgeBaseConfig;
+import com.geedgenetworks.common.config.*;
import com.geedgenetworks.common.exception.CommonErrorCode;
import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
import com.geedgenetworks.common.udf.ScalarFunction;
@@ -24,10 +23,23 @@ public class AsnLookup implements ScalarFunction {
private String lookupFieldName;
private String outputFieldName;
+
+ enum Option {
+ IP_TO_ASN;
+
+ public static boolean isValid(String option) {
+ try {
+ Option.valueOf(option);
+ return true;
+ } catch (IllegalArgumentException e) {
+ return false;
+ }
+ }
+ }
@Override
public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
- checkUdfContext(udfContext);
+ checkConfig(udfContext);
this.kbName = udfContext.getParameters().get("kb_name").toString();
this.option = udfContext.getParameters().get("option").toString();
Configuration configuration = (Configuration) runtimeContext
@@ -44,8 +56,8 @@ public class AsnLookup implements ScalarFunction {
} else {
log.error("AsnLookup init KnowledgeBase error ");
}
- this.lookupFieldName = udfContext.getLookup_fields().get(0);
- this.outputFieldName = udfContext.getOutput_fields().get(0);
+ this.lookupFieldName = udfContext.getLookupFields().get(0);
+ this.outputFieldName = udfContext.getOutputFields().get(0);
AsnKnowledgeBaseHandler.increment();
log.warn("AsnKnowledgeBaseHandlerCount "+AsnKnowledgeBaseHandler.getCount());
}
@@ -54,8 +66,9 @@ public class AsnLookup implements ScalarFunction {
@Override
public Event evaluate(Event event) {
- if (event.getExtractedFields().containsKey(lookupFieldName)) {
- String asn = AsnKnowledgeBaseHandler.lookUp(kbName, option, event.getExtractedFields().get(lookupFieldName).toString());
+ Object valueObj = event.getExtractedFields().get(lookupFieldName);
+ if (valueObj!=null) {
+ String asn = AsnKnowledgeBaseHandler.lookUp(kbName, option, valueObj.toString());
if (!asn.isEmpty()) {
event.getExtractedFields().put(outputFieldName, asn);
}
@@ -78,29 +91,38 @@ public class AsnLookup implements ScalarFunction {
}
}
- private void checkUdfContext(UDFContext udfContext) {
-
- if (udfContext.getLookup_fields() == null || udfContext.getOutput_fields() == null || udfContext.getParameters() == null) {
- throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
+ @Override
+ public void checkConfig(UDFContext udfContext) {
+ CheckResult result = CheckUDFContextUtil.checkAllExists(udfContext,
+ UDFContextConfigOptions.LOOKUP_FIELDS.key(),
+ UDFContextConfigOptions.OUTPUT_FIELDS.key(),
+ UDFContextConfigOptions.PARAMETERS.key());
+
+ if (!result.isSuccess()) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.CONFIG_VALIDATION_FAILED, result.getMsg());
}
- if (udfContext.getLookup_fields().size() != 1) {
- throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function lookup fields only support 1 value");
+ result = CheckUDFContextUtil.checkCollectionSingleItemExists(udfContext, UDFContextConfigOptions.LOOKUP_FIELDS.key());
+ if (!result.isSuccess()) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, result.getMsg());
}
- if (udfContext.getOutput_fields().size() != 1) {
- throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value");
+
+ result = CheckUDFContextUtil.checkCollectionSingleItemExists(udfContext, UDFContextConfigOptions.OUTPUT_FIELDS.key());
+ if (!result.isSuccess()) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, result.getMsg());
}
- if (!udfContext.getParameters().containsKey("kb_name")) {
- throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters need containkey kb_name");
+ result = CheckUDFContextUtil.checkParametersContainsKeys(udfContext, UDFContextConfigOptions.PARAMETERS_KB_NAME.key(), UDFContextConfigOptions.PARAMETERS_OPTION.key());
+ if (!result.isSuccess()) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, result.getMsg());
}
- if (!udfContext.getParameters().containsKey("option")) {
- throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters need containkey option");
- } else {
- if (!udfContext.getParameters().get("option").toString().equals("IP_TO_ASN")) {
- throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters option value is not correct");
- }
+
+ String optionValue = udfContext.getParameters().get(UDFContextConfigOptions.PARAMETERS_OPTION.key()).toString();
+ if (!Option.isValid(optionValue)) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, String.format( "UDF: %s, [%s] Option value is not correct.",
+ udfContext.getFunction(), UDFContextConfigOptions.PARAMETERS.key()));
}
+
}
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/CurrentUnixTimestamp.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/CurrentUnixTimestamp.java
index 5770201..98b2d68 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/CurrentUnixTimestamp.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/CurrentUnixTimestamp.java
@@ -18,10 +18,10 @@ public class CurrentUnixTimestamp implements ScalarFunction {
public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
- if( udfContext.getOutput_fields()==null || udfContext.getParameters() == null){
+ if( udfContext.getOutputFields()==null || udfContext.getParameters() == null){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
}
- if(udfContext.getOutput_fields().size() != 1){
+ if(udfContext.getOutputFields().size() != 1){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value");
}
if(!udfContext.getParameters().containsKey("precision")){
@@ -34,7 +34,7 @@ public class CurrentUnixTimestamp implements ScalarFunction {
}
}
this.precision = udfContext.getParameters().get("precision").toString();
- this.outputFieldName = udfContext.getOutput_fields().get(0);
+ this.outputFieldName = udfContext.getOutputFields().get(0);
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/DecodeBase64.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/DecodeBase64.java
index 3581d5c..bc8563a 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/DecodeBase64.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/DecodeBase64.java
@@ -22,16 +22,16 @@ public class DecodeBase64 implements ScalarFunction {
@Override
public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
- if(udfContext.getParameters()==null || udfContext.getOutput_fields()==null){
+ if(udfContext.getParameters()==null || udfContext.getOutputFields()==null){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
}
- if(udfContext.getOutput_fields().size() != 1){
+ if(udfContext.getOutputFields().size() != 1){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value");
}
if(!udfContext.getParameters().containsKey("value_field") ||!udfContext.getParameters().containsKey("charset_field") ){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters must containkey value_field and charset_field");
}
- this.outputFieldName = udfContext.getOutput_fields().get(0);
+ this.outputFieldName = udfContext.getOutputFields().get(0);
this.valueField =udfContext.getParameters().get("value_field").toString();
this.charsetField =udfContext.getParameters().get("charset_field").toString();
@@ -41,15 +41,11 @@ public class DecodeBase64 implements ScalarFunction {
@Override
public Event evaluate(Event event) {
- if (event.getExtractedFields().containsKey(valueField)) {
-
+ Object valueObj = event.getExtractedFields().get(valueField);
+ if (valueObj!=null) {
String decodeResult = "";
- String message =
- (String)
- event.getExtractedFields()
- .get(valueField);
- Object charset =
- event.getExtractedFields().getOrDefault(charsetField,"");
+ String message = (String)valueObj;
+ Object charset = event.getExtractedFields().getOrDefault(charsetField,"UTF-8");
try {
if (StringUtil.isNotBlank(message)) {
byte[] base64decodedBytes = Base64.getDecoder().decode(message);
@@ -62,14 +58,9 @@ public class DecodeBase64 implements ScalarFunction {
} catch (RuntimeException e) {
log.error("Resolve Base64 exception, exception information:" + e.getMessage());
} catch (UnsupportedEncodingException e) {
- log.error(
- "The Character Encoding ["
- + charset.toString()
- + "] is not supported.exception information:"
- + e.getMessage());
+ log.error("The Character Encoding [" + charset.toString() + "] is not supported.exception information:" + e.getMessage());
}
- event.getExtractedFields()
- .put(outputFieldName, decodeResult);
+ event.getExtractedFields().put(outputFieldName, decodeResult);
}
return event;
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/Domain.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/Domain.java
index 77b3246..74816f5 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/Domain.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/Domain.java
@@ -1,111 +1,124 @@
package com.geedgenetworks.core.udf;
-
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.config.CheckResult;
+import com.geedgenetworks.common.config.CheckUDFContextUtil;
+import com.geedgenetworks.common.config.UDFContextConfigOptions;
import com.geedgenetworks.common.exception.CommonErrorCode;
import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
import com.geedgenetworks.common.udf.ScalarFunction;
-import com.geedgenetworks.common.Event;
import com.geedgenetworks.common.udf.UDFContext;
+import com.geedgenetworks.shaded.com.google.common.net.InternetDomainName;
import lombok.extern.slf4j.Slf4j;
import org.apache.flink.api.common.functions.RuntimeContext;
-
import java.util.List;
-
import static com.geedgenetworks.utils.FormatUtils.getTopPrivateDomain;
@Slf4j
public class Domain implements ScalarFunction {
-
-
- private String option;
+ private Option option;
private List<String> lookupFields;
private String outputFieldName;
- @Override
- public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
- if(udfContext.getLookup_fields()==null || udfContext.getOutput_fields()==null || udfContext.getParameters() == null){
- throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
- }
- if(udfContext.getLookup_fields().isEmpty()){
- throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function lookup field is not empty");
- }
- if(udfContext.getOutput_fields().size() != 1){
- throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value");
- }
- if(!udfContext.getParameters().containsKey("option")){
- throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters need containkey option");
- }
- else{
- if(!udfContext.getParameters().get("option").toString().equals("TOP_LEVEL_DOMAIN") &&
- !udfContext.getParameters().get("option").toString().equals("FIRST_SIGNIFICANT_SUBDOMAIN") &&
- !udfContext.getParameters().get("option").toString().equals("FQDN")){
- throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters option value is not correct");
+ enum Option {
+ TOP_LEVEL_DOMAIN,
+ FIRST_SIGNIFICANT_SUBDOMAIN,
+ FQDN;
+
+ public static boolean isValid(String option) {
+ try {
+ Option.valueOf(option);
+ return true;
+ } catch (IllegalArgumentException e) {
+ return false;
}
}
- this.option = udfContext.getParameters().get("option").toString();
- this.lookupFields = udfContext.getLookup_fields();
- this.outputFieldName = udfContext.getOutput_fields().get(0);
+ }
+ @Override
+ public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
+ checkConfig(udfContext);
+ this.option = Option.valueOf(udfContext.getParameters().get("option").toString());
+ this.lookupFields = udfContext.getLookupFields();
+ this.outputFieldName = udfContext.getOutputFields().get(0);
}
@Override
public Event evaluate(Event event) {
String domain = "";
- switch (option) {
- case "TOP_LEVEL_DOMAIN":
- for (String lookupField : lookupFields){
- if(event.getExtractedFields().containsKey(lookupField)) {
- domain =
- getTopPrivateDomain(
- (String)
- event.getExtractedFields()
- .get(lookupField));
- if (domain.contains(".")) {
- domain = domain.substring(domain.indexOf(".") + 1);
- }
- if(!domain.isEmpty()){
- break;
- }
- }
- }
- break;
- case "FIRST_SIGNIFICANT_SUBDOMAIN":
- for (String lookupField : lookupFields){
- if(event.getExtractedFields().containsKey(lookupField)) {
- domain =
- getTopPrivateDomain(
- (String)
- event.getExtractedFields()
- .get(lookupField));
- }
- if(!domain.isEmpty()){
+ for (String fieldName : lookupFields) {
+ Object fieldValue = event.getExtractedFields().get(fieldName);
+ if (fieldValue == null) {
+ continue;
+ }
+
+ String value = fieldValue.toString();
+ try {
+
+ String topPrivateDomain = getTopPrivateDomain(value);
+
+ switch (option) {
+ case TOP_LEVEL_DOMAIN:
+ domain = InternetDomainName.from(topPrivateDomain).publicSuffix().toString();
break;
- }
- }
- break;
- case "FQDN":
- for (String lookupField : lookupFields) {
- if (event.getExtractedFields().containsKey(lookupField)) {
- domain = (String) event.getExtractedFields()
- .get(lookupField);
- }
- if(!domain.isEmpty()){
+ case FIRST_SIGNIFICANT_SUBDOMAIN:
+ domain = topPrivateDomain;
+ break;
+ case FQDN: // Use the original value
+ domain = value;
break;
- }
+ default:
+ throw new IllegalArgumentException("Unknown option: " + option);
+ }
+
+ if (!domain.isEmpty()) { // Found a valid domain will break the loop
+ break;
}
- break;
+
+ } catch (IllegalArgumentException e) {
+ log.error("Invalid domain: {}", value);
+ }
}
+
event.getExtractedFields().put(outputFieldName, domain);
return event;
}
@Override
+ public void checkConfig(UDFContext udfContext) {
+ CheckResult result = CheckUDFContextUtil.checkAllExists(udfContext,
+ UDFContextConfigOptions.LOOKUP_FIELDS.key(),
+ UDFContextConfigOptions.OUTPUT_FIELDS.key(),
+ UDFContextConfigOptions.PARAMETERS.key());
+
+ if (!result.isSuccess()) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.CONFIG_VALIDATION_FAILED, result.getMsg());
+ }
+
+ result = CheckUDFContextUtil.checkCollectionSingleItemExists(udfContext, UDFContextConfigOptions.OUTPUT_FIELDS.key());
+ if (!result.isSuccess()) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, result.getMsg());
+ }
+
+ if(!udfContext.getParameters().containsKey(UDFContextConfigOptions.PARAMETERS_OPTION.key())){
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, String.format(
+ "UDF: %s, [%s] Option should be specified.",
+ udfContext.getFunction(), UDFContextConfigOptions.PARAMETERS.key()));
+ }
+
+ String optionValue = udfContext.getParameters().get(UDFContextConfigOptions.PARAMETERS_OPTION.key()).toString();
+ if (!Option.isValid(optionValue)) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, String.format( "UDF: %s, [%s] Option value is not correct.",
+ udfContext.getFunction(), UDFContextConfigOptions.PARAMETERS.key()));
+ }
+
+ }
+
+ @Override
public String functionName() {
return "DOMAIN";
}
-
-
@Override
public void close() {
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/Drop.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/Drop.java
index 93cd0db..c7f13c2 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/Drop.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/Drop.java
@@ -1,5 +1,10 @@
package com.geedgenetworks.core.udf;
+import com.geedgenetworks.common.config.CheckResult;
+import com.geedgenetworks.common.config.CheckUDFContextUtil;
+import com.geedgenetworks.common.config.UDFContextConfigOptions;
+import com.geedgenetworks.common.exception.CommonErrorCode;
+import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
import com.geedgenetworks.common.udf.ScalarFunction;
import com.geedgenetworks.common.Event;
import com.geedgenetworks.common.udf.UDFContext;
@@ -10,6 +15,7 @@ public class Drop implements ScalarFunction {
@Override
public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
+ checkConfig(udfContext);
}
@Override
@@ -24,6 +30,17 @@ public class Drop implements ScalarFunction {
}
@Override
+ public void checkConfig(UDFContext udfContext) {
+ CheckResult result = CheckUDFContextUtil.checkAllExists(udfContext,
+ UDFContextConfigOptions.FILTER.key());
+
+ if (!result.isSuccess()) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.CONFIG_VALIDATION_FAILED, result.getMsg());
+ }
+
+ }
+
+ @Override
public void close() {
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/EncodeBase64.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/EncodeBase64.java
index c22ff54..a950252 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/EncodeBase64.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/EncodeBase64.java
@@ -8,44 +8,59 @@ import com.geedgenetworks.common.udf.UDFContext;
import lombok.extern.slf4j.Slf4j;
import org.apache.flink.api.common.functions.RuntimeContext;
+import java.nio.charset.StandardCharsets;
+import java.text.SimpleDateFormat;
import java.util.Base64;
+import java.util.List;
@Slf4j
public class EncodeBase64 implements ScalarFunction {
- private String valueField;
+ private String input_type;
private String outputFieldName;
+ private String lookupFieldName;
@Override
public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
- if(udfContext.getParameters()==null || udfContext.getOutput_fields()==null){
+ if(udfContext.getParameters()==null || udfContext.getOutputFields()==null || udfContext.getLookupFields() == null){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
}
- if(udfContext.getOutput_fields().size() != 1){
+ if(udfContext.getOutputFields().size() != 1){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value");
}
- if(!udfContext.getParameters().containsKey("value_field") ){
+ if(udfContext.getLookupFields().size() != 1){
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function lookup fields only support 1 value");
+ }
+ if(!udfContext.getParameters().containsKey("input_type") ){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters must containkey value_field ");
}
- this.outputFieldName = udfContext.getOutput_fields().get(0);
- this.valueField =udfContext.getParameters().get("value_field").toString();
-
-
+ this.outputFieldName = udfContext.getOutputFields().get(0);
+ this.lookupFieldName = udfContext.getLookupFields().get(0);
+ this.input_type =udfContext.getParameters().get("input_type").toString();
}
@Override
public Event evaluate(Event event) {
String encodeResult = "";
- if (event.getExtractedFields().containsKey(valueField)) {
+ Object valueObj = event.getExtractedFields().get(lookupFieldName);
+ if (valueObj!=null) {
try {
- encodeResult = Base64.getEncoder().encodeToString((byte[]) event.getExtractedFields().getOrDefault(valueField,"".getBytes()));
+ switch (input_type) {
+ case "byte_array":
+ encodeResult = Base64.getEncoder().encodeToString((byte[]) valueObj);
+ break;
+ case "string":
+ encodeResult = Base64.getEncoder().encodeToString(valueObj.toString().getBytes(StandardCharsets.UTF_8));
+ break;
+ default:
+ log.error("Encode Base64 exception, Unsupport input_type :" + input_type);
+ break;
+ }
} catch (RuntimeException e) {
log.error("Encode Base64 exception, exception information:" + e.getMessage());
}
-
- event.getExtractedFields()
- .put(outputFieldName, encodeResult);
+ event.getExtractedFields().put(outputFieldName, encodeResult);
}
return event;
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/Encrypt.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/Encrypt.java
new file mode 100644
index 0000000..c8e21b2
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/Encrypt.java
@@ -0,0 +1,167 @@
+package com.geedgenetworks.core.udf;
+
+import cn.hutool.core.util.URLUtil;
+import cn.hutool.json.JSONArray;
+import cn.hutool.json.JSONObject;
+import cn.hutool.json.JSONUtil;
+import com.alibaba.fastjson2.JSON;
+import com.geedgenetworks.common.Constants;
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.config.CommonConfig;
+import com.geedgenetworks.common.config.KmsConfig;
+import com.geedgenetworks.common.config.SSLConfig;
+import com.geedgenetworks.common.exception.CommonErrorCode;
+import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
+import com.geedgenetworks.common.udf.ScalarFunction;
+import com.geedgenetworks.common.udf.UDFContext;
+import com.geedgenetworks.core.pojo.KmsKey;
+import com.geedgenetworks.core.udf.encrypt.EncryptionAlgorithm;
+import com.geedgenetworks.core.utils.*;
+import com.geedgenetworks.shaded.org.apache.http.HttpHeaders;
+import com.geedgenetworks.shaded.org.apache.http.HttpStatus;
+import com.geedgenetworks.shaded.org.apache.http.message.BasicHeader;
+import com.geedgenetworks.utils.StringUtil;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.configuration.Configuration;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+@Slf4j
+public class Encrypt implements ScalarFunction {
+
+ private String lookupFieldName;
+ private String outputFieldName;
+ private String identifier;
+ private String defaultVal;
+ private String type;
+ private transient SingleValueMap.Data<LoadIntervalDataUtil<Set<String>>> sensitiveFieldsData;
+ private transient SingleValueMap.Data<LoadIntervalDataUtil<KmsKey>> kmsKeyData;
+ private transient EncryptionAlgorithm encryptionAlgorithm;
+
+ @Override
+ public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
+ checkUdfContext(udfContext);
+ if (udfContext.getParameters().containsKey("default_val")) {
+ this.defaultVal = udfContext.getParameters().get("default_val").toString();
+ }
+ this.lookupFieldName = udfContext.getLookupFields().get(0);
+ this.outputFieldName = udfContext.getOutputFields().get(0);
+ this.identifier = udfContext.getParameters().get("identifier").toString();
+ Configuration configuration = (Configuration) runtimeContext.getExecutionConfig().getGlobalJobParameters();
+ CommonConfig commonConfig = JSON.parseObject(configuration.toMap().get(Constants.SYSPROP_GROOTSTREAM_CONFIG), CommonConfig.class);
+ KmsConfig kmsConfig = commonConfig.getKmsConfig().get(configuration.toMap().get(Constants.SYSPROP_KMS_TYPE_CONFIG));
+ SSLConfig sslConfig = commonConfig.getSslConfig();
+ Map<String, String> propertiesConfig = commonConfig.getPropertiesConfig();
+ type = kmsConfig.getType();
+ try {
+ encryptionAlgorithm = EncryptionAlgorithmUtils.createEncryptionAlgorithm(identifier);
+ if (encryptionAlgorithm == null) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Parameters identifier is illegal!");
+ }
+ if (!type.equals(KmsUtils.KMS_TYPE_LOCAL)) {
+ kmsKeyData = SingleValueMap.acquireData("kmsKeyData",
+ () -> LoadIntervalDataUtil.newInstance(() -> KmsUtils.getVaultKey(kmsConfig, sslConfig, identifier),
+ LoadIntervalDataOptions.defaults("kmsKeyData", Integer.parseInt(propertiesConfig.getOrDefault(Constants.SYSPROP_ENCRYPT_KMS_KEY_SCHEDULER_INTERVAL_NAME, "5")) * 60000L)),
+ LoadIntervalDataUtil::stop);
+ KmsKey kmsKey = kmsKeyData.getData().data();
+ if (kmsKey == null) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Initialization UDF Encrypt failed!");
+ }
+ if (encryptionAlgorithm.getSecretKeyLength() != kmsKey.getKeyData().length) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Global parameter kms secret Key requires " + encryptionAlgorithm.getSecretKeyLength() + " bytes!");
+ }
+ encryptionAlgorithm.setKmsKey(kmsKey);
+ }
+ sensitiveFieldsData = SingleValueMap.acquireData("sensitiveFields",
+ () -> LoadIntervalDataUtil.newInstance(() -> getSensitiveFields(propertiesConfig.get("projection.encrypt.schema.registry.uri")),
+ LoadIntervalDataOptions.defaults("sensitiveFields", Integer.parseInt(propertiesConfig.getOrDefault(Constants.SYSPROP_ENCRYPT_SENSITIVE_FIELDS_SCHEDULER_INTERVAL_NAME, "5")) * 60000L)),
+ LoadIntervalDataUtil::stop);
+ if (sensitiveFieldsData.getData().data() == null) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Initialization UDF Encrypt failed!");
+ }
+ } catch (Exception e) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.UNSUPPORTED_OPERATION, "Initialization UDF Encrypt failed!", e);
+ }
+ }
+
+ @Override
+ public Event evaluate(Event event) {
+ try {
+ if (!type.equals(KmsUtils.KMS_TYPE_LOCAL)) {
+ KmsKey kmsKey = kmsKeyData.getData().data();
+ if (kmsKey.getKeyVersion() != encryptionAlgorithm.getKmsKey().getKeyVersion() || !Arrays.equals(kmsKey.getKeyData(), encryptionAlgorithm.getKmsKey().getKeyData())) {
+ encryptionAlgorithm.setKmsKey(kmsKey);
+ }
+ }
+ if (sensitiveFieldsData.getData().data().contains(lookupFieldName) && event.getExtractedFields().containsKey(lookupFieldName)) {
+ String value = (String) event.getExtractedFields().get(lookupFieldName);
+ if (StringUtil.isNotBlank(value)) {
+ String encryptResult = encryptionAlgorithm.encrypt(value);
+ if (StringUtil.isEmpty(encryptResult)) {
+ event.getExtractedFields().put(outputFieldName, StringUtil.isNotBlank(defaultVal) ? defaultVal : value);
+ } else {
+ if (KmsUtils.KMS_TYPE_VAULT.equals(type)) {
+ encryptResult = "vault:v" + encryptionAlgorithm.getKmsKey().getKeyVersion() + ":" + encryptResult;
+ }
+ event.getExtractedFields().put(outputFieldName, encryptResult);
+ }
+ }
+ }
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ return event;
+ }
+
+ @Override
+ public String functionName() {
+ return "ENCRYPT";
+ }
+
+ @Override
+ public void close() {
+ if (sensitiveFieldsData != null) {
+ sensitiveFieldsData.release();
+ }
+ if (kmsKeyData != null) {
+ kmsKeyData.release();
+ }
+ }
+
+ private void checkUdfContext(UDFContext udfContext) {
+ if (udfContext.getParameters() == null) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
+ }
+ if (udfContext.getLookupFields().size() != 1) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function lookup fields only support 1 value");
+ }
+ if (udfContext.getOutputFields().size() != 1) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value");
+ }
+ if (!udfContext.getParameters().containsKey("identifier")) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Parameters must contains identifier");
+ }
+ }
+
+ public Set<String> getSensitiveFields(String url) throws IOException {
+ Set<String> sensitiveFieldsSet;
+ String sensitiveFieldsStr = HttpClientPoolUtil.getInstance().httpGet(URI.create(URLUtil.normalize(url)), new BasicHeader(HttpHeaders.CONTENT_TYPE, "application/x-www-form-urlencoded"));
+ JSONObject sensitiveFieldsJson = JSONUtil.parseObj(sensitiveFieldsStr);
+ if (sensitiveFieldsJson.getInt("status", HttpStatus.SC_INTERNAL_SERVER_ERROR) == HttpStatus.SC_OK) {
+ JSONArray sensitiveFieldsJsonArr = sensitiveFieldsJson.getJSONArray("data");
+ sensitiveFieldsSet = IntStream.range(0, sensitiveFieldsJsonArr.size())
+ .mapToObj(sensitiveFieldsJsonArr::getStr)
+ .collect(Collectors.toSet());
+ } else {
+ throw new GrootStreamRuntimeException(CommonErrorCode.UNSUPPORTED_OPERATION, "Get encrypt fields error! Error message: " + sensitiveFieldsStr);
+ }
+ return sensitiveFieldsSet;
+ }
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/Eval.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/Eval.java
index 1b83d94..b04dc97 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/Eval.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/Eval.java
@@ -21,14 +21,14 @@ public class Eval implements ScalarFunction {
@Override
public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
- if(udfContext.getOutput_fields()==null || udfContext.getParameters() == null){
+ if(udfContext.getOutputFields()==null || udfContext.getParameters() == null){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
}
if(!udfContext.getParameters().containsKey("value_expression")){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters must containkey value_expression");
}
String expr = (String) udfContext.getParameters().get("value_expression");
- List<String> outputField = udfContext.getOutput_fields();
+ List<String> outputField = udfContext.getOutputFields();
output = outputField.get(0);
calc = new EvalExecutor(expr);
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/Flatten.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/Flatten.java
index 84c2c2a..d5d5761 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/Flatten.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/Flatten.java
@@ -30,7 +30,7 @@ public class Flatten implements ScalarFunction {
prefix = udfContext.getParameters().getOrDefault("prefix", "").toString();
delimiter = udfContext.getParameters().getOrDefault("delimiter", ".").toString();
flattenKeys = new HashSet<>();
- for (String key : udfContext.getLookup_fields()) {
+ for (String key : udfContext.getLookupFields()) {
this.flattenKeys.add(prefix.isEmpty() ? key : prefix + delimiter + key);
}
depth = Integer.parseInt(udfContext.getParameters().getOrDefault("depth", "5").toString());
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/FromUnixTimestamp.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/FromUnixTimestamp.java
index e1ba384..d8803c3 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/FromUnixTimestamp.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/FromUnixTimestamp.java
@@ -9,66 +9,73 @@ import lombok.extern.slf4j.Slf4j;
import org.apache.flink.api.common.functions.RuntimeContext;
import java.text.SimpleDateFormat;
+import java.util.Arrays;
import java.util.TimeZone;
@Slf4j
public class FromUnixTimestamp implements ScalarFunction {
private String precision;
private String outputFieldName;
private String lookupFieldName;
+
+ private String timeZone = "UTC";
+
private SimpleDateFormat sdf;
@Override
public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
- if(udfContext.getLookup_fields()==null || udfContext.getOutput_fields()==null || udfContext.getParameters() == null){
+ if(udfContext.getLookupFields()==null || udfContext.getOutputFields()==null || udfContext.getParameters() == null){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
}
- if(udfContext.getOutput_fields().size() != 1){
+ if(udfContext.getOutputFields().size() != 1){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value");
}
- if(udfContext.getLookup_fields().size() != 1){
+ if(udfContext.getLookupFields().size() != 1){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function lookup fields only support 1 value");
}
if(!udfContext.getParameters().containsKey("precision")){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters must containkey precision");
}
else{
- if(!udfContext.getParameters().get("precision").toString().equals("seconds") &&
- !udfContext.getParameters().get("precision").toString().equals("milliseconds") &&
- !udfContext.getParameters().get("precision").toString().equals("microseconds") &&
- !udfContext.getParameters().get("precision").toString().equals("nanoseconds") ){
+ if (!Arrays.asList("seconds", "milliseconds").contains(udfContext.getParameters().get("precision").toString().trim())) {
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters precision value is not correct");
}
}
+ if(udfContext.getParameters().containsKey("timezone")){
+ timeZone = udfContext.getParameters().get("timezone").toString();
+ }
+
this.precision = udfContext.getParameters().get("precision").toString();
- this.outputFieldName = udfContext.getOutput_fields().get(0);
- this.lookupFieldName = udfContext.getLookup_fields().get(0);
+ this.outputFieldName = udfContext.getOutputFields().get(0);
+ this.lookupFieldName = udfContext.getLookupFields().get(0);
this.sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
switch (precision) {
case "seconds":
sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
break;
case "milliseconds":
- sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss:SSS");
- break;
- case "microseconds":
- sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss:SSS:000");
- break;
- case "nanoseconds":
- sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss:SSS:000:000");
+ sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
break;
default:
break;
}
- sdf.setTimeZone(TimeZone.getTimeZone("UTC"));
+ sdf.setTimeZone(TimeZone.getTimeZone(timeZone));
}
@Override
public Event evaluate(Event event) {
- if(event.getExtractedFields().containsKey(lookupFieldName)){
- String timestamp = sdf.format(Long.parseLong(event.getExtractedFields().get(lookupFieldName).toString()));
- event.getExtractedFields().put(outputFieldName, timestamp);
+ Object objectValue = event.getExtractedFields().get(lookupFieldName);
+ if(objectValue!=null){
+ String value = objectValue.toString();
+ String dateTimeFormat ="";
+ try {
+ long timestamp = Long.parseLong(value);
+ dateTimeFormat = sdf.format(timestamp >= 10000000000L ? timestamp : timestamp * 1000);
+ } catch (NumberFormatException e) {
+ log.error("Invalid timestamp format for field {}: {}", lookupFieldName, value, e);
+ }
+ event.getExtractedFields().put(outputFieldName, dateTimeFormat);
}
return event;
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/GenerateStringArray.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/GenerateStringArray.java
index ce4fc48..366f204 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/GenerateStringArray.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/GenerateStringArray.java
@@ -18,14 +18,14 @@ public class GenerateStringArray implements ScalarFunction {
@Override
public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
- if(udfContext.getLookup_fields()==null || udfContext.getOutput_fields()==null ){
+ if(udfContext.getLookupFields()==null || udfContext.getOutputFields()==null ){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
}
- if(udfContext.getOutput_fields().size() != 1){
+ if(udfContext.getOutputFields().size() != 1){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value");
}
- this.lookupFieldNames = udfContext.getLookup_fields();
- this.outputFieldName = udfContext.getOutput_fields().get(0);
+ this.lookupFieldNames = udfContext.getLookupFields();
+ this.outputFieldName = udfContext.getOutputFields().get(0);
}
@Override
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/GeoIpLookup.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/GeoIpLookup.java
index afe1bfb..e800e5d 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/GeoIpLookup.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/GeoIpLookup.java
@@ -1,8 +1,7 @@
package com.geedgenetworks.core.udf;
import com.geedgenetworks.common.Constants;
-import com.geedgenetworks.common.config.CommonConfig;
-import com.geedgenetworks.common.config.KnowledgeBaseConfig;
+import com.geedgenetworks.common.config.*;
import com.geedgenetworks.common.exception.CommonErrorCode;
import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
import com.geedgenetworks.common.udf.ScalarFunction;
@@ -28,13 +27,55 @@ public class GeoIpLookup implements ScalarFunction {
private String outputFieldName;
private Map<String, String> geoLocationFieldMapping;
+ enum Option {
+ IP_TO_COUNTRY,
+ IP_TO_PROVINCE,
+ IP_TO_CITY,
+ IP_TO_SUBDIVISION_ADDR,
+ IP_TO_DETAIL,
+ IP_TO_LATLNG,
+ IP_TO_PROVIDER,
+ IP_TO_JSON,
+ IP_TO_OBJECT
+ ;
+
+ public static boolean isValid(String option) {
+ try {
+ Option.valueOf(option);
+ return true;
+ } catch (IllegalArgumentException e) {
+ return false;
+ }
+ }
+ }
+
+ enum GeolocationFieldMapping {
+ COUNTRY,
+ PROVINCE,
+ CITY,
+ LONGITUDE,
+ LATITUDE,
+ ISP,
+ ORGANIZATION
+ ;
+
+ public static boolean isValid(String option) {
+ try {
+ GeolocationFieldMapping.valueOf(option);
+ return true;
+ } catch (IllegalArgumentException e) {
+ return false;
+ }
+ }
+ }
+
@Override
public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
- checkUdfContext(udfContext);
- this.kbName = udfContext.getParameters().get("kb_name").toString();
- this.option = udfContext.getParameters().get("option").toString();
- if (option.equals("IP_TO_OBJECT")) {
- this.geoLocationFieldMapping = (Map<String, String>) udfContext.getParameters().get("geolocation_field_mapping");
+ checkConfig(udfContext);
+ this.kbName = udfContext.getParameters().get(UDFContextConfigOptions.PARAMETERS_KB_NAME.key()).toString();
+ this.option = udfContext.getParameters().get(UDFContextConfigOptions.PARAMETERS_OPTION.key()).toString();
+ if (option.equals(Option.IP_TO_OBJECT.name())) {
+ this.geoLocationFieldMapping = (Map<String, String>) udfContext.getParameters().get(UDFContextConfigOptions.PARAMETERS_GEOLOCATION_FIELD_MAPPING.key());
}
Configuration configuration = (Configuration) runtimeContext
.getExecutionConfig().getGlobalJobParameters();
@@ -50,9 +91,9 @@ public class GeoIpLookup implements ScalarFunction {
} else {
log.error("GeoIpLookup init KnowledgeBase error ");
}
- this.lookupFieldName = udfContext.getLookup_fields().get(0);
- if(udfContext.getOutput_fields()!=null && !udfContext.getOutput_fields().isEmpty()){
- this.outputFieldName = udfContext.getOutput_fields().get(0);
+ this.lookupFieldName = udfContext.getLookupFields().get(0);
+ if(udfContext.getOutputFields()!=null && !udfContext.getOutputFields().isEmpty()){
+ this.outputFieldName = udfContext.getOutputFields().get(0);
}
GeoIpKnowledgeBaseHandler.increment();
log.warn("GeoIpKnowledgeBaseHandler "+GeoIpKnowledgeBaseHandler.getCount());
@@ -61,10 +102,10 @@ public class GeoIpLookup implements ScalarFunction {
@Override
public Event evaluate(Event event) {
- if (event.getExtractedFields().containsKey(lookupFieldName)) {
-
- if ("IP_TO_OBJECT".equals(option)) {
- LocationResponse response = GeoIpKnowledgeBaseHandler.lookUpObject(kbName,event.getExtractedFields().get(lookupFieldName).toString());
+ Object valueObj = event.getExtractedFields().get(lookupFieldName);
+ if (valueObj!=null) {
+ if (Option.IP_TO_OBJECT.name().equals(option)) {
+ LocationResponse response = GeoIpKnowledgeBaseHandler.lookUpObject(kbName,valueObj.toString());
for (Map.Entry<String, String> entry : geoLocationFieldMapping.entrySet()) {
String result = "";
if (response!=null) {
@@ -97,65 +138,56 @@ public class GeoIpLookup implements ScalarFunction {
}
}
} else {
- event.getExtractedFields().put(outputFieldName, GeoIpKnowledgeBaseHandler
- .lookUp(kbName, option, event.getExtractedFields().get(lookupFieldName).toString()));
+ event.getExtractedFields().put(outputFieldName, GeoIpKnowledgeBaseHandler.lookUp(kbName, option, valueObj.toString()));
}
}
return event;
}
- private void checkUdfContext(UDFContext udfContext) {
+ @Override
+ public void checkConfig(UDFContext udfContext) {
+ CheckResult result = CheckUDFContextUtil.checkAllExists(udfContext,
+ UDFContextConfigOptions.LOOKUP_FIELDS.key(),
+ UDFContextConfigOptions.PARAMETERS.key());
- if (udfContext.getLookup_fields() == null || udfContext.getParameters() == null) {
- throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
- }
- if (udfContext.getLookup_fields().size() != 1) {
- throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function lookup fields only support 1 value");
+ if (!result.isSuccess()) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.CONFIG_VALIDATION_FAILED, result.getMsg());
}
- if (!udfContext.getParameters().containsKey("kb_name")) {
- throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters need containkey kb_name");
- }
- if (!udfContext.getParameters().containsKey("option")) {
- throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters need containkey option");
- } else {
- if (!udfContext.getParameters().get("option").toString().equals("IP_TO_COUNTRY") && //IP_TO_COUNTRY
- !udfContext.getParameters().get("option").toString().equals("IP_TO_PROVINCE") && //IP_TO_PROVINCE
- !udfContext.getParameters().get("option").toString().equals("IP_TO_CITY") && //IP_TO_CITY
- !udfContext.getParameters().get("option").toString().equals("IP_TO_SUBDIVISION_ADDR") && //IP_TO_SUBDIVISION_ADDR
- !udfContext.getParameters().get("option").toString().equals("IP_TO_DETAIL") && //IP_TO_DETAIL
- !udfContext.getParameters().get("option").toString().equals("IP_TO_LATLNG") && //IP_TO_LATLNG
- !udfContext.getParameters().get("option").toString().equals("IP_TO_PROVIDER") && //IP_TO_PROVIDER
- !udfContext.getParameters().get("option").toString().equals("IP_TO_JSON") && //IP_TO_JSON
- !udfContext.getParameters().get("option").toString().equals("IP_TO_OBJECT")) {
- throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters option value is not correct");
- }
- if (udfContext.getParameters().get("option").toString().equals("IP_TO_OBJECT")) {
- if (!udfContext.getParameters().containsKey("geolocation_field_mapping")) {
+ result = CheckUDFContextUtil.checkCollectionSingleItemExists(udfContext, UDFContextConfigOptions.LOOKUP_FIELDS.key());
+ if (!result.isSuccess()) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, result.getMsg());
+ }
- throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters need containkey geolocation_field_mapping");
+ result = CheckUDFContextUtil.checkParametersContainsKeys(udfContext, UDFContextConfigOptions.PARAMETERS_KB_NAME.key(), UDFContextConfigOptions.PARAMETERS_OPTION.key());
+ if (!result.isSuccess()) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, result.getMsg());
+ }
- } else {
- Map<String, String> geolocation_field_mapping = (Map<String, String>) udfContext.getParameters().get("geolocation_field_mapping");
+ String optionValue = udfContext.getParameters().get(UDFContextConfigOptions.PARAMETERS_OPTION.key()).toString();
+ if (!Option.isValid(optionValue)) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, String.format( "UDF: %s, [%s] Option value is not correct.",
+ udfContext.getFunction(), UDFContextConfigOptions.PARAMETERS.key()));
+ }
- if (!geolocation_field_mapping.isEmpty()) {
+ if (optionValue.equals(Option.IP_TO_OBJECT.name())) {
+ result = CheckUDFContextUtil.checkParametersContainsKeys(udfContext, UDFContextConfigOptions.PARAMETERS_GEOLOCATION_FIELD_MAPPING.key());
+ if (!result.isSuccess()) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, result.getMsg());
+ }
- for (Map.Entry<String, String> entry : geolocation_field_mapping.entrySet()) {
+ Map<String, String> fieldMap = (Map<String, String>) udfContext.getParameters().get(UDFContextConfigOptions.PARAMETERS_GEOLOCATION_FIELD_MAPPING.key());
- if (!entry.getKey().equals("COUNTRY") && !entry.getKey().equals("PROVINCE") && !entry.getKey().equals("CITY") && !entry.getKey().equals("LONGITUDE") && !entry.getKey().equals("LATITUDE") && !entry.getKey().equals("ISP") && !entry.getKey().equals("ORGANIZATION")) {
- throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters geolocation_field_mapping value is not correct");
- }
- }
- } else {
- throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters geolocation_field_mapping value is not correct");
- }
+ for (Map.Entry<String, String> entry : fieldMap.entrySet()) {
+ if (!GeolocationFieldMapping.isValid(entry.getKey())) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters geolocation_field_mapping value is not correct");
}
}
+
}
}
-
@Override
public String functionName() {
return "GEOIP_LOOKUP";
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/Hmac.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/Hmac.java
new file mode 100644
index 0000000..098cdef
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/Hmac.java
@@ -0,0 +1,104 @@
+package com.geedgenetworks.core.udf;
+
+import cn.hutool.crypto.digest.HMac;
+import cn.hutool.crypto.digest.HmacAlgorithm;
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.exception.CommonErrorCode;
+import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
+import com.geedgenetworks.common.udf.ScalarFunction;
+import com.geedgenetworks.common.udf.UDFContext;
+import com.geedgenetworks.utils.StringUtil;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+@Slf4j
+public class Hmac implements ScalarFunction {
+
+ private String lookupFieldName;
+ private String outputFieldName;
+ private String outputFormat;
+ private HMac hMac;
+
+ @Override
+ public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
+ checkUdfContext(udfContext);
+ String secretKey = udfContext.getParameters().get("secret_key").toString();
+ String algorithm = "sha256";
+ if (udfContext.getParameters().containsKey("algorithm")) {
+ algorithm = udfContext.getParameters().get("algorithm").toString();
+ }
+ this.hMac = new HMac(getHmacAlgorithm(algorithm), secretKey.getBytes());
+ this.lookupFieldName = udfContext.getLookupFields().get(0);
+ this.outputFieldName = udfContext.getOutputFields().get(0);
+ this.outputFormat = "base64";
+ if (udfContext.getParameters().containsKey("output_format")) {
+ this.outputFormat = udfContext.getParameters().get("output_format").toString();
+ }
+ }
+
+ @Override
+ public Event evaluate(Event event) {
+ String encodeResult = "";
+ String message = (String) event.getExtractedFields().get(lookupFieldName);
+ if (StringUtil.isNotBlank(message)) {
+ switch (outputFormat) {
+ case "hex":
+ encodeResult = hMac.digestHex(message);
+ break;
+ case "base64":
+ encodeResult = hMac.digestBase64(message, false);
+ break;
+ default:
+ encodeResult = hMac.digestBase64(message, false);
+ break;
+ }
+ }
+ event.getExtractedFields().put(outputFieldName, encodeResult);
+ return event;
+ }
+
+ @Override
+ public String functionName() {
+ return "HMAC";
+ }
+
+ @Override
+ public void close() {
+
+ }
+
+ private void checkUdfContext(UDFContext udfContext) {
+ if (udfContext.getParameters() == null || udfContext.getOutputFields() == null) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
+ }
+ if (udfContext.getLookupFields().size() != 1) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function lookup fields only support 1 value");
+ }
+ if (udfContext.getOutputFields().size() != 1) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value");
+ }
+ if (!udfContext.getParameters().containsKey("secret_key")) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters must contains secret_key");
+ }
+ }
+
+ private String getHmacAlgorithm(String algorithm) {
+ if (StringUtil.containsIgnoreCase(algorithm, "sha256")) {
+ return HmacAlgorithm.HmacSHA256.getValue();
+ } else if (StringUtil.containsIgnoreCase(algorithm, "sha1")) {
+ return HmacAlgorithm.HmacSHA1.getValue();
+ } else if (StringUtil.containsIgnoreCase(algorithm, "md5")) {
+ return HmacAlgorithm.HmacMD5.getValue();
+ } else if (StringUtil.containsIgnoreCase(algorithm, "sha384")) {
+ return HmacAlgorithm.HmacSHA384.getValue();
+ } else if (StringUtil.containsIgnoreCase(algorithm, "sha512")) {
+ return HmacAlgorithm.HmacSHA512.getValue();
+ } else if (StringUtil.containsIgnoreCase(algorithm, "sm3")) {
+ return HmacAlgorithm.HmacSM3.getValue();
+ } else if (StringUtil.containsIgnoreCase(algorithm, "sm4")) {
+ return HmacAlgorithm.SM4CMAC.getValue();
+ } else {
+ return HmacAlgorithm.HmacSHA256.getValue();
+ }
+ }
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/JsonExtract.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/JsonExtract.java
index f78b952..57fe847 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/JsonExtract.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/JsonExtract.java
@@ -17,20 +17,20 @@ public class JsonExtract implements ScalarFunction {
@Override
public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
- if(udfContext.getLookup_fields()==null || udfContext.getOutput_fields()==null || udfContext.getParameters() == null){
+ if(udfContext.getLookupFields()==null || udfContext.getOutputFields()==null || udfContext.getParameters() == null){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
}
- if(udfContext.getLookup_fields().size() != 1){
+ if(udfContext.getLookupFields().size() != 1){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function lookup fields only support 1 value");
}
- if(udfContext.getOutput_fields().size() != 1){
+ if(udfContext.getOutputFields().size() != 1){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value");
}
if(!udfContext.getParameters().containsKey("value_expression")){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters must containkey value_expression");
}
- this.lookupFieldName = udfContext.getLookup_fields().get(0);
- this.outputFieldName = udfContext.getOutput_fields().get(0);
+ this.lookupFieldName = udfContext.getLookupFields().get(0);
+ this.outputFieldName = udfContext.getOutputFields().get(0);
this.expression =udfContext.getParameters().get("value_expression").toString();
}
@@ -39,14 +39,9 @@ public class JsonExtract implements ScalarFunction {
@Override
public Event evaluate(Event event) {
- if (event.getExtractedFields().containsKey(lookupFieldName)) {
-
- String result =
- (String)
- JsonPathUtil.analysis(
- event.getExtractedFields()
- .get(lookupFieldName)
- .toString(),expression);
+ Object valueObj = event.getExtractedFields().get(lookupFieldName);
+ if (valueObj!=null) {
+ String result = (String) JsonPathUtil.analysis(valueObj.toString(),expression);
event.getExtractedFields().put(outputFieldName, result);
}
return event;
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/PathCombine.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/PathCombine.java
index 874735d..0141a46 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/PathCombine.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/PathCombine.java
@@ -55,7 +55,7 @@ public class PathCombine implements ScalarFunction {
}
}
}
- this.outputFieldName = udfContext.getOutput_fields().get(0);
+ this.outputFieldName = udfContext.getOutputFields().get(0);
}
@@ -65,9 +65,9 @@ public class PathCombine implements ScalarFunction {
List<String> pathBuilder = new LinkedList<>();
for (Map.Entry<String, String> entry : pathParameters.entrySet()) {
if (entry.getValue().isEmpty()) {
-
- if(event.getExtractedFields().containsKey(entry.getKey())) {
- pathBuilder.add(event.getExtractedFields().get(entry.getKey()).toString());
+ Object valueObj = event.getExtractedFields().get(entry.getKey());
+ if (valueObj!=null) {
+ pathBuilder.add(valueObj.toString());
}
else {
return event;
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/SnowflakeId.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/SnowflakeId.java
index 42a19e6..b206f3b 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/SnowflakeId.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/SnowflakeId.java
@@ -17,15 +17,15 @@ public class SnowflakeId implements Serializable, ScalarFunction {
@Override
public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
- if(udfContext.getOutput_fields()==null || udfContext.getParameters() == null ){
+ if(udfContext.getOutputFields()==null || udfContext.getParameters() == null ){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
}
- if(udfContext.getOutput_fields().size() != 1){
+ if(udfContext.getOutputFields().size() != 1){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value");
}
String data_center_id_num = udfContext.getParameters().getOrDefault("data_center_id_num","0").toString();//转为数字
snowflakeIdUtils = new SnowflakeIdUtils(Integer.parseInt(data_center_id_num),runtimeContext.getIndexOfThisSubtask());
- this.outputFieldName = udfContext.getOutput_fields().get(0);
+ this.outputFieldName = udfContext.getOutputFields().get(0);
}
@Override
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/StringJoiner.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/StringJoiner.java
index df5c5b4..7e4ab68 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/StringJoiner.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/StringJoiner.java
@@ -21,14 +21,14 @@ public class StringJoiner implements ScalarFunction {
@Override
public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
- if(udfContext.getLookup_fields()==null || udfContext.getOutput_fields()==null || udfContext.getParameters() == null){
+ if(udfContext.getLookupFields()==null || udfContext.getOutputFields()==null || udfContext.getParameters() == null){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
}
- if(udfContext.getOutput_fields().size() != 1){
+ if(udfContext.getOutputFields().size() != 1){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value");
}
- this.lookupFieldNames = udfContext.getLookup_fields();
- this.outputFieldName = udfContext.getOutput_fields().get(0);
+ this.lookupFieldNames = udfContext.getLookupFields();
+ this.outputFieldName = udfContext.getOutputFields().get(0);
this.separator =udfContext.getParameters().getOrDefault("separator","").toString().trim();
this.prefix =udfContext.getParameters().getOrDefault("prefix","").toString().trim();
this.suffix =udfContext.getParameters().getOrDefault("suffix","").toString().trim();
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/UnixTimestampConverter.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/UnixTimestampConverter.java
index bdb41e0..62c4dfa 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/UnixTimestampConverter.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/UnixTimestampConverter.java
@@ -23,13 +23,13 @@ public class UnixTimestampConverter implements ScalarFunction {
public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
this.udfContext = udfContext;
- if(udfContext.getLookup_fields()==null || udfContext.getOutput_fields()==null || udfContext.getParameters() == null){
+ if(udfContext.getLookupFields()==null || udfContext.getOutputFields()==null || udfContext.getParameters() == null){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
}
- if(udfContext.getOutput_fields().size() != 1){
+ if(udfContext.getOutputFields().size() != 1){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value");
}
- if(udfContext.getLookup_fields().size() != 1){
+ if(udfContext.getLookupFields().size() != 1){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value");
}
if(!udfContext.getParameters().containsKey("precision")){
@@ -51,26 +51,22 @@ public class UnixTimestampConverter implements ScalarFunction {
else{
this.interval = Long.parseLong(udfContext.getParameters().get("interval").toString());
}
- this.lookupFieldName = udfContext.getLookup_fields().get(0);
- this.outputFieldName = udfContext.getOutput_fields().get(0);
+ this.lookupFieldName = udfContext.getLookupFields().get(0);
+ this.outputFieldName = udfContext.getOutputFields().get(0);
}
@Override
public Event evaluate(Event event) {
-
- if(event.getExtractedFields().containsKey(lookupFieldName)) {
- Long timestamp = Long.parseLong(event.getExtractedFields().get(lookupFieldName).toString());
- Instant instant = null;
- if (String.valueOf(timestamp).length() == 13) {
- // 时间戳长度大于10,表示为毫秒级时间戳
+ Object valueObj = event.getExtractedFields().get(lookupFieldName);
+ if (valueObj!=null) {
+ Long timestamp = Long.parseLong(valueObj.toString());
+ Instant instant ;
+ if (String.valueOf(timestamp).length() > 10) {
instant = Instant.ofEpochMilli(timestamp);
- } else if (String.valueOf(timestamp).length() == 10) {
- // 时间戳长度小于等于10,表示为秒级时间戳
- instant = Instant.ofEpochSecond(timestamp);
} else {
- return event;
+ instant = Instant.ofEpochSecond(timestamp);
}
switch (precision) {
case "seconds":
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/AbstractKnowledgeScalarFunction.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/AbstractKnowledgeScalarFunction.java
index 7c4aca2..e54b612 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/AbstractKnowledgeScalarFunction.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/AbstractKnowledgeScalarFunction.java
@@ -50,14 +50,14 @@ public abstract class AbstractKnowledgeScalarFunction implements ScalarFunction
registerKnowledges();
- this.lookupFieldName = udfContext.getLookup_fields().get(0);
- if (udfContext.getOutput_fields() != null && udfContext.getOutput_fields().size() > 0) {
- this.outputFieldName = udfContext.getOutput_fields().get(0);
+ this.lookupFieldName = udfContext.getLookupFields().get(0);
+ if (udfContext.getOutputFields() != null && udfContext.getOutputFields().size() > 0) {
+ this.outputFieldName = udfContext.getOutputFields().get(0);
}
}
private String buildMetricPrefix(UDFContext udfContext) {
- return functionName().toLowerCase() + "_" + udfContext.getLookup_fields().get(0);
+ return functionName().toLowerCase() + "_" + udfContext.getLookupFields().get(0);
}
protected abstract void registerKnowledges();
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/ArrayElementsPrepend.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/ArrayElementsPrepend.java
index a591884..de64073 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/ArrayElementsPrepend.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/ArrayElementsPrepend.java
@@ -24,8 +24,8 @@ public class ArrayElementsPrepend implements ScalarFunction {
@Override
public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
this.prefix = udfContext.getParameters().get("prefix").toString();
- this.lookupFieldName = udfContext.getLookup_fields().get(0);
- this.outputFieldName = udfContext.getOutput_fields().get(0);
+ this.lookupFieldName = udfContext.getLookupFields().get(0);
+ this.outputFieldName = udfContext.getOutputFields().get(0);
}
@Override
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/BaseStationLookup.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/BaseStationLookup.java
new file mode 100644
index 0000000..191edd5
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/BaseStationLookup.java
@@ -0,0 +1,81 @@
+package com.geedgenetworks.core.udf.cn;
+
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.exception.CommonErrorCode;
+import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
+import com.geedgenetworks.common.udf.UDFContext;
+import com.geedgenetworks.core.udf.knowlegdebase.KnowledgeBaseUpdateJob;
+import com.geedgenetworks.core.udf.knowlegdebase.handler.BaseStationKnowledgeBaseHandler;
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * @author gujinkai
+ * @version 1.0
+ * @date 2024/8/19 15:21
+ */
+public class BaseStationLookup extends AbstractKnowledgeScalarFunction {
+
+ private String cellIdFieldName;
+
+ private String longitudeFieldName;
+
+ private String latitudeFieldName;
+
+ private BaseStationKnowledgeBaseHandler knowledgeBaseHandler;
+
+ @Override
+ public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
+ super.open(runtimeContext, udfContext);
+ if (udfContext.getLookupFields() == null || udfContext.getOutputFields() == null || udfContext.getParameters() == null) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
+ }
+
+ int lookupFieldsSize = 1;
+ if (udfContext.getLookupFields().size() != lookupFieldsSize) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "lookup_fields must contain field: cell_id");
+ }
+
+ int outputFieldsSize = 2;
+ if (udfContext.getOutputFields().size() != outputFieldsSize) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "output_fields must contain two fields: longitude and latitude");
+ }
+
+ List<String> lookupFields = udfContext.getLookupFields();
+ cellIdFieldName = lookupFields.get(0);
+ List<String> outputFields = udfContext.getOutputFields();
+ longitudeFieldName = outputFields.get(0);
+ latitudeFieldName = outputFields.get(1);
+ }
+
+ @Override
+ public Event evaluate(Event event) {
+ Map<String, Object> extractedFields = event.getExtractedFields();
+ if (extractedFields == null || extractedFields.get(cellIdFieldName) == null) {
+ return event;
+ }
+ BaseStationKnowledgeBaseHandler.BaseStationLocation lookup = knowledgeBaseHandler.lookup(extractedFields.get(cellIdFieldName).toString());
+ if (lookup != null) {
+ extractedFields.put(longitudeFieldName, lookup.getLongitude());
+ extractedFields.put(latitudeFieldName, lookup.getLatitude());
+ }
+ return event;
+ }
+
+ @Override
+ public String functionName() {
+ return "BASE_STATION_LOOKUP";
+ }
+
+ @Override
+ public void close() {
+ }
+
+ @Override
+ protected void registerKnowledges() {
+ knowledgeBaseHandler = BaseStationKnowledgeBaseHandler.getInstance();
+ KnowledgeBaseUpdateJob.registerKnowledgeBase(knowledgeBaseHandler, knowledgeBaseConfigs.get(0));
+ }
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/FieldsMerge.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/FieldsMerge.java
index 6cd0c6b..f4338fc 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/FieldsMerge.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/FieldsMerge.java
@@ -22,8 +22,8 @@ public class FieldsMerge implements ScalarFunction {
@Override
public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
- this.lookupFieldNames = udfContext.getLookup_fields();
- this.outputFieldName = udfContext.getOutput_fields().get(0);
+ this.lookupFieldNames = udfContext.getLookupFields();
+ this.outputFieldName = udfContext.getOutputFields().get(0);
}
@Override
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/H3CellLookup.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/H3CellLookup.java
new file mode 100644
index 0000000..7389f4a
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/H3CellLookup.java
@@ -0,0 +1,84 @@
+package com.geedgenetworks.core.udf.cn;
+
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.exception.CommonErrorCode;
+import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
+import com.geedgenetworks.common.udf.ScalarFunction;
+import com.geedgenetworks.common.udf.UDFContext;
+import com.uber.h3core.H3Core;
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * @author gujinkai
+ * @version 1.0
+ * @date 2024/8/19 14:50
+ */
+public class H3CellLookup implements ScalarFunction {
+
+ private String longitudeFieldName;
+
+ private String latitudeFieldName;
+
+ private String outputFieldName;
+
+ private int res;
+
+ private H3Core h3;
+
+
+ @Override
+ public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
+ if (udfContext.getLookupFields() == null || udfContext.getOutputFields() == null || udfContext.getParameters() == null) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
+ }
+
+ int lookupFieldsSize = 2;
+ if (udfContext.getLookupFields().size() != lookupFieldsSize) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "lookup_fields must contain two fields: longitude and latitude");
+ }
+
+ String resolution = "resolution";
+ int maxResolution = 15;
+ if (!udfContext.getParameters().containsKey(resolution) || !(udfContext.getParameters().get(resolution) instanceof Integer) || (int) udfContext.getParameters().get(resolution) < 0 || (int) udfContext.getParameters().get(resolution) > maxResolution) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters must contain key resolution and the value is between 0 and 15");
+ }
+
+ List<String> lookupFields = udfContext.getLookupFields();
+ longitudeFieldName = lookupFields.get(0);
+ latitudeFieldName = lookupFields.get(1);
+ outputFieldName = udfContext.getOutputFields().get(0);
+ res = (int) udfContext.getParameters().get("resolution");
+ try {
+ h3 = H3Core.newInstance();
+ } catch (IOException io) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.FILE_OPERATION_ERROR, "Failed to create H3Core instance");
+ }
+ }
+
+ @Override
+ public Event evaluate(Event event) {
+ Map<String, Object> extractedFields = event.getExtractedFields();
+ if (extractedFields == null || extractedFields.get(longitudeFieldName) == null || extractedFields.get(latitudeFieldName) == null) {
+ return event;
+ }
+ if (!(extractedFields.get(longitudeFieldName) instanceof Double) || !(extractedFields.get(latitudeFieldName) instanceof Double)) {
+ return event;
+ }
+ String cellAddress = h3.latLngToCellAddress((double) extractedFields.get(latitudeFieldName), (double) extractedFields.get(longitudeFieldName), res);
+ extractedFields.put(outputFieldName, cellAddress);
+ return event;
+ }
+
+ @Override
+ public String functionName() {
+ return "H3_CELL_LOOKUP";
+ }
+
+ @Override
+ public void close() {
+ }
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/IntelligenceIndicatorLookup.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/IntelligenceIndicatorLookup.java
index 9f14bd2..857ae74 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/IntelligenceIndicatorLookup.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/IntelligenceIndicatorLookup.java
@@ -57,6 +57,17 @@ public class IntelligenceIndicatorLookup extends AbstractKnowledgeScalarFunction
}
}
break;
+ case "SUBSCRIBER_TO_TAG":
+ List<String> subscriberTags = knowledgeBaseHandler.lookupBySubscriber(lookupValue);
+ if (subscriberTags != null && subscriberTags.size() > 0) {
+ hitCounter.inc();
+ if (event.getExtractedFields().get(outputFieldName) != null && event.getExtractedFields().get(outputFieldName) instanceof List) {
+ ((List<String>) event.getExtractedFields().get(outputFieldName)).addAll(subscriberTags);
+ } else {
+ event.getExtractedFields().put(outputFieldName, subscriberTags);
+ }
+ }
+ break;
default:
logger.error("unknown option :" + option);
break;
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/AES128GCM96.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/AES128GCM96.java
new file mode 100644
index 0000000..90669b3
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/AES128GCM96.java
@@ -0,0 +1,84 @@
+package com.geedgenetworks.core.udf.encrypt;
+
+import cn.hutool.core.util.RandomUtil;
+import com.geedgenetworks.core.pojo.KmsKey;
+
+import javax.crypto.Cipher;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import java.util.Base64;
+
+public class AES128GCM96 implements EncryptionAlgorithm {
+ private static final String IDENTIFIER = "aes-128-gcm96";
+ private static final String ALGORITHM = "AES";
+ private static final String TRANSFORMATION = "AES/GCM/NoPadding";
+ private static final int GCM_TAG_LENGTH = 128;
+ private static final int GCM_96_NONCE_LENGTH = 12;
+ private static final int SECRET_KEY_LENGTH = 16;
+ private static final byte[] DEFAULT_SECRET_KEY = ".geedgenetworks.".getBytes();
+
+ private final Cipher cipher;
+ private KmsKey kmsKey;
+
+ public AES128GCM96() throws Exception {
+ this.cipher = Cipher.getInstance(TRANSFORMATION);
+ this.kmsKey = new KmsKey(DEFAULT_SECRET_KEY, 1);
+ }
+
+ @Override
+ public String getIdentifier() {
+ return IDENTIFIER;
+ }
+
+ @Override
+ public int getSecretKeyLength() {
+ return SECRET_KEY_LENGTH;
+ }
+
+ @Override
+ public KmsKey getKmsKey() {
+ return kmsKey;
+ }
+
+ @Override
+ public void setKmsKey(KmsKey kmsKey) {
+ this.kmsKey = kmsKey;
+ }
+
+ @Override
+ public String encrypt(String content) {
+ String encryptedString = "";
+ try {
+ byte[] nonce = RandomUtil.randomBytes(GCM_96_NONCE_LENGTH);
+ GCMParameterSpec gcmSpec = new GCMParameterSpec(GCM_TAG_LENGTH, nonce);
+ cipher.init(Cipher.ENCRYPT_MODE, new SecretKeySpec(kmsKey.getKeyData(), ALGORITHM), gcmSpec);
+ byte[] encryptedBytes = cipher.doFinal(content.getBytes());
+ byte[] combinedBytes = new byte[GCM_96_NONCE_LENGTH + encryptedBytes.length];
+ System.arraycopy(nonce, 0, combinedBytes, 0, GCM_96_NONCE_LENGTH);
+ System.arraycopy(encryptedBytes, 0, combinedBytes, GCM_96_NONCE_LENGTH, encryptedBytes.length);
+ encryptedString = Base64.getEncoder().encodeToString(combinedBytes);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ return encryptedString;
+ }
+
+ @Override
+ public String decrypt(String content) {
+ String decryptedString = "";
+ try {
+ byte[] nonce = RandomUtil.randomBytes(GCM_96_NONCE_LENGTH);
+ byte[] combined = Base64.getDecoder().decode(content);
+ byte[] encryptedBytes = new byte[combined.length - GCM_96_NONCE_LENGTH];
+ System.arraycopy(combined, 0, nonce, 0, GCM_96_NONCE_LENGTH);
+ System.arraycopy(combined, GCM_96_NONCE_LENGTH, encryptedBytes, 0, encryptedBytes.length);
+ GCMParameterSpec gcmSpec = new GCMParameterSpec(GCM_TAG_LENGTH, nonce);
+ cipher.init(Cipher.DECRYPT_MODE, new SecretKeySpec(kmsKey.getKeyData(), ALGORITHM), gcmSpec);
+ byte[] decryptedBytes = cipher.doFinal(encryptedBytes);
+ decryptedString = new String(decryptedBytes);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ return decryptedString;
+ }
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/AES256GCM96.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/AES256GCM96.java
new file mode 100644
index 0000000..0306616
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/AES256GCM96.java
@@ -0,0 +1,84 @@
+package com.geedgenetworks.core.udf.encrypt;
+
+import cn.hutool.core.util.RandomUtil;
+import com.geedgenetworks.core.pojo.KmsKey;
+
+import javax.crypto.Cipher;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import java.util.Base64;
+
+public class AES256GCM96 implements EncryptionAlgorithm {
+ private static final String IDENTIFIER = "aes-256-gcm96";
+ private static final String ALGORITHM = "AES";
+ private static final String TRANSFORMATION = "AES/GCM/NoPadding";
+ private static final int GCM_TAG_LENGTH = 128;
+ private static final int GCM_96_NONCE_LENGTH = 12;
+ private static final int SECRET_KEY_LENGTH = 32;
+ private static final byte[] DEFAULT_SECRET_KEY = ".........geedgenetworks.........".getBytes();
+
+ private final Cipher cipher;
+ private KmsKey kmsKey;
+
+ public AES256GCM96() throws Exception {
+ this.cipher = Cipher.getInstance(TRANSFORMATION);
+ this.kmsKey = new KmsKey(DEFAULT_SECRET_KEY, 1);
+ }
+
+ @Override
+ public String getIdentifier() {
+ return IDENTIFIER;
+ }
+
+ @Override
+ public int getSecretKeyLength() {
+ return SECRET_KEY_LENGTH;
+ }
+
+ @Override
+ public KmsKey getKmsKey() {
+ return kmsKey;
+ }
+
+ @Override
+ public void setKmsKey(KmsKey kmsKey) {
+ this.kmsKey = kmsKey;
+ }
+
+ @Override
+ public String encrypt(String content) {
+ String encryptedString = "";
+ try {
+ byte[] nonce = RandomUtil.randomBytes(GCM_96_NONCE_LENGTH);
+ GCMParameterSpec gcmSpec = new GCMParameterSpec(GCM_TAG_LENGTH, nonce);
+ cipher.init(Cipher.ENCRYPT_MODE, new SecretKeySpec(kmsKey.getKeyData(), ALGORITHM), gcmSpec);
+ byte[] encryptedBytes = cipher.doFinal(content.getBytes());
+ byte[] combinedBytes = new byte[GCM_96_NONCE_LENGTH + encryptedBytes.length];
+ System.arraycopy(nonce, 0, combinedBytes, 0, GCM_96_NONCE_LENGTH);
+ System.arraycopy(encryptedBytes, 0, combinedBytes, GCM_96_NONCE_LENGTH, encryptedBytes.length);
+ encryptedString = Base64.getEncoder().encodeToString(combinedBytes);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ return encryptedString;
+ }
+
+ @Override
+ public String decrypt(String content) {
+ String decryptedString = "";
+ try {
+ byte[] nonce = RandomUtil.randomBytes(GCM_96_NONCE_LENGTH);
+ byte[] combined = Base64.getDecoder().decode(content);
+ byte[] encryptedBytes = new byte[combined.length - GCM_96_NONCE_LENGTH];
+ System.arraycopy(combined, 0, nonce, 0, GCM_96_NONCE_LENGTH);
+ System.arraycopy(combined, GCM_96_NONCE_LENGTH, encryptedBytes, 0, encryptedBytes.length);
+ GCMParameterSpec gcmSpec = new GCMParameterSpec(GCM_TAG_LENGTH, nonce);
+ cipher.init(Cipher.DECRYPT_MODE, new SecretKeySpec(kmsKey.getKeyData(), ALGORITHM), gcmSpec);
+ byte[] decryptedBytes = cipher.doFinal(encryptedBytes);
+ decryptedString = new String(decryptedBytes);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ return decryptedString;
+ }
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/EncryptionAlgorithm.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/EncryptionAlgorithm.java
new file mode 100644
index 0000000..3fc4e74
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/EncryptionAlgorithm.java
@@ -0,0 +1,17 @@
+package com.geedgenetworks.core.udf.encrypt;
+
+import com.geedgenetworks.core.pojo.KmsKey;
+
+public interface EncryptionAlgorithm {
+ String getIdentifier();
+
+ int getSecretKeyLength();
+
+ KmsKey getKmsKey();
+
+ void setKmsKey(KmsKey kmsKey);
+
+ String encrypt(String content);
+
+ String decrypt(String content);
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/SM4GCM96.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/SM4GCM96.java
new file mode 100644
index 0000000..f4ad0a2
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/SM4GCM96.java
@@ -0,0 +1,84 @@
+package com.geedgenetworks.core.udf.encrypt;
+
+import cn.hutool.core.util.RandomUtil;
+import com.geedgenetworks.core.pojo.KmsKey;
+
+import javax.crypto.Cipher;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import java.util.Base64;
+
+public class SM4GCM96 implements EncryptionAlgorithm {
+ private static final String IDENTIFIER = "sm4-gcm96";
+ private static final String ALGORITHM = "SM4";
+ private static final String TRANSFORMATION = "SM4/GCM/NoPadding";
+ private static final int GCM_TAG_LENGTH = 128;
+ private static final int GCM_96_NONCE_LENGTH = 12;
+ private static final int SECRET_KEY_LENGTH = 16;
+ private static final byte[] DEFAULT_SECRET_KEY = ".geedgenetworks.".getBytes();
+
+ private final Cipher cipher;
+ private KmsKey kmsKey;
+
+ public SM4GCM96() throws Exception {
+ this.cipher = Cipher.getInstance(TRANSFORMATION);
+ this.kmsKey = new KmsKey(DEFAULT_SECRET_KEY, 1);
+ }
+
+ @Override
+ public String getIdentifier() {
+ return IDENTIFIER;
+ }
+
+ @Override
+ public int getSecretKeyLength() {
+ return SECRET_KEY_LENGTH;
+ }
+
+ @Override
+ public KmsKey getKmsKey() {
+ return kmsKey;
+ }
+
+ @Override
+ public void setKmsKey(KmsKey kmsKey) {
+ this.kmsKey = kmsKey;
+ }
+
+ @Override
+ public String encrypt(String content) {
+ String encryptedString = "";
+ try {
+ byte[] nonce = RandomUtil.randomBytes(GCM_96_NONCE_LENGTH);
+ GCMParameterSpec gcmSpec = new GCMParameterSpec(GCM_TAG_LENGTH, nonce);
+ cipher.init(Cipher.ENCRYPT_MODE, new SecretKeySpec(kmsKey.getKeyData(), ALGORITHM), gcmSpec);
+ byte[] encryptedBytes = cipher.doFinal(content.getBytes());
+ byte[] combinedBytes = new byte[GCM_96_NONCE_LENGTH + encryptedBytes.length];
+ System.arraycopy(nonce, 0, combinedBytes, 0, GCM_96_NONCE_LENGTH);
+ System.arraycopy(encryptedBytes, 0, combinedBytes, GCM_96_NONCE_LENGTH, encryptedBytes.length);
+ encryptedString = Base64.getEncoder().encodeToString(combinedBytes);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ return encryptedString;
+ }
+
+ @Override
+ public String decrypt(String content) {
+ String decryptedString = "";
+ try {
+ byte[] nonce = RandomUtil.randomBytes(GCM_96_NONCE_LENGTH);
+ byte[] combined = Base64.getDecoder().decode(content);
+ byte[] encryptedBytes = new byte[combined.length - GCM_96_NONCE_LENGTH];
+ System.arraycopy(combined, 0, nonce, 0, GCM_96_NONCE_LENGTH);
+ System.arraycopy(combined, GCM_96_NONCE_LENGTH, encryptedBytes, 0, encryptedBytes.length);
+ GCMParameterSpec gcmSpec = new GCMParameterSpec(GCM_TAG_LENGTH, nonce);
+ cipher.init(Cipher.DECRYPT_MODE, new SecretKeySpec(kmsKey.getKeyData(), ALGORITHM), gcmSpec);
+ byte[] decryptedBytes = cipher.doFinal(encryptedBytes);
+ decryptedString = new String(decryptedBytes);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ return decryptedString;
+ }
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/AbstractSingleKnowledgeBaseHandler.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/AbstractSingleKnowledgeBaseHandler.java
index 6ac292c..759ab38 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/AbstractSingleKnowledgeBaseHandler.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/AbstractSingleKnowledgeBaseHandler.java
@@ -31,6 +31,13 @@ public abstract class AbstractSingleKnowledgeBaseHandler extends AbstractKnowled
if ("http".equals(knowledgeBaseConfig.getFsType())) {
this.knowledgeMetedataCache = getMetadata(knowledgeBaseConfig.getFsType(), knowledgeBaseConfig.getFsPath(), knowledgeBaseConfig.getFiles().get(0));
}
+ if ("local".equals(knowledgeBaseConfig.getFsType())) {
+ knowledgeMetedataCache = new KnowLedgeBaseFileMeta();
+ knowledgeMetedataCache.setIsValid(1);
+ knowledgeMetedataCache.setFormat(knowledgeBaseConfig.getFiles().get(0).substring(knowledgeBaseConfig.getFiles().get(0).lastIndexOf(".") + 1));
+ knowledgeMetedataCache.setName(knowledgeBaseConfig.getFiles().get(0));
+ knowledgeMetedataCache.setPath(knowledgeBaseConfig.getFsPath() + knowledgeBaseConfig.getFiles().get(0));
+ }
return buildKnowledgeBase();
}
@@ -81,11 +88,8 @@ public abstract class AbstractSingleKnowledgeBaseHandler extends AbstractKnowled
case "aes":
result = AESUtil.decrypt(data, AES_KEY);
break;
- case "csv":
- result = data;
- break;
default:
- logger.error("unknown format: " + knowledgeMetedataCache.getFormat());
+ result = data;
}
} catch (Exception e) {
logger.error("decrypt error", e);
@@ -102,6 +106,10 @@ public abstract class AbstractSingleKnowledgeBaseHandler extends AbstractKnowled
knowledgeMetedataCache = knowledgeMetedata;
return true;
}
+ if (knowledgeMetedata.getIsValid() != knowledgeMetedataCache.getIsValid()) {
+ knowledgeMetedataCache = knowledgeMetedata;
+ return true;
+ }
if (knowledgeMetedataCache.getSha256().equals(knowledgeMetedata.getSha256())) {
return false;
} else {
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/AsnKnowledgeBaseHandler.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/AsnKnowledgeBaseHandler.java
index a1927db..a0b9ce5 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/AsnKnowledgeBaseHandler.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/AsnKnowledgeBaseHandler.java
@@ -117,6 +117,7 @@ public class AsnKnowledgeBaseHandler extends AbstractKnowledgeBaseHandler {
}
} catch (Exception e) {
+ log.error("Current class path {}", this.getClass().getProtectionDomain().getCodeSource().getLocation().getPath());
log.error("File {} operation failed. {} ", knowledgeBaseConfig.getFiles().get(i), e.getMessage());
return false;
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/BaseStationKnowledgeBaseHandler.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/BaseStationKnowledgeBaseHandler.java
new file mode 100644
index 0000000..0eeae53
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/BaseStationKnowledgeBaseHandler.java
@@ -0,0 +1,90 @@
+package com.geedgenetworks.core.udf.knowlegdebase.handler;
+
+import com.geedgenetworks.core.utils.cn.csv.HighCsvReader;
+import lombok.Data;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * @author gujinkai
+ * @version 1.0
+ * @date 2024/8/26 16:27
+ */
+public class BaseStationKnowledgeBaseHandler extends AbstractSingleKnowledgeBaseHandler {
+
+ private static final Logger logger = LoggerFactory.getLogger(BaseStationKnowledgeBaseHandler.class);
+
+ private HashMap<String, BaseStationLocation> baseStationLocationHashMap = new HashMap<>();
+
+ private BaseStationKnowledgeBaseHandler() {
+ }
+
+ private static final class InstanceHolder {
+ private static final BaseStationKnowledgeBaseHandler instance = new BaseStationKnowledgeBaseHandler();
+ }
+
+ public static BaseStationKnowledgeBaseHandler getInstance() {
+ return BaseStationKnowledgeBaseHandler.InstanceHolder.instance;
+ }
+
+ @Override
+ public Boolean buildKnowledgeBase() {
+ try {
+ List<String> needColumns = new ArrayList<>();
+ needColumns.add("cell_id");
+ needColumns.add("longitude");
+ needColumns.add("latitude");
+ byte[] content = downloadFile();
+ HighCsvReader highCsvReader = new HighCsvReader(new InputStreamReader(new ByteArrayInputStream(content)), needColumns);
+ HashMap<String, BaseStationLocation> newBaseStationLocationHashMap = new HashMap<>((int) (highCsvReader.getLineNumber() / 0.75F + 1.0F));
+ HighCsvReader.CsvIterator iterator = highCsvReader.getIterator();
+ while (iterator.hasNext()) {
+ Map<String, String> line = iterator.next();
+ try {
+ String cellId = line.get("cell_id");
+ BaseStationLocation baseStationLocation = new BaseStationLocation();
+ baseStationLocation.setLongitude(parseDouble(line.get("longitude")));
+ baseStationLocation.setLatitude(parseDouble(line.get("latitude")));
+ newBaseStationLocationHashMap.put(cellId, baseStationLocation);
+ } catch (Exception lineException) {
+ logger.error(this.getClass().getSimpleName() + " line: " + line.toString() + " parse error:" + lineException, lineException);
+ }
+ }
+ baseStationLocationHashMap = newBaseStationLocationHashMap;
+ } catch (Exception e) {
+ logger.error(this.getClass().getSimpleName() + " update error", e);
+ return false;
+ }
+ return true;
+ }
+
+ public BaseStationLocation lookup(String ecgi) {
+ return baseStationLocationHashMap.get(ecgi);
+ }
+
+ public void close() {
+ baseStationLocationHashMap.clear();
+ baseStationLocationHashMap = null;
+ }
+
+ private double parseDouble(String str) {
+ try {
+ return Double.parseDouble(str);
+ } catch (Exception e) {
+ return 0;
+ }
+ }
+
+ @Data
+ public static final class BaseStationLocation {
+ private double longitude;
+ private double latitude;
+ }
+} \ No newline at end of file
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/IntelligenceIndicatorKnowledgeBaseHandler.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/IntelligenceIndicatorKnowledgeBaseHandler.java
index 94fdae1..093dd1d 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/IntelligenceIndicatorKnowledgeBaseHandler.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/IntelligenceIndicatorKnowledgeBaseHandler.java
@@ -31,6 +31,8 @@ public class IntelligenceIndicatorKnowledgeBaseHandler extends AbstractSingleKno
// *开头,模糊匹配
private Trie<String> domainSuffix = new Trie<>();
+ private HashMap<String, List<String>> subscriberTagMap = new HashMap<>();
+
private IntelligenceIndicatorKnowledgeBaseHandler() {
}
@@ -51,12 +53,14 @@ public class IntelligenceIndicatorKnowledgeBaseHandler extends AbstractSingleKno
needColumns.add("ip1");
needColumns.add("ip2");
needColumns.add("domain");
+ needColumns.add("subscriber");
needColumns.add("tags");
byte[] content = downloadFile();
HighCsvReader highCsvReader = new HighCsvReader(new InputStreamReader(new ByteArrayInputStream(content)), needColumns);
TreeRangeMap<IPAddress, List<String>> newIpTagMap = TreeRangeMap.create();
HashMap<String, List<String>> newDomainMap = new HashMap<>((int) (highCsvReader.getLineNumber() / 0.75F + 1.0F));
Trie<String> newDomainSuffix = new Trie<>();
+ HashMap<String, List<String>> newSubscriberTagMap = new HashMap<>();
HighCsvReader.CsvIterator iterator = highCsvReader.getIterator();
while (iterator.hasNext()) {
Map<String, String> line = iterator.next();
@@ -66,6 +70,7 @@ public class IntelligenceIndicatorKnowledgeBaseHandler extends AbstractSingleKno
String ip1 = line.get("ip1");
String ip2 = line.get("ip2");
String domain = line.get("domain");
+ String subscriberId = line.get("subscriber");
List<String> tags = Arrays.asList(line.get("tags").split(","));
if ("IP".equals(type)) {
@@ -121,6 +126,8 @@ public class IntelligenceIndicatorKnowledgeBaseHandler extends AbstractSingleKno
} else {
logger.warn("intelligence indicator find unknown domain: " + domain);
}
+ } else if ("Subscriber".equals(type)) {
+ newSubscriberTagMap.computeIfAbsent(subscriberId, k -> new ArrayList<>()).addAll(tags);
}
} catch (Exception lineException) {
logger.error(this.getClass().getSimpleName() + " line: " + line.toString() + " parse error:" + lineException, lineException);
@@ -129,6 +136,7 @@ public class IntelligenceIndicatorKnowledgeBaseHandler extends AbstractSingleKno
ipTagMap = newIpTagMap;
domainTagMap = newDomainMap;
domainSuffix = newDomainSuffix;
+ subscriberTagMap = newSubscriberTagMap;
} catch (Exception e) {
logger.error(this.getClass().getSimpleName() + " update error", e);
return false;
@@ -155,6 +163,10 @@ public class IntelligenceIndicatorKnowledgeBaseHandler extends AbstractSingleKno
return result;
}
+ public List<String> lookupBySubscriber(String subscriber) {
+ return subscriberTagMap.computeIfAbsent(subscriber, k -> new ArrayList<>());
+ }
+
@Override
public void close() {
ipTagMap.clear();
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectList.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectList.java
index 3921ee2..8219fbd 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectList.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectList.java
@@ -36,12 +36,12 @@ public class CollectList implements AggregateFunction {
@Override
public void open(UDFContext udfContext) {
- if (udfContext.getLookup_fields() == null) {
+ if (udfContext.getLookupFields() == null) {
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
}
- this.lookupField = udfContext.getLookup_fields().get(0);
- if (udfContext.getOutput_fields() != null && !udfContext.getOutput_fields().isEmpty()) {
- this.outputField = udfContext.getOutput_fields().get(0);
+ this.lookupField = udfContext.getLookupFields().get(0);
+ if (udfContext.getOutputFields() != null && !udfContext.getOutputFields().isEmpty()) {
+ this.outputField = udfContext.getOutputFields().get(0);
} else {
outputField = lookupField;
}
@@ -56,8 +56,9 @@ public class CollectList implements AggregateFunction {
@Override
public Accumulator add(Event event, Accumulator acc) {
- if (event.getExtractedFields().containsKey(lookupField)) {
- Object object = event.getExtractedFields().get(lookupField);
+ Object valueObj = event.getExtractedFields().get(lookupField);
+ if (valueObj != null) {
+ Object object = valueObj;
List<Object> aggregate = (List<Object>) acc.getMetricsFields().get(outputField);
aggregate.add(object);
acc.getMetricsFields().put(outputField, aggregate);
@@ -77,12 +78,14 @@ public class CollectList implements AggregateFunction {
@Override
public Accumulator merge(Accumulator firstAcc, Accumulator secondAcc) {
- if (firstAcc.getMetricsFields().containsKey(outputField) && secondAcc.getMetricsFields().containsKey(outputField)) {
- List<Object> firstValue = (List<Object>) firstAcc.getMetricsFields().get(outputField);
- List<Object> secondValue = (List<Object>) secondAcc.getMetricsFields().get(outputField);
+ Object firstValueObj = firstAcc.getMetricsFields().get(outputField);
+ Object secondValueObj = secondAcc.getMetricsFields().get(outputField);
+ if (firstValueObj != null && secondValueObj != null) {
+ List<Object> firstValue = (List<Object>) firstValueObj;
+ List<Object> secondValue = (List<Object>) secondValueObj;
firstValue.addAll(secondValue);
- } else if (!firstAcc.getMetricsFields().containsKey(outputField) && secondAcc.getMetricsFields().containsKey(outputField)) {
- List<Object> secondValue = (List<Object>) secondAcc.getMetricsFields().get(outputField);
+ } else if (firstValueObj == null && secondValueObj != null) {
+ List<Object> secondValue = (List<Object>) secondValueObj;
firstAcc.getMetricsFields().put(outputField, secondValue);
}
return firstAcc;
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectSet.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectSet.java
index 9ec9b09..c23f0ca 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectSet.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectSet.java
@@ -22,12 +22,12 @@ public class CollectSet implements AggregateFunction {
@Override
public void open(UDFContext udfContext) {
- if (udfContext.getLookup_fields() == null) {
+ if (udfContext.getLookupFields() == null) {
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
}
- this.lookupField = udfContext.getLookup_fields().get(0);
- if (udfContext.getOutput_fields() != null && !udfContext.getOutput_fields().isEmpty()) {
- this.outputField = udfContext.getOutput_fields().get(0);
+ this.lookupField = udfContext.getLookupFields().get(0);
+ if (udfContext.getOutputFields() != null && !udfContext.getOutputFields().isEmpty()) {
+ this.outputField = udfContext.getOutputFields().get(0);
} else {
outputField = lookupField;
}
@@ -41,8 +41,9 @@ public class CollectSet implements AggregateFunction {
@Override
public Accumulator add(Event event, Accumulator acc) {
- if (event.getExtractedFields().containsKey(lookupField)) {
- Object object = event.getExtractedFields().get(lookupField);
+ Object valueObj = event.getExtractedFields().get(lookupField);
+ if (valueObj != null) {
+ Object object = valueObj;
Set<Object> aggregate = (Set<Object>) acc.getMetricsFields().get(outputField);
aggregate.add(object);
acc.getMetricsFields().put(outputField, aggregate);
@@ -62,12 +63,14 @@ public class CollectSet implements AggregateFunction {
@Override
public Accumulator merge(Accumulator firstAcc, Accumulator secondAcc) {
- if (firstAcc.getMetricsFields().containsKey(outputField) && secondAcc.getMetricsFields().containsKey(outputField)) {
- Set<Object> firstValue = (Set<Object>) firstAcc.getMetricsFields().get(outputField);
- Set<Object> secondValue = (Set<Object>) secondAcc.getMetricsFields().get(outputField);
+ Object firstValueObj = firstAcc.getMetricsFields().get(outputField);
+ Object secondValueObj = secondAcc.getMetricsFields().get(outputField);
+ if (firstValueObj != null && secondValueObj != null) {
+ Set<Object> firstValue = (Set<Object>) firstValueObj;
+ Set<Object> secondValue = (Set<Object>) secondValueObj;
firstValue.addAll(secondValue);
- } else if (!firstAcc.getMetricsFields().containsKey(outputField) && secondAcc.getMetricsFields().containsKey(outputField)) {
- Set<Object> secondValue = (Set<Object>) secondAcc.getMetricsFields().get(outputField);
+ } else if (firstValueObj == null && secondValueObj !=null) {
+ Set<Object> secondValue = (Set<Object>)secondValueObj;
firstAcc.getMetricsFields().put(outputField, secondValue);
}
return firstAcc;
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/FirstValue.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/FirstValue.java
index a1a35be..f68448f 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/FirstValue.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/FirstValue.java
@@ -34,12 +34,12 @@ public class FirstValue implements AggregateFunction {
@Override
public void open(UDFContext udfContext) {
- if (udfContext.getLookup_fields() == null) {
+ if (udfContext.getLookupFields() == null) {
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
}
- this.lookupField = udfContext.getLookup_fields().get(0);
- if (udfContext.getOutput_fields() != null && !udfContext.getOutput_fields().isEmpty()) {
- this.outputField = udfContext.getOutput_fields().get(0);
+ this.lookupField = udfContext.getLookupFields().get(0);
+ if (udfContext.getOutputFields() != null && !udfContext.getOutputFields().isEmpty()) {
+ this.outputField = udfContext.getOutputFields().get(0);
} else {
outputField = lookupField;
}
@@ -52,7 +52,8 @@ public class FirstValue implements AggregateFunction {
@Override
public Accumulator add(Event event, Accumulator acc) {
- if (!acc.getMetricsFields().containsKey(outputField) && event.getExtractedFields().containsKey(lookupField)) {
+ Object valueObj = event.getExtractedFields().get(lookupField);
+ if (!acc.getMetricsFields().containsKey(outputField) && valueObj != null) {
acc.getMetricsFields().put(outputField, event.getExtractedFields().get(lookupField));
}
return acc;
@@ -70,8 +71,10 @@ public class FirstValue implements AggregateFunction {
@Override
public Accumulator merge(Accumulator firstAcc, Accumulator secondAcc) {
- if (!firstAcc.getMetricsFields().containsKey(outputField) && secondAcc.getMetricsFields().containsKey(outputField)) {
- firstAcc.getMetricsFields().put(outputField, secondAcc.getMetricsFields().get(outputField));
+ Object firstValueObj = firstAcc.getMetricsFields().get(outputField);
+ Object secondValueObj = secondAcc.getMetricsFields().get(outputField);
+ if (firstValueObj == null && secondValueObj != null ) {
+ firstAcc.getMetricsFields().put(outputField, secondValueObj);
}
return firstAcc;
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramBaseAggregate.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramBaseAggregate.java
index a099fde..d8656e0 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramBaseAggregate.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramBaseAggregate.java
@@ -1,121 +1,121 @@
-package com.geedgenetworks.core.udf.udaf.HdrHistogram;
-
-import com.geedgenetworks.common.Accumulator;
-import com.geedgenetworks.common.Event;
-import com.geedgenetworks.common.udf.AggregateFunction;
-import com.geedgenetworks.common.udf.UDFContext;
-import com.geedgenetworks.sketch.util.StringUtils;
-import org.HdrHistogram.ArrayHistogram;
-import org.HdrHistogram.DirectMapHistogram;
-import org.HdrHistogram.Histogramer;
-import org.apache.commons.collections.CollectionUtils;
-
-import java.nio.charset.StandardCharsets;
-import java.util.Map;
-
-public abstract class HdrHistogramBaseAggregate implements AggregateFunction {
- protected String inputField;
- protected String outputField;
- protected boolean inputSketch;
- protected long lowestDiscernibleValue;
- protected long highestTrackableValue;
- protected int numberOfSignificantValueDigits;
- protected boolean autoResize;
-
- @Override
- public void open(UDFContext c) {
- inputField = c.getLookup_fields().get(0);
- if (CollectionUtils.isNotEmpty(c.getOutput_fields())) {
- outputField = c.getOutput_fields().get(0);
- } else {
- outputField = inputField;
- }
- Map<String, Object> params = c.getParameters();
- lowestDiscernibleValue = Long.parseLong(params.getOrDefault("lowestDiscernibleValue", "1").toString());
- highestTrackableValue = Long.parseLong(params.getOrDefault("highestTrackableValue", "2").toString());
- numberOfSignificantValueDigits = Integer.parseInt(params.getOrDefault("numberOfSignificantValueDigits", "1").toString());
- autoResize = Boolean.valueOf(params.getOrDefault("autoResize", "true").toString());
- inputSketch = "sketch".equalsIgnoreCase(params.getOrDefault("input_type", "sketch").toString());
- }
-
- @Override
- public Accumulator initAccumulator(Accumulator acc) {
- return acc;
- }
-
- @Override
- public Accumulator add(Event event, Accumulator acc) {
- Object value = event.getExtractedFields().get(inputField);
- if (value == null) {
- return acc;
- }
-
- if (inputSketch) {
- updateHdrMerge(acc, value);
- } else {
- updateHdr(acc, value);
- }
-
- return acc;
- }
-
- @Override
- public Accumulator merge(Accumulator acc, Accumulator other) {
- Object agg = acc.getMetricsFields().get(outputField);
- Object aggOther = other.getMetricsFields().get(outputField);
- Object rst;
-
- if(agg == null){
- rst = aggOther;
- } else if (aggOther == null) {
- rst = agg;
- }else{
- rst = ((Histogramer)agg).merge(((Histogramer) aggOther));
- }
-
- if(rst != null){
- acc.getMetricsFields().put(outputField, rst);
- }
- return acc;
- }
-
- protected void updateHdr(Accumulator acc, Object value) {
- Map<String, Object> aggs = acc.getMetricsFields();
- ArrayHistogram his = (ArrayHistogram) aggs.get(outputField);
- if (his == null) {
- his = new ArrayHistogram(lowestDiscernibleValue, highestTrackableValue, numberOfSignificantValueDigits);
- his.setAutoResize(autoResize);
- aggs.put(outputField, his);
- }
-
- his.recordValue(((Number) value).longValue());
- }
-
-
- protected void updateHdrMerge(Accumulator acc, Object value) {
- Map<String, Object> aggs = acc.getMetricsFields();
- ArrayHistogram his = (ArrayHistogram) aggs.get(outputField);
- if (his == null) {
- his = new ArrayHistogram(lowestDiscernibleValue, highestTrackableValue, numberOfSignificantValueDigits);
- his.setAutoResize(autoResize);
- aggs.put(outputField, his);
- }
-
- Histogramer h;
- if (value instanceof String) {
- byte[] bytes = StringUtils.decodeBase64(((String) value).getBytes(StandardCharsets.UTF_8));
- h = DirectMapHistogram.wrapBytes(bytes);
- } else if (value instanceof byte[]) {
- h = DirectMapHistogram.wrapBytes((byte[]) value);
- } else if (value instanceof Histogramer) {
- h = (Histogramer) value;
- } else {
- throw new IllegalArgumentException("Unsupported type " + value.getClass());
- }
-
- his.merge(h);
- }
-
- @Override
- public void close() {}
-}
+package com.geedgenetworks.core.udf.udaf.HdrHistogram;
+
+import com.geedgenetworks.common.Accumulator;
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.udf.AggregateFunction;
+import com.geedgenetworks.common.udf.UDFContext;
+import com.geedgenetworks.sketch.util.StringUtils;
+import org.HdrHistogram.ArrayHistogram;
+import org.HdrHistogram.DirectMapHistogram;
+import org.HdrHistogram.Histogramer;
+import org.apache.commons.collections.CollectionUtils;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+
+public abstract class HdrHistogramBaseAggregate implements AggregateFunction {
+ protected String inputField;
+ protected String outputField;
+ protected boolean inputSketch;
+ protected long lowestDiscernibleValue;
+ protected long highestTrackableValue;
+ protected int numberOfSignificantValueDigits;
+ protected boolean autoResize;
+
+ @Override
+ public void open(UDFContext c) {
+ inputField = c.getLookupFields().get(0);
+ if (CollectionUtils.isNotEmpty(c.getOutputFields())) {
+ outputField = c.getOutputFields().get(0);
+ } else {
+ outputField = inputField;
+ }
+ Map<String, Object> params = c.getParameters();
+ lowestDiscernibleValue = Long.parseLong(params.getOrDefault("lowestDiscernibleValue", "1").toString());
+ highestTrackableValue = Long.parseLong(params.getOrDefault("highestTrackableValue", "2").toString());
+ numberOfSignificantValueDigits = Integer.parseInt(params.getOrDefault("numberOfSignificantValueDigits", "1").toString());
+ autoResize = Boolean.valueOf(params.getOrDefault("autoResize", "true").toString());
+ inputSketch = "sketch".equalsIgnoreCase(params.getOrDefault("input_type", "sketch").toString());
+ }
+
+ @Override
+ public Accumulator initAccumulator(Accumulator acc) {
+ return acc;
+ }
+
+ @Override
+ public Accumulator add(Event event, Accumulator acc) {
+ Object value = event.getExtractedFields().get(inputField);
+ if (value == null) {
+ return acc;
+ }
+
+ if (inputSketch) {
+ updateHdrMerge(acc, value);
+ } else {
+ updateHdr(acc, value);
+ }
+
+ return acc;
+ }
+
+ @Override
+ public Accumulator merge(Accumulator acc, Accumulator other) {
+ Object agg = acc.getMetricsFields().get(outputField);
+ Object aggOther = other.getMetricsFields().get(outputField);
+ Object rst;
+
+ if(agg == null){
+ rst = aggOther;
+ } else if (aggOther == null) {
+ rst = agg;
+ }else{
+ rst = ((Histogramer)agg).merge(((Histogramer) aggOther));
+ }
+
+ if(rst != null){
+ acc.getMetricsFields().put(outputField, rst);
+ }
+ return acc;
+ }
+
+ protected void updateHdr(Accumulator acc, Object value) {
+ Map<String, Object> aggs = acc.getMetricsFields();
+ ArrayHistogram his = (ArrayHistogram) aggs.get(outputField);
+ if (his == null) {
+ his = new ArrayHistogram(lowestDiscernibleValue, highestTrackableValue, numberOfSignificantValueDigits);
+ his.setAutoResize(autoResize);
+ aggs.put(outputField, his);
+ }
+
+ his.recordValue(((Number) value).longValue());
+ }
+
+
+ protected void updateHdrMerge(Accumulator acc, Object value) {
+ Map<String, Object> aggs = acc.getMetricsFields();
+ ArrayHistogram his = (ArrayHistogram) aggs.get(outputField);
+ if (his == null) {
+ his = new ArrayHistogram(lowestDiscernibleValue, highestTrackableValue, numberOfSignificantValueDigits);
+ his.setAutoResize(autoResize);
+ aggs.put(outputField, his);
+ }
+
+ Histogramer h;
+ if (value instanceof String) {
+ byte[] bytes = StringUtils.decodeBase64(((String) value).getBytes(StandardCharsets.UTF_8));
+ h = DirectMapHistogram.wrapBytes(bytes);
+ } else if (value instanceof byte[]) {
+ h = DirectMapHistogram.wrapBytes((byte[]) value);
+ } else if (value instanceof Histogramer) {
+ h = (Histogramer) value;
+ } else {
+ throw new IllegalArgumentException("Unsupported type " + value.getClass());
+ }
+
+ his.merge(h);
+ }
+
+ @Override
+ public void close() {}
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LastValue.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LastValue.java
index 44b374e..f319e8d 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LastValue.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LastValue.java
@@ -34,12 +34,12 @@ public class LastValue implements AggregateFunction {
@Override
public void open(UDFContext udfContext) {
- if (udfContext.getLookup_fields() == null) {
+ if (udfContext.getLookupFields() == null) {
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
}
- this.lookupField = udfContext.getLookup_fields().get(0);
- if (udfContext.getOutput_fields() != null && !udfContext.getOutput_fields().isEmpty()) {
- this.outputField = udfContext.getOutput_fields().get(0);
+ this.lookupField = udfContext.getLookupFields().get(0);
+ if (udfContext.getOutputFields() != null && !udfContext.getOutputFields().isEmpty()) {
+ this.outputField = udfContext.getOutputFields().get(0);
} else {
outputField = lookupField;
}
@@ -70,8 +70,10 @@ public class LastValue implements AggregateFunction {
@Override
public Accumulator merge(Accumulator firstAcc, Accumulator secondAcc) {
- if (secondAcc.getMetricsFields().containsKey(outputField)) {
- firstAcc.getMetricsFields().put(outputField, secondAcc.getMetricsFields().get(outputField));
+
+ Object secondValueObj = secondAcc.getMetricsFields().get(outputField);
+ if (secondValueObj != null) {
+ firstAcc.getMetricsFields().put(outputField, secondValueObj);
}
return firstAcc;
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LongCount.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LongCount.java
index 05de38c..418eb9c 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LongCount.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LongCount.java
@@ -14,10 +14,10 @@ public class LongCount implements AggregateFunction {
@Override
public void open(UDFContext udfContext) {
- if (udfContext.getOutput_fields() == null) {
+ if (udfContext.getOutputFields() == null) {
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
}
- outputField = udfContext.getOutput_fields().get(0);
+ outputField = udfContext.getOutputFields().get(0);
}
@Override
@@ -44,14 +44,16 @@ public class LongCount implements AggregateFunction {
@Override
public Accumulator merge(Accumulator firstAcc, Accumulator secondAcc) {
- if (firstAcc.getMetricsFields().containsKey(outputField) && secondAcc.getMetricsFields().containsKey(outputField)) {
- long firstValue = (long) firstAcc.getMetricsFields().get(outputField);
- long secondValue = (long) secondAcc.getMetricsFields().get(outputField);
+ Object firstValueObj = firstAcc.getMetricsFields().get(outputField);
+ Object secondValueObj = secondAcc.getMetricsFields().get(outputField);
+ if (firstValueObj!=null && secondValueObj!=null) {
+ long firstValue = (long) firstValueObj;
+ long secondValue = (long) secondValueObj;
firstValue = firstValue + secondValue;
firstAcc.getMetricsFields().put(outputField, firstValue);
- } else if (!firstAcc.getMetricsFields().containsKey(outputField) && secondAcc.getMetricsFields().containsKey(outputField)) {
- Number secondValue = (Number) secondAcc.getMetricsFields().get(outputField);
+ } else if (firstValueObj==null && secondValueObj!=null) {
+ Number secondValue = (Number) secondValueObj;
firstAcc.getMetricsFields().put(outputField, secondValue);
}
return firstAcc;
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Max.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Max.java
new file mode 100644
index 0000000..226df0a
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Max.java
@@ -0,0 +1,121 @@
+package com.geedgenetworks.core.udf.udaf;
+
+
+import com.geedgenetworks.common.Accumulator;
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.exception.CommonErrorCode;
+import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
+import com.geedgenetworks.common.udf.AggregateFunction;
+import com.geedgenetworks.common.udf.UDFContext;
+
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Map;
+
+
+public class Max implements AggregateFunction {
+
+ private String lookupField;
+ private String outputField;
+
+ @Override
+ public void open(UDFContext udfContext) {
+ if (udfContext.getLookupFields() == null) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
+ }
+ this.lookupField = udfContext.getLookupFields().get(0);
+ if (udfContext.getOutputFields() != null && !udfContext.getOutputFields().isEmpty()) {
+ this.outputField = udfContext.getOutputFields().get(0);
+ } else {
+ outputField = lookupField;
+ }
+ }
+
+ @Override
+ public Accumulator initAccumulator(Accumulator acc) {
+ return acc;
+ }
+
+ @Override
+ public Accumulator add(Event event, Accumulator acc) {
+ Map<String, Object> eventFields = event.getExtractedFields();
+ Map<String, Object> metricsFields = acc.getMetricsFields();
+
+ if (metricsFields.get(outputField) == null && eventFields.get(lookupField) != null) {
+ metricsFields.put(outputField, eventFields.get(lookupField));
+ } else if (metricsFields.get(outputField) != null && eventFields.get(lookupField) != null) {
+ Object currentValue = metricsFields.get(outputField);
+ Object newValue = eventFields.get(lookupField);
+
+ // 直接检测是否为时间类型对象
+ if (currentValue instanceof LocalDateTime && newValue instanceof LocalDateTime) {
+ LocalDateTime time1 = (LocalDateTime) currentValue;
+ LocalDateTime time2 = (LocalDateTime) newValue;
+ if (time1.isBefore(time2)) {
+ metricsFields.put(outputField, newValue);
+ }
+ } else if (currentValue instanceof String && newValue instanceof String) {
+ // 处理字符串比较
+ String value1 = currentValue.toString();
+ String value2 = newValue.toString();
+ if (value1.compareTo(value2) < 0) {
+ metricsFields.put(outputField, newValue);
+ }
+
+ } else {
+ // 数字进行比较
+ double value1 = Double.parseDouble(currentValue.toString());
+ double value2 = Double.parseDouble(newValue.toString());
+ if (value1 < value2) {
+ metricsFields.put(outputField, newValue);
+ }
+ }
+ }
+ return acc;
+ }
+
+ @Override
+ public String functionName() {
+ return "MAX";
+ }
+
+ @Override
+ public Accumulator getResult(Accumulator acc) {
+ return acc;
+ }
+
+ @Override
+ public Accumulator merge(Accumulator firstAcc, Accumulator secondAcc) {
+ Map<String, Object> firstMetrics = firstAcc.getMetricsFields();
+ Map<String, Object> secondMetrics = secondAcc.getMetricsFields();
+ Object firstValue = firstMetrics.get(outputField);
+ Object secondValue = secondMetrics.get(outputField);
+ if (firstValue == null && secondValue != null) {
+ firstMetrics.put(outputField, secondValue);
+ } else if (firstValue != null && secondValue != null) {
+ // 直接检测是否为时间类型对象
+ if (firstValue instanceof LocalDateTime && secondValue instanceof LocalDateTime) {
+ LocalDateTime time1 = (LocalDateTime) firstValue;
+ LocalDateTime time2 = (LocalDateTime) secondValue;
+ if (time1.isBefore(time2)) {
+ firstMetrics.put(outputField, secondValue);
+ }
+ } else if (firstValue instanceof String && secondValue instanceof String) {
+
+ String value1 = firstValue.toString();
+ String value2 = secondValue.toString();
+ if (value1.compareTo(value2) < 0) {
+ firstMetrics.put(outputField, secondValue);
+ }
+ } else {
+ // 假设为数字,进行比较
+ double value1 = Double.parseDouble(firstValue.toString());
+ double value2 = Double.parseDouble(secondValue.toString());
+ if (value1 < value2) {
+ firstMetrics.put(outputField, secondValue);
+ }
+ }
+ }
+ return firstAcc;
+ }
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Mean.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Mean.java
index 9c4e070..88f693f 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Mean.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Mean.java
@@ -20,12 +20,12 @@ public class Mean implements AggregateFunction {
@Override
public void open(UDFContext udfContext) {
- if (udfContext.getLookup_fields() == null) {
+ if (udfContext.getLookupFields() == null) {
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
}
- lookupField = udfContext.getLookup_fields().get(0);
- if (udfContext.getOutput_fields() != null && !udfContext.getOutput_fields().isEmpty()) {
- outputField = udfContext.getOutput_fields().get(0);
+ lookupField = udfContext.getLookupFields().get(0);
+ if (udfContext.getOutputFields() != null && !udfContext.getOutputFields().isEmpty()) {
+ outputField = udfContext.getOutputFields().get(0);
} else {
outputField = lookupField;
}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Min.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Min.java
new file mode 100644
index 0000000..6fd7046
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Min.java
@@ -0,0 +1,120 @@
+package com.geedgenetworks.core.udf.udaf;
+
+
+import com.geedgenetworks.common.Accumulator;
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.exception.CommonErrorCode;
+import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
+import com.geedgenetworks.common.udf.AggregateFunction;
+import com.geedgenetworks.common.udf.UDFContext;
+
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Map;
+
+
+public class Min implements AggregateFunction {
+
+ private String lookupField;
+ private String outputField;
+
+ @Override
+ public void open(UDFContext udfContext) {
+ if (udfContext.getLookupFields() == null) {
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
+ }
+ this.lookupField = udfContext.getLookupFields().get(0);
+ if (udfContext.getOutputFields() != null && !udfContext.getOutputFields().isEmpty()) {
+ this.outputField = udfContext.getOutputFields().get(0);
+ } else {
+ outputField = lookupField;
+ }
+ }
+
+ @Override
+ public Accumulator initAccumulator(Accumulator acc) {
+ return acc;
+ }
+
+ @Override
+ public Accumulator add(Event event, Accumulator acc) {
+ Map<String, Object> eventFields = event.getExtractedFields();
+ Map<String, Object> metricsFields = acc.getMetricsFields();
+
+ if (metricsFields.get(outputField) == null && eventFields.get(lookupField) != null) {
+ metricsFields.put(outputField, eventFields.get(lookupField));
+ } else if (metricsFields.get(outputField) != null && eventFields.get(lookupField) != null) {
+ Object currentValue = metricsFields.get(outputField);
+ Object newValue = eventFields.get(lookupField);
+
+ // 直接检测是否为时间类型对象
+ if (currentValue instanceof LocalDateTime && newValue instanceof LocalDateTime) {
+ LocalDateTime time1 = (LocalDateTime) currentValue;
+ LocalDateTime time2 = (LocalDateTime) newValue;
+ if (time1.isAfter(time2)) {
+ metricsFields.put(outputField, newValue);
+ }
+ } else if (currentValue instanceof String && newValue instanceof String) {
+ // 处理字符串比较
+ String value1 = currentValue.toString();
+ String value2 = newValue.toString();
+ if (value1.compareTo(value2) > 0) {
+ metricsFields.put(outputField, newValue);
+ }
+
+ } else {
+ // 假设为数字,进行比较
+ double value1 = Double.parseDouble(currentValue.toString());
+ double value2 = Double.parseDouble(newValue.toString());
+ if (value1 > value2) {
+ metricsFields.put(outputField, newValue);
+ }
+ }
+ }
+ return acc;
+ }
+
+ @Override
+ public String functionName() {
+ return "MIN";
+ }
+
+ @Override
+ public Accumulator getResult(Accumulator acc) {
+ return acc;
+ }
+
+ @Override
+ public Accumulator merge(Accumulator firstAcc, Accumulator secondAcc) {
+ Map<String, Object> firstMetrics = firstAcc.getMetricsFields();
+ Map<String, Object> secondMetrics = secondAcc.getMetricsFields();
+ Object firstValue = firstMetrics.get(outputField);
+ Object secondValue = secondMetrics.get(outputField);
+ if (firstValue == null && secondValue != null) {
+ firstMetrics.put(outputField, secondValue);
+ } else if (firstValue != null && secondValue != null) {
+ // 直接检测是否为时间类型对象
+ if (firstValue instanceof LocalDateTime && secondValue instanceof LocalDateTime) {
+ LocalDateTime time1 = (LocalDateTime) firstValue;
+ LocalDateTime time2 = (LocalDateTime) secondValue;
+ if (time1.isAfter(time2)) {
+ firstMetrics.put(outputField, secondValue);
+ }
+ } else if (firstValue instanceof String && secondValue instanceof String) {
+ String value1 = firstValue.toString();
+ String value2 = secondValue.toString();
+ if (value1.compareTo(value2) > 0) {
+ firstMetrics.put(outputField, secondValue);
+ }
+ } else {
+ // 假设为数字,进行比较
+ double value1 = Double.parseDouble(firstValue.toString());
+ double value2 = Double.parseDouble(secondValue.toString());
+ if (value1 > value2) {
+ firstAcc.getMetricsFields().put(outputField, secondValue);
+ }
+ }
+ }
+ return firstAcc;
+ }
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/NumberSum.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/NumberSum.java
index e972133..ab8f744 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/NumberSum.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/NumberSum.java
@@ -15,12 +15,12 @@ public class NumberSum implements AggregateFunction {
@Override
public void open(UDFContext udfContext) {
- if (udfContext.getLookup_fields() == null) {
+ if (udfContext.getLookupFields() == null) {
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
}
- lookupField = udfContext.getLookup_fields().get(0);
- if (udfContext.getOutput_fields() != null && !udfContext.getOutput_fields().isEmpty()) {
- outputField = udfContext.getOutput_fields().get(0);
+ lookupField = udfContext.getLookupFields().get(0);
+ if (udfContext.getOutputFields() != null && !udfContext.getOutputFields().isEmpty()) {
+ outputField = udfContext.getOutputFields().get(0);
} else {
outputField = lookupField;
}
@@ -33,9 +33,9 @@ public class NumberSum implements AggregateFunction {
@Override
public Accumulator add(Event event, Accumulator acc) {
-
- if (event.getExtractedFields().containsKey(lookupField)) {
- Number val = (Number) event.getExtractedFields().get(lookupField);
+ Object valueObj = event.getExtractedFields().get(lookupField);
+ if (valueObj!=null) {
+ Number val = (Number) valueObj;
Number aggregate = (Number) acc.getMetricsFields().getOrDefault(outputField, 0L);
if (aggregate instanceof Long && (val instanceof Integer || val instanceof Long)) {
aggregate = aggregate.longValue() + val.longValue();
@@ -66,10 +66,14 @@ public class NumberSum implements AggregateFunction {
@Override
public Accumulator merge(Accumulator firstAcc, Accumulator secondAcc) {
- if (firstAcc.getMetricsFields().containsKey(outputField) && secondAcc.getMetricsFields().containsKey(outputField)) {
- Number firstValue = (Number) firstAcc.getMetricsFields().get(outputField);
- Number secondValue = (Number) secondAcc.getMetricsFields().get(outputField);
+
+ Object firstValueObj = firstAcc.getMetricsFields().get(outputField);
+ Object secondValueObj = secondAcc.getMetricsFields().get(outputField);
+ if (firstValueObj != null && secondValueObj != null) {
+
+ Number firstValue = (Number) firstValueObj;
+ Number secondValue = (Number) secondValueObj;
if (firstValue instanceof Long && (secondValue instanceof Integer || secondValue instanceof Long)) {
firstValue = firstValue.longValue() + secondValue.longValue();
} else if (firstValue instanceof Float || secondValue instanceof Float) {
@@ -78,8 +82,8 @@ public class NumberSum implements AggregateFunction {
firstValue = firstValue.doubleValue() + secondValue.doubleValue();
}
firstAcc.getMetricsFields().put(outputField, firstValue);
- } else if (!firstAcc.getMetricsFields().containsKey(outputField) && secondAcc.getMetricsFields().containsKey(outputField)) {
- Number secondValue = (Number) secondAcc.getMetricsFields().get(outputField);
+ } else if (firstValueObj == null && secondValueObj != null) {
+ Number secondValue = (Number) secondValueObj;
firstAcc.getMetricsFields().put(outputField, secondValue);
}
return firstAcc;
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/HlldBaseAggregate.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/HlldBaseAggregate.java
index 0802c22..c113c4a 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/HlldBaseAggregate.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/HlldBaseAggregate.java
@@ -1,126 +1,126 @@
-package com.geedgenetworks.core.udf.udaf.hlld;
-
-import com.geedgenetworks.common.Accumulator;
-import com.geedgenetworks.common.Event;
-import com.geedgenetworks.common.udf.AggregateFunction;
-import com.geedgenetworks.common.udf.UDFContext;
-import com.geedgenetworks.sketch.hlld.Hll;
-import com.geedgenetworks.sketch.hlld.HllUnion;
-import com.geedgenetworks.sketch.hlld.HllUtils;
-import org.apache.commons.collections.CollectionUtils;
-
-import java.util.Map;
-
-public abstract class HlldBaseAggregate implements AggregateFunction {
- protected String inputField;
- protected String outputField;
- protected boolean inputSketch;
- protected int precision = 12;
-
- @Override
- public void open(UDFContext c) {
- inputField = c.getLookup_fields().get(0);
- if (CollectionUtils.isNotEmpty(c.getOutput_fields())) {
- outputField = c.getOutput_fields().get(0);
- } else {
- outputField = inputField;
- }
- Map<String, Object> params = c.getParameters();
- precision = Integer.parseInt(params.getOrDefault("precision", "12").toString());
- inputSketch = "sketch".equalsIgnoreCase(params.getOrDefault("input_type", "sketch").toString());
- }
-
- @Override
- public Accumulator initAccumulator(Accumulator acc) {
- return acc;
- }
-
- @Override
- public Accumulator add(Event event, Accumulator acc) {
- Object value = event.getExtractedFields().get(inputField);
- if (value == null) {
- return acc;
- }
-
- if (inputSketch) {
- updateHllUnion(acc, value);
- } else {
- updateHll(acc, value);
- }
-
- return acc;
- }
-
- @Override
- public Accumulator merge(Accumulator acc, Accumulator other) {
- Object agg = acc.getMetricsFields().get(outputField);
- Object aggOther = other.getMetricsFields().get(outputField);
- Object rst;
-
- if(agg == null){
- rst = aggOther;
- } else if (aggOther == null) {
- rst = agg;
- }else{
- if(inputSketch){
- ((HllUnion)agg).update(((HllUnion) aggOther).getResult());
- rst = agg;
- }else{
- final HllUnion union = new HllUnion(precision);
- union.update((Hll) agg);
- union.update((Hll) aggOther);
- rst = union.getResult();
- }
- }
-
- if(rst != null){
- acc.getMetricsFields().put(outputField, rst);
- }
- return acc;
- }
-
- protected Hll getResultHll(Accumulator acc){
- Object agg = acc.getMetricsFields().get(outputField);
- if (agg == null) {
- return null;
- }
-
- return inputSketch ? ((HllUnion) agg).getResult() : (Hll) agg;
- }
-
- protected void updateHll(Accumulator acc, Object value) {
- Map<String, Object> aggs = acc.getMetricsFields();
- Hll hll = (Hll) aggs.get(outputField);
- if (hll == null) {
- hll = new Hll(precision);
- aggs.put(outputField, hll);
- }
-
- if (value instanceof Integer || value instanceof Long) {
- hll.add(((Number) value).longValue());
- } else if (value instanceof Float || value instanceof Double) {
- hll.add(((Number) value).doubleValue());
- } else if (value instanceof String) {
- hll.add((String) value);
- } else if (value instanceof byte[]) {
- hll.add((byte[]) value);
- } else {
- throw new IllegalArgumentException("Unsupported type " + value.getClass());
- }
- }
-
- protected void updateHllUnion(Accumulator acc, Object value) {
- Map<String, Object> aggs = acc.getMetricsFields();
- HllUnion hllUnion = (HllUnion) aggs.get(outputField);
- if (hllUnion == null) {
- hllUnion = new HllUnion(precision);
- aggs.put(outputField, hllUnion);
- }
-
- Hll hll = HllUtils.deserializeHll(value);
- hllUnion.update(hll);
- }
-
- @Override
- public void close() {}
-}
+package com.geedgenetworks.core.udf.udaf.hlld;
+
+import com.geedgenetworks.common.Accumulator;
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.udf.AggregateFunction;
+import com.geedgenetworks.common.udf.UDFContext;
+import com.geedgenetworks.sketch.hlld.Hll;
+import com.geedgenetworks.sketch.hlld.HllUnion;
+import com.geedgenetworks.sketch.hlld.HllUtils;
+import org.apache.commons.collections.CollectionUtils;
+
+import java.util.Map;
+
+public abstract class HlldBaseAggregate implements AggregateFunction {
+ protected String inputField;
+ protected String outputField;
+ protected boolean inputSketch;
+ protected int precision = 12;
+
+ @Override
+ public void open(UDFContext c) {
+ inputField = c.getLookupFields().get(0);
+ if (CollectionUtils.isNotEmpty(c.getOutputFields())) {
+ outputField = c.getOutputFields().get(0);
+ } else {
+ outputField = inputField;
+ }
+ Map<String, Object> params = c.getParameters();
+ precision = Integer.parseInt(params.getOrDefault("precision", "12").toString());
+ inputSketch = "sketch".equalsIgnoreCase(params.getOrDefault("input_type", "sketch").toString());
+ }
+
+ @Override
+ public Accumulator initAccumulator(Accumulator acc) {
+ return acc;
+ }
+
+ @Override
+ public Accumulator add(Event event, Accumulator acc) {
+ Object value = event.getExtractedFields().get(inputField);
+ if (value == null) {
+ return acc;
+ }
+
+ if (inputSketch) {
+ updateHllUnion(acc, value);
+ } else {
+ updateHll(acc, value);
+ }
+
+ return acc;
+ }
+
+ @Override
+ public Accumulator merge(Accumulator acc, Accumulator other) {
+ Object agg = acc.getMetricsFields().get(outputField);
+ Object aggOther = other.getMetricsFields().get(outputField);
+ Object rst;
+
+ if(agg == null){
+ rst = aggOther;
+ } else if (aggOther == null) {
+ rst = agg;
+ }else{
+ if(inputSketch){
+ ((HllUnion)agg).update(((HllUnion) aggOther).getResult());
+ rst = agg;
+ }else{
+ final HllUnion union = new HllUnion(precision);
+ union.update((Hll) agg);
+ union.update((Hll) aggOther);
+ rst = union.getResult();
+ }
+ }
+
+ if(rst != null){
+ acc.getMetricsFields().put(outputField, rst);
+ }
+ return acc;
+ }
+
+ protected Hll getResultHll(Accumulator acc){
+ Object agg = acc.getMetricsFields().get(outputField);
+ if (agg == null) {
+ return null;
+ }
+
+ return inputSketch ? ((HllUnion) agg).getResult() : (Hll) agg;
+ }
+
+ protected void updateHll(Accumulator acc, Object value) {
+ Map<String, Object> aggs = acc.getMetricsFields();
+ Hll hll = (Hll) aggs.get(outputField);
+ if (hll == null) {
+ hll = new Hll(precision);
+ aggs.put(outputField, hll);
+ }
+
+ if (value instanceof Integer || value instanceof Long) {
+ hll.add(((Number) value).longValue());
+ } else if (value instanceof Float || value instanceof Double) {
+ hll.add(((Number) value).doubleValue());
+ } else if (value instanceof String) {
+ hll.add((String) value);
+ } else if (value instanceof byte[]) {
+ hll.add((byte[]) value);
+ } else {
+ throw new IllegalArgumentException("Unsupported type " + value.getClass());
+ }
+ }
+
+ protected void updateHllUnion(Accumulator acc, Object value) {
+ Map<String, Object> aggs = acc.getMetricsFields();
+ HllUnion hllUnion = (HllUnion) aggs.get(outputField);
+ if (hllUnion == null) {
+ hllUnion = new HllUnion(precision);
+ aggs.put(outputField, hllUnion);
+ }
+
+ Hll hll = HllUtils.deserializeHll(value);
+ hllUnion.update(hll);
+ }
+
+ @Override
+ public void close() {}
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/JsonUnroll.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/JsonUnroll.java
index 2e8eb7e..de89e2c 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/JsonUnroll.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/JsonUnroll.java
@@ -34,12 +34,12 @@ public class JsonUnroll implements TableFunction {
@Override
public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
- if(udfContext.getLookup_fields()==null ){
+ if(udfContext.getLookupFields()==null ){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
}
- this.lookupFieldName = udfContext.getLookup_fields().get(0);
- if(udfContext.getOutput_fields()!=null && !udfContext.getOutput_fields().isEmpty()) {
- this.outputFieldName = udfContext.getOutput_fields().get(0);
+ this.lookupFieldName = udfContext.getLookupFields().get(0);
+ if(udfContext.getOutputFields()!=null && !udfContext.getOutputFields().isEmpty()) {
+ this.outputFieldName = udfContext.getOutputFields().get(0);
}
else {
outputFieldName = lookupFieldName;
@@ -57,13 +57,14 @@ public class JsonUnroll implements TableFunction {
@Override
public List<Event> evaluate(Event event) {
try {
- if(event.getExtractedFields().containsKey(lookupFieldName) ){
+ Object valueObj = event.getExtractedFields().get(lookupFieldName);
+ if (valueObj!=null) {
try {
if(path.isEmpty()){
- JSONArray jsonArray = JSONArray.parseArray(event.getExtractedFields().get(lookupFieldName).toString());
+ JSONArray jsonArray = JSONArray.parseArray(valueObj.toString());
return parseList(jsonArray,event);
}else {
- JSONObject jsonObject = JSONObject.parseObject(event.getExtractedFields().get(lookupFieldName).toString());
+ JSONObject jsonObject = JSONObject.parseObject(valueObj.toString());
Object obj = JsonPathUtil.get(jsonObject,path);
if(obj instanceof List || obj instanceof Array) {
List list = (List) obj;
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/PathUnroll.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/PathUnroll.java
new file mode 100644
index 0000000..e5732e0
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/PathUnroll.java
@@ -0,0 +1,118 @@
+package com.geedgenetworks.core.udf.udtf;
+
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.udf.TableFunction;
+import com.geedgenetworks.common.udf.UDFContext;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.util.Preconditions;
+
+import java.util.*;
+
+public class PathUnroll implements TableFunction {
+ private String pathField;
+ private String fileField;
+ private char separator;
+
+ private String outputPathField;
+ private String outputFileField;
+ private List<Event> events;
+
+ @Override
+ public void open(RuntimeContext runtimeContext, UDFContext c) {
+ Preconditions.checkArgument(c.getLookupFields().size() >= 1, "input fields requested one path param at least");
+ Preconditions.checkArgument(CollectionUtils.isEmpty(c.getOutputFields()) || c.getOutputFields().size() == c.getOutputFields().size(), "output fields requested same count param with input fields");
+ pathField = c.getLookupFields().get(0);
+ fileField = c.getLookupFields().size() == 1? null: c.getLookupFields().get(1);
+
+ outputPathField = CollectionUtils.isEmpty(c.getOutputFields())? pathField : c.getOutputFields().get(0);
+ outputFileField = CollectionUtils.isEmpty(c.getOutputFields()) || c.getLookupFields().size() == 1 ? fileField : c.getOutputFields().get(1);
+ Map<String, Object> params = c.getParameters() == null? Collections.EMPTY_MAP:c.getParameters();
+ String sep = params.getOrDefault("separator", "/").toString();
+ Preconditions.checkArgument(sep.length() == 1, "separator mush has one char");
+ separator = sep.charAt(0);
+ events = new ArrayList<>();
+ }
+
+ @Override
+ public List<Event> evaluate(Event event) {
+ Map<String, Object> map = event.getExtractedFields();
+ String p = (String) map.get(pathField);
+ // 去除path结尾的分隔符
+ final String path = StringUtils.isBlank(p)? null: (separator != p.charAt(p.length() - 1) ? p: p.substring(0, p.length() - 1));
+ final String fileName = fileField == null? null: (String) map.get(fileField);
+
+ if (StringUtils.isBlank(path)) {
+ return Collections.emptyList();
+ }
+
+ if(events.size() > 100){
+ events = new ArrayList<>();
+ }else if(events.size() > 0){
+ events.clear();
+ }
+ Event e;
+ Map<String, Object> fields;
+
+ // 拆分path
+ int index = path.indexOf(separator);
+ String subPath;
+ while (index > 0) {
+ subPath = path.substring(0, index);
+ e = new Event();
+ fields = new HashMap<>(map);
+ fields.put(outputPathField, subPath);
+ if(outputFileField != null){
+ fields.put(outputFileField, null);
+ }
+ e.setExtractedFields(fields);
+ events.add(e);
+ index = path.indexOf(separator, index + 1);
+ }
+ boolean hasFile = StringUtils.isNotBlank(fileName);
+ boolean pathContainsFile = hasFile && path.endsWith(fileName);
+
+ if(!hasFile){
+ e = new Event();
+ fields = new HashMap<>(map);
+ fields.put(outputPathField, path);
+ if(outputFileField != null){
+ fields.put(outputFileField, null);
+ }
+ e.setExtractedFields(fields);
+ events.add(e);
+ }else{
+ e = new Event();
+ fields = new HashMap<>(map);
+ fields.put(outputPathField, path);
+ if(outputFileField != null){
+ fields.put(outputFileField, pathContainsFile? fileName:null);
+ }
+ e.setExtractedFields(fields);
+ events.add(e);
+
+ // 输出path + file
+ if(!pathContainsFile){
+ e = new Event();
+ fields = new HashMap<>(map);
+ fields.put(outputPathField, path + separator + fileName);
+ if(outputFileField != null){
+ fields.put(outputFileField, fileName);
+ }
+ e.setExtractedFields(fields);
+ events.add(e);
+ }
+ }
+
+ return events;
+ }
+
+ @Override
+ public void close() {}
+
+ @Override
+ public String functionName() {
+ return "PATH_UNROLL";
+ }
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/Unroll.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/Unroll.java
index 5becb8e..ff4a9ae 100644
--- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/Unroll.java
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/Unroll.java
@@ -21,12 +21,12 @@ public class Unroll implements TableFunction {
@Override
public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
- if(udfContext.getLookup_fields()==null ){
+ if(udfContext.getLookupFields()==null ){
throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
}
- this.lookupFieldName = udfContext.getLookup_fields().get(0);
- if(udfContext.getOutput_fields()!=null && !udfContext.getOutput_fields().isEmpty()) {
- this.outputFieldName = udfContext.getOutput_fields().get(0);
+ this.lookupFieldName = udfContext.getLookupFields().get(0);
+ if(udfContext.getOutputFields()!=null && !udfContext.getOutputFields().isEmpty()) {
+ this.outputFieldName = udfContext.getOutputFields().get(0);
}
else {
outputFieldName = lookupFieldName;
@@ -42,20 +42,20 @@ public class Unroll implements TableFunction {
@Override
public List<Event> evaluate(Event event) {
try {
- if(event.getExtractedFields().containsKey(lookupFieldName)) {
-
+ Object valueObj = event.getExtractedFields().get(lookupFieldName);
+ if (valueObj!=null) {
if(regex.isEmpty()){
- if (event.getExtractedFields().get(lookupFieldName) instanceof List ) {
- return parseList(event.getExtractedFields().get(lookupFieldName), event);
- } else if(event.getExtractedFields().get(lookupFieldName) instanceof Object[]){
- return parseArray(event.getExtractedFields().get(lookupFieldName), event);
+ if (valueObj instanceof List ) {
+ return parseList(valueObj, event);
+ } else if(valueObj instanceof Object[]){
+ return parseArray(valueObj, event);
}else {
log.error("Invalid unroll ! Object is not instance of list or array. expression=" + regex);
}
}
else {
- if (event.getExtractedFields().get(lookupFieldName) instanceof String) {
- String[] array =((String) event.getExtractedFields().get(lookupFieldName)).split(regex);
+ if (valueObj instanceof String) {
+ String[] array =((String) valueObj).split(regex);
return parseArray(array, event);
}else {
log.error("Invalid unroll ! Object is not instance of String. expression=" + regex);
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUID.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUID.java
new file mode 100644
index 0000000..4e9a031
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUID.java
@@ -0,0 +1,48 @@
+package com.geedgenetworks.core.udf.uuid;
+
+import com.fasterxml.uuid.Generators;
+import com.fasterxml.uuid.impl.RandomBasedGenerator;
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.exception.CommonErrorCode;
+import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
+import com.geedgenetworks.common.udf.ScalarFunction;
+import com.geedgenetworks.common.udf.UDFContext;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+@Slf4j
+public class UUID implements ScalarFunction {
+ private String outputFieldName;
+ private RandomBasedGenerator randomBasedGenerator;
+ @Override
+ public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
+
+ if(udfContext.getOutputFields()==null ){
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
+ }
+ if(udfContext.getOutputFields().size() != 1){
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value");
+ }
+ this.outputFieldName = udfContext.getOutputFields().get(0);
+ this.randomBasedGenerator = Generators.randomBasedGenerator();
+ }
+
+ @Override
+ public Event evaluate(Event event) {
+
+ event.getExtractedFields()
+ .put(outputFieldName, randomBasedGenerator.generate().toString());
+ return event;
+ }
+
+ @Override
+ public String functionName() {
+ return "UUID";
+ }
+
+ @Override
+ public void close() {
+ }
+
+
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDNameSpace.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDNameSpace.java
new file mode 100644
index 0000000..a8941e2
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDNameSpace.java
@@ -0,0 +1,43 @@
+package com.geedgenetworks.core.udf.uuid;
+
+import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+import static com.geedgenetworks.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT;
+
+public enum UUIDNameSpace {
+
+ NAMESPACE_IP(UUID.fromString("6ba7b890-9dad-11d1-80b4-00c04fd430c8")),
+ NAMESPACE_DOMAIN(UUID.fromString("6ba7b891-9dad-11d1-80b4-00c04fd430c8")),
+ NAMESPACE_APP(UUID.fromString("6ba7b892-9dad-11d1-80b4-00c04fd430c8")),
+ NAMESPACE_SUBSCRIBER(UUID.fromString("6ba7b893-9dad-11d1-80b4-00c04fd430c8"));
+
+ private final UUID uuid;
+
+ // Static map to hold the mapping from name to UUID
+ private static final Map<String, UUID> NAME_TO_UUID_MAP = new HashMap<>();
+
+ // Static block to populate the map
+ static {
+ for (UUIDNameSpace namespace : UUIDNameSpace.values()) {
+ NAME_TO_UUID_MAP.put(namespace.name(), namespace.uuid);
+ }
+ }
+
+ UUIDNameSpace(UUID uuid) {
+ this.uuid = uuid;
+ }
+
+ public static UUID getUUID(String name) {
+ UUID uuid = NAME_TO_UUID_MAP.get(name);
+ if (uuid == null) {
+ throw new GrootStreamRuntimeException(ILLEGAL_ARGUMENT,"No enum constant " + UUIDNameSpace.class.getCanonicalName() + "." + name);
+ }
+ return uuid;
+ }
+
+
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDv5.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDv5.java
new file mode 100644
index 0000000..3a433b8
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDv5.java
@@ -0,0 +1,68 @@
+package com.geedgenetworks.core.udf.uuid;
+
+import com.fasterxml.uuid.Generators;
+import com.fasterxml.uuid.impl.NameBasedGenerator;
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.exception.CommonErrorCode;
+import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
+import com.geedgenetworks.common.udf.ScalarFunction;
+import com.geedgenetworks.common.udf.UDFContext;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+import java.util.List;
+
+@Slf4j
+public class UUIDv5 implements ScalarFunction {
+ private List<String> lookupFieldNames;
+ private String outputFieldName;
+ private NameBasedGenerator nameBasedGenerator;
+ private static final String NAMESPACE_KEY = "namespace";
+ @Override
+ public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
+
+ if(udfContext.getOutputFields() == null || udfContext.getParameters() == null || udfContext.getLookupFields() == null){
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
+ }
+ if(udfContext.getOutputFields().size() != 1){
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value");
+ }
+ if(!udfContext.getParameters().containsKey(NAMESPACE_KEY) ){
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Parameters must contain key: " + NAMESPACE_KEY);
+ }
+
+ this.outputFieldName = udfContext.getOutputFields().get(0);
+ this.lookupFieldNames = udfContext.getLookupFields();
+ String namespace = udfContext.getParameters().get(NAMESPACE_KEY).toString();
+ this.nameBasedGenerator = Generators.nameBasedGenerator(UUIDNameSpace.getUUID(namespace));
+ }
+
+ @Override
+ public Event evaluate(Event event) {
+
+ String concatenatedFields = String.join("_",
+ lookupFieldNames.stream()
+ .map(field -> event.getExtractedFields().getOrDefault(field, ""))
+ .toArray(String[]::new)
+ );
+
+ // Generate the UUID based on concatenated fields
+ String generatedUUID = nameBasedGenerator.generate(concatenatedFields).toString();
+
+ // Set the generated UUID in the output field
+ event.getExtractedFields().put(outputFieldName, generatedUUID);
+ return event;
+
+ }
+
+ @Override
+ public String functionName() {
+ return "UUIDv5";
+ }
+
+ @Override
+ public void close() {
+
+ }
+
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDv7.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDv7.java
new file mode 100644
index 0000000..60c388f
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDv7.java
@@ -0,0 +1,47 @@
+package com.geedgenetworks.core.udf.uuid;
+
+import com.fasterxml.uuid.Generators;
+import com.fasterxml.uuid.impl.TimeBasedEpochGenerator;
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.exception.CommonErrorCode;
+import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
+import com.geedgenetworks.common.udf.ScalarFunction;
+import com.geedgenetworks.common.udf.UDFContext;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+@Slf4j
+public class UUIDv7 implements ScalarFunction {
+ private String outputFieldName;
+ private TimeBasedEpochGenerator timeBasedEpochRandomGenerator;
+
+ @Override
+ public void open(RuntimeContext runtimeContext, UDFContext udfContext) {
+
+ if(udfContext.getOutputFields()==null ){
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters");
+ }
+ if(udfContext.getOutputFields().size() != 1){
+ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value");
+ }
+ this.outputFieldName = udfContext.getOutputFields().get(0);
+ this.timeBasedEpochRandomGenerator = Generators.timeBasedEpochGenerator();
+
+ }
+
+ @Override
+ public Event evaluate(Event event) {
+ event.getExtractedFields()
+ .put(outputFieldName, timeBasedEpochRandomGenerator.generate().toString());
+ return event;
+ }
+
+ @Override
+ public String functionName() {
+ return "UUIDv7";
+ }
+
+ @Override
+ public void close() {
+ }
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/utils/EncryptionAlgorithmUtils.java b/groot-core/src/main/java/com/geedgenetworks/core/utils/EncryptionAlgorithmUtils.java
new file mode 100644
index 0000000..0327e49
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/utils/EncryptionAlgorithmUtils.java
@@ -0,0 +1,30 @@
+package com.geedgenetworks.core.utils;
+
+import com.geedgenetworks.core.udf.encrypt.EncryptionAlgorithm;
+import com.geedgenetworks.core.udf.encrypt.AES128GCM96;
+import com.geedgenetworks.core.udf.encrypt.AES256GCM96;
+import com.geedgenetworks.core.udf.encrypt.SM4GCM96;
+import lombok.extern.slf4j.Slf4j;
+
+/**
+ * Crypto shade utilities
+ */
+@Slf4j
+public final class EncryptionAlgorithmUtils {
+ public static final String ALGORITHM_AES_128_GCM96_NAME = "aes-128-gcm96";
+ public static final String ALGORITHM_AES_256_GCM96_NAME = "aes-256-gcm96";
+ public static final String ALGORITHM_SM4_GCM96_NAME = "sm4-gcm96";
+
+ public static EncryptionAlgorithm createEncryptionAlgorithm(String identifier) throws Exception {
+ switch (identifier) {
+ case ALGORITHM_AES_128_GCM96_NAME:
+ return new AES128GCM96();
+ case ALGORITHM_AES_256_GCM96_NAME:
+ return new AES256GCM96();
+ case ALGORITHM_SM4_GCM96_NAME:
+ return new SM4GCM96();
+ default:
+ return null;
+ }
+ }
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/utils/KmsUtils.java b/groot-core/src/main/java/com/geedgenetworks/core/utils/KmsUtils.java
new file mode 100644
index 0000000..9519dd5
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/utils/KmsUtils.java
@@ -0,0 +1,71 @@
+package com.geedgenetworks.core.utils;
+
+import cn.hutool.core.util.StrUtil;
+import com.geedgenetworks.common.config.KmsConfig;
+import com.geedgenetworks.common.config.SSLConfig;
+import com.geedgenetworks.core.pojo.KmsKey;
+import io.github.jopenlibs.vault.SslConfig;
+import io.github.jopenlibs.vault.Vault;
+import io.github.jopenlibs.vault.VaultConfig;
+import io.github.jopenlibs.vault.VaultException;
+import io.github.jopenlibs.vault.json.JsonObject;
+import io.github.jopenlibs.vault.response.AuthResponse;
+import io.github.jopenlibs.vault.response.LogicalResponse;
+import lombok.extern.slf4j.Slf4j;
+
+import java.io.File;
+import java.util.Base64;
+
+@Slf4j
+public class KmsUtils {
+ public static final String KMS_TYPE_LOCAL = "local";
+ public static final String KMS_TYPE_VAULT = "vault";
+
+ public static KmsKey getVaultKey(KmsConfig kmsConfig, SSLConfig sslConfig, String identifier) throws Exception {
+ Vault vault = getVaultClient(kmsConfig, sslConfig);
+ String exportKeyPath;
+ if (EncryptionAlgorithmUtils.ALGORITHM_SM4_GCM96_NAME.equals(identifier)) {
+ exportKeyPath = kmsConfig.getPluginKeyPath() + "/export/encryption-key/" + identifier;
+ } else {
+ exportKeyPath = kmsConfig.getDefaultKeyPath() + "/export/encryption-key/" + identifier;
+ }
+ LogicalResponse exportResponse = vault.logical().read(exportKeyPath);
+ if (exportResponse.getRestResponse().getStatus() == 200) {
+ JsonObject keys = exportResponse.getDataObject().get("keys").asObject();
+ return new KmsKey(Base64.getDecoder().decode(StrUtil.trim(keys.get(keys.size() + "").asString(), '"')), keys.size());
+ } else {
+ throw new RuntimeException("Get vault key error! code: " + exportResponse.getRestResponse().getStatus() + " body: " + new String(exportResponse.getRestResponse().getBody()));
+ }
+ }
+
+ public static Vault getVaultClient(KmsConfig kmsConfig, SSLConfig sslConfig) throws VaultException {
+ String username = kmsConfig.getUsername();
+ String password = kmsConfig.getPassword();
+ String url = kmsConfig.getUrl();
+ boolean skipVerification = true;
+ String caCertificatePath = null;
+ String certificatePath = null;
+ String privateKeyPath = null;
+ if (sslConfig != null) {
+ skipVerification = sslConfig.getSkipVerification();
+ caCertificatePath = sslConfig.getCaCertificatePath();
+ certificatePath = sslConfig.getCertificatePath();
+ privateKeyPath = sslConfig.getPrivateKeyPath();
+ }
+ SslConfig vaultSslConfig = new SslConfig().verify(!skipVerification).build();
+ if (!skipVerification) {
+ vaultSslConfig.pemFile(new File(caCertificatePath))
+ .clientPemFile(new File(certificatePath))
+ .clientKeyPemFile(new File(privateKeyPath))
+ .build();
+ }
+ VaultConfig config = new VaultConfig()
+ .address(url)
+ .engineVersion(1)
+ .sslConfig(vaultSslConfig)
+ .build();
+ AuthResponse authResponse = Vault.create(config).auth().loginByUserPass(username, password);
+ config.token(authResponse.getAuthClientToken());
+ return Vault.create(config);
+ }
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/utils/LoadIntervalDataOptions.java b/groot-core/src/main/java/com/geedgenetworks/core/utils/LoadIntervalDataOptions.java
new file mode 100644
index 0000000..a81794d
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/utils/LoadIntervalDataOptions.java
@@ -0,0 +1,80 @@
+package com.geedgenetworks.core.utils;
+
+import java.io.Serializable;
+
+public class LoadIntervalDataOptions implements Serializable {
+ final String name;
+
+ final long intervalMs;
+ final boolean failOnException;
+ final boolean updateDataOnStart;
+
+ /**
+ * @param name 名称, 用于日志打印以及线程名称标识
+ * @param intervalMs 每隔多长时间更新数据
+ * @param failOnException 更新数据时发生异常是否失败(默认false), 为true时如果发现异常data()方法下次返回数据时会抛出异常
+ * @param updateDataOnStart start时是否先更新数据(默认true), 为false时start候intervalMs时间后才会第一个更新数据
+ */
+ private LoadIntervalDataOptions(String name, long intervalMs, boolean failOnException, boolean updateDataOnStart) {
+ this.name = name;
+ this.intervalMs = intervalMs;
+ this.failOnException = failOnException;
+ this.updateDataOnStart = updateDataOnStart;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public long getIntervalMs() {
+ return intervalMs;
+ }
+
+ public boolean isFailOnException() {
+ return failOnException;
+ }
+
+ public boolean isUpdateDataOnStart() {
+ return updateDataOnStart;
+ }
+
+ public static Builder builder() {
+ return new Builder();
+ }
+
+ public static LoadIntervalDataOptions defaults(String name, long intervalMs) {
+ return builder().withName(name).withIntervalMs(intervalMs).build();
+ }
+
+ public static final class Builder {
+ private String name = "";
+ private long intervalMs = 1000 * 60 * 10;
+ private boolean failOnException = false;
+ private boolean updateDataOnStart = true;
+
+ public Builder withName(String name) {
+ this.name = name;
+ return this;
+ }
+
+ public Builder withIntervalMs(long intervalMs) {
+ this.intervalMs = intervalMs;
+ return this;
+ }
+
+ public Builder withFailOnException(boolean failOnException) {
+ this.failOnException = failOnException;
+ return this;
+ }
+
+ public Builder withUpdateDataOnStart(boolean updateDataOnStart) {
+ this.updateDataOnStart = updateDataOnStart;
+ return this;
+ }
+
+ public LoadIntervalDataOptions build() {
+ return new LoadIntervalDataOptions(name, intervalMs, failOnException, updateDataOnStart);
+ }
+ }
+
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/utils/LoadIntervalDataUtil.java b/groot-core/src/main/java/com/geedgenetworks/core/utils/LoadIntervalDataUtil.java
new file mode 100644
index 0000000..566d217
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/utils/LoadIntervalDataUtil.java
@@ -0,0 +1,86 @@
+package com.geedgenetworks.core.utils;
+
+import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.flink.util.function.SupplierWithException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class LoadIntervalDataUtil<T> {
+ static final Logger LOG = LoggerFactory.getLogger(LoadIntervalDataUtil.class);
+
+ private final SupplierWithException<T, Exception> dataSupplier;
+ private final LoadIntervalDataOptions options;
+
+ private final AtomicBoolean started = new AtomicBoolean(false);
+ private final AtomicBoolean stopped = new AtomicBoolean(false);
+ private ScheduledExecutorService scheduler;
+ private volatile Exception exception;
+ private volatile T data;
+
+ private LoadIntervalDataUtil(SupplierWithException<T, Exception> dataSupplier, LoadIntervalDataOptions options) {
+ this.dataSupplier = dataSupplier;
+ this.options = options;
+ }
+
+ public static <T> LoadIntervalDataUtil<T> newInstance(SupplierWithException<T, Exception> dataSupplier, LoadIntervalDataOptions options) {
+ LoadIntervalDataUtil<T> loadIntervalDataUtil = new LoadIntervalDataUtil(dataSupplier, options);
+ loadIntervalDataUtil.start();
+ return loadIntervalDataUtil;
+ }
+
+ public T data() throws Exception {
+ if (!options.failOnException || exception == null) {
+ return data;
+ } else {
+ throw exception;
+ }
+ }
+
+ private void updateData() {
+ try {
+ LOG.info("{} updateData start....", options.name);
+ data = dataSupplier.get();
+ LOG.info("{} updateData end....", options.name);
+ } catch (Throwable t) {
+ if (options.failOnException) {
+ exception = new RuntimeException(t);
+ }
+ LOG.info("{} updateData error", options.name, t);
+ }
+ }
+
+ private void start() {
+ if (started.compareAndSet(false, true)) {
+ if (options.updateDataOnStart) {
+ updateData();
+ }
+ this.scheduler = newDaemonSingleThreadScheduledExecutor(String.format("LoadIntervalDataUtil[%s]", options.name));
+ this.scheduler.scheduleWithFixedDelay(() -> updateData(), options.intervalMs, options.intervalMs, TimeUnit.MILLISECONDS);
+ LOG.info("{} start....", options.name);
+ }
+ }
+
+ public void stop() {
+ if (stopped.compareAndSet(false, true)) {
+ if (scheduler != null) {
+ this.scheduler.shutdown();
+ }
+ LOG.info("{} stop....", options.name);
+ }
+ }
+
+ private static ScheduledExecutorService newDaemonSingleThreadScheduledExecutor(String threadName) {
+ ThreadFactory threadFactory = new ThreadFactoryBuilder().setDaemon(true).setNameFormat(threadName).build();
+ ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1, threadFactory);
+ // By default, a cancelled task is not automatically removed from the work queue until its delay
+ // elapses. We have to enable it manually.
+ executor.setRemoveOnCancelPolicy(true);
+ return executor;
+ }
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/utils/SingleValueMap.java b/groot-core/src/main/java/com/geedgenetworks/core/utils/SingleValueMap.java
new file mode 100644
index 0000000..f6f73c3
--- /dev/null
+++ b/groot-core/src/main/java/com/geedgenetworks/core/utils/SingleValueMap.java
@@ -0,0 +1,125 @@
+package com.geedgenetworks.core.utils;
+
+import org.apache.flink.util.function.SupplierWithException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.function.Consumer;
+
+/**
+ * 主要用于实现全局对象
+ * 主要用于flink算子中,方便复用全局对象
+ * flink算子中使用方法:
+ * // open方法中根据传入的获取data函数获取data
+ * data = SingleValueMap.acquireData("key", () -> getDataFunc());
+ * // process方法中使用data
+ * data.getData()
+ * // close方法中释放data
+ * if(data != null)
+ * data.release();
+ */
+public class SingleValueMap {
+ static final Logger LOG = LoggerFactory.getLogger(SingleValueMap.class);
+ private static Map<Object, Data<?>> cache = new LinkedHashMap<>();
+
+ public static synchronized <T> Data<T> acquireData(Object key, SupplierWithException<T, Exception> dataSupplier) throws Exception {
+ return acquireData(key, dataSupplier, x -> {});
+ }
+
+ public static synchronized <T> Data<T> acquireData(Object key, SupplierWithException<T, Exception> dataSupplier, Consumer<T> releaseFunc) throws Exception {
+ assert releaseFunc != null;
+ Data<?> existingData = cache.get(key);
+ Data<T> data;
+ if (existingData == null) {
+ Data<T> newData = new Data<>(key, dataSupplier.get(), releaseFunc);
+ cache.put(key, newData);
+ data = newData;
+ } else {
+ data = (Data<T>) existingData;
+ }
+ data.useCnt += 1;
+
+ LOG.info("acquireData: {}", data);
+
+ return data;
+ }
+
+ private static synchronized <T> void releaseData(Data<T> data) {
+ Data<?> cachedData = cache.get(data.key);
+ if (cachedData == null) {
+ LOG.error("can not get data: {}", data);
+ return;
+ }
+
+ assert data == cachedData;
+ LOG.info("releaseData: {}", data);
+
+ data.useCnt -= 1;
+ if (!data.inUse()) {
+ data.destroy();
+ cache.remove(data.key);
+
+ LOG.info("removeData: {}", data);
+ }
+ }
+
+ public static synchronized void clear() {
+ Iterator<Map.Entry<Object, Data<?>>> iter = cache.entrySet().iterator();
+ while (iter.hasNext()) {
+ Data<?> data = iter.next().getValue();
+ data.destroy();
+ iter.remove();
+ }
+ }
+
+ public final static class Data<T> {
+ final Object key;
+ final T data;
+ final Consumer<T> destroyFunc;
+ volatile int useCnt = 0;
+
+ Data(Object key, T data, Consumer<T> destroyFunc) {
+ this.key = key;
+ this.data = data;
+ this.destroyFunc = destroyFunc;
+ }
+
+ boolean inUse() {
+ return useCnt > 0;
+ }
+
+ void destroy() {
+ if (destroyFunc != null) {
+ try {
+ destroyFunc.accept(data);
+ } catch (Exception e) {
+ LOG.error("error when destroy data: {}", data);
+ }
+ }
+ }
+
+ public void release() {
+ releaseData(this);
+ }
+
+ public Object getKey() {
+ return key;
+ }
+
+ public T getData() {
+ return data;
+ }
+
+ @Override
+ public String toString() {
+ return "Data{" +
+ "key=" + key +
+ ", data=" + data +
+ ", useCnt=" + useCnt +
+ '}';
+ }
+ }
+}
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/types/TypesTest.java b/groot-core/src/test/java/com/geedgenetworks/core/types/TypesTest.java
index ca8d4e5..518a3f4 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/types/TypesTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/types/TypesTest.java
@@ -1,8 +1,11 @@
package com.geedgenetworks.core.types;
+import com.alibaba.fastjson2.JSON;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
@@ -29,6 +32,42 @@ public class TypesTest {
}
@Test
+ void test() {
+ Map<String, Object> map = new LinkedHashMap<>();
+ map.put("a", 1);
+ map.put("b", "aa");
+ map.put("c", List.of(1, 2, 3));
+ map.put("int_array", new int[]{1, 2, 3});
+ map.put("str_array", new String[]{"1", "2", "3"});
+ map.put("obj_array", new Object[]{"1", "2", "3"});
+ String jsonString = JSON.toJSONString(map);
+ System.out.println(jsonString);
+ }
+
+ @Test
+ void test2() {
+ Object obj = new int[]{1, 2, 3};
+ System.out.println(obj instanceof byte[]);
+ System.out.println(obj instanceof int[]);
+ System.out.println(obj instanceof String[]);
+ System.out.println(obj instanceof Object[]);
+ System.out.println();
+
+ obj = new String[]{"1", "2", "3"};
+ System.out.println(obj instanceof byte[]);
+ System.out.println(obj instanceof int[]);
+ System.out.println(obj instanceof String[]);
+ System.out.println(obj instanceof Object[]);
+ System.out.println();
+
+ obj = new Object[]{"1", "2", "3"};
+ System.out.println(obj instanceof byte[]);
+ System.out.println(obj instanceof int[]);
+ System.out.println(obj instanceof String[]);
+ System.out.println(obj instanceof Object[]);
+ }
+
+ @Test
void testParserBaseType() {
assertEquals(new IntegerType(), Types.parseDataType("INT"));
assertEquals(new LongType(), Types.parseDataType("biGint"));
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/AnonymityLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/AnonymityLookupTest.java
index a52deb1..ae74fea 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/AnonymityLookupTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/AnonymityLookupTest.java
@@ -37,8 +37,8 @@ class AnonymityLookupTest {
parameters.put("kb_name", kbName);
parameters.put("option", "IP_TO_NODE_TYPE");
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("server_ip"));
- udfContext.setOutput_fields(Collections.singletonList("server_node_type"));
+ udfContext.setLookupFields(Collections.singletonList("server_ip"));
+ udfContext.setOutputFields(Collections.singletonList("server_node_type"));
anonymityLookup.open(runtimeContext, udfContext);
Event event = new Event();
@@ -56,8 +56,8 @@ class AnonymityLookupTest {
parameters.put("kb_name", kbName);
parameters.put("option", "DOMAIN_TO_NODE_TYPE");
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("domain"));
- udfContext.setOutput_fields(Collections.singletonList("domain_node_type"));
+ udfContext.setLookupFields(Collections.singletonList("domain"));
+ udfContext.setOutputFields(Collections.singletonList("domain_node_type"));
anonymityLookup.open(runtimeContext, udfContext);
Event event = new Event();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/AppCategoryLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/AppCategoryLookupTest.java
index f74ce39..713be3f 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/AppCategoryLookupTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/AppCategoryLookupTest.java
@@ -30,7 +30,7 @@ class AppCategoryLookupTest {
fieldMapping.put("COMPANY_CATEGORY", "app_company_category");
parameters.put("field_mapping", fieldMapping);
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("app"));
+ udfContext.setLookupFields(Collections.singletonList("app"));
RuntimeContext runtimeContext = mockRuntimeContext();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/BaseStationLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/BaseStationLookupTest.java
new file mode 100644
index 0000000..43b0bd5
--- /dev/null
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/BaseStationLookupTest.java
@@ -0,0 +1,69 @@
+package com.geedgenetworks.core.udf.cn;
+
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.udf.UDFContext;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import static com.geedgenetworks.core.udf.cn.LookupTestUtils.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * @author gujinkai
+ * @version 1.0
+ * @date 2024/9/2 15:20
+ */
+class BaseStationLookupTest {
+
+ private static BaseStationLookup baseStationLookup;
+
+ @BeforeAll
+ static void setUp() {
+ /**
+ * Create a UDFContext object and set the parameters
+ * e.g.:
+ * - function: BASE_STATION_LOOKUP
+ * lookup_fields: [ cell_id ]
+ * output_fields: [ subscriber_longitude,subscriber_latitude ]
+ * parameters:
+ * kb_name: base_station_location
+ */
+ UDFContext udfContext = new UDFContext();
+ Map<String, Object> parameters = new HashMap<>();
+ parameters.put("kb_name", kbName);
+ udfContext.setParameters(parameters);
+ udfContext.setLookupFields(Arrays.asList("cell_id"));
+ udfContext.setOutputFields(Arrays.asList("subscriber_longitude", "subscriber_latitude"));
+
+ RuntimeContext runtimeContext = mockRuntimeContext();
+
+ String content = "cell_id,longitude,latitude\n460-11-630947-1,93.9290001,42.66884";
+ mockKnowledgeBaseHandler(content);
+
+ baseStationLookup = new BaseStationLookup();
+ baseStationLookup.open(runtimeContext, udfContext);
+ }
+
+ @Test
+ void evaluate() {
+ Event event = new Event();
+ Map<String, Object> fields = new HashMap<>();
+ fields.put("cell_id", "460-11-630947-1");
+ fields.put("cell_type", 1L);
+ event.setExtractedFields(fields);
+ Event evaluate = baseStationLookup.evaluate(event);
+ assertEquals(93.9290001, evaluate.getExtractedFields().get("subscriber_longitude"));
+ assertEquals(42.66884, evaluate.getExtractedFields().get("subscriber_latitude"));
+ }
+
+ @AfterEach
+ void afterAll() {
+ clearState();
+ }
+}
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/DnsServerInfoLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/DnsServerInfoLookupTest.java
index 7f526d5..d1cca09 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/DnsServerInfoLookupTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/DnsServerInfoLookupTest.java
@@ -25,8 +25,8 @@ class DnsServerInfoLookupTest {
Map<String, Object> parameters = new HashMap<>();
parameters.put("kb_name", kbName);
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("server_ip"));
- udfContext.setOutput_fields(Collections.singletonList("server_dns_server"));
+ udfContext.setLookupFields(Collections.singletonList("server_ip"));
+ udfContext.setOutputFields(Collections.singletonList("server_dns_server"));
RuntimeContext runtimeContext = mockRuntimeContext();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/FqdnCategoryLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/FqdnCategoryLookupTest.java
index db15642..0e64982 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/FqdnCategoryLookupTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/FqdnCategoryLookupTest.java
@@ -29,7 +29,7 @@ class FqdnCategoryLookupTest {
fieldMapping.put("REPUTATION_LEVEL", "domain_reputation_level");
parameters.put("field_mapping", fieldMapping);
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("domain"));
+ udfContext.setLookupFields(Collections.singletonList("domain"));
RuntimeContext runtimeContext = mockRuntimeContext();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/FqdnWhoisLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/FqdnWhoisLookupTest.java
index 42a98dc..93ee663 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/FqdnWhoisLookupTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/FqdnWhoisLookupTest.java
@@ -24,8 +24,8 @@ class FqdnWhoisLookupTest {
Map<String, Object> parameters = new HashMap<>();
parameters.put("kb_name", kbName);
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("domain"));
- udfContext.setOutput_fields(Collections.singletonList("domain_whois_org"));
+ udfContext.setLookupFields(Collections.singletonList("domain"));
+ udfContext.setOutputFields(Collections.singletonList("domain_whois_org"));
RuntimeContext runtimeContext = mockRuntimeContext();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/H3CellLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/H3CellLookupTest.java
new file mode 100644
index 0000000..a7b98ab
--- /dev/null
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/H3CellLookupTest.java
@@ -0,0 +1,65 @@
+package com.geedgenetworks.core.udf.cn;
+
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.udf.UDFContext;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * @author gujinkai
+ * @version 1.0
+ * @date 2024/9/6 16:15
+ */
+public class H3CellLookupTest {
+
+
+ private static H3CellLookup h3CellLookup;
+
+ @BeforeAll
+ static void setUp() {
+ /**
+ * Create a UDFContext object and set the parameters
+ * e.g.:
+ * - function: BASE_STATION_LOOKUP
+ * lookup_fields: [ subscriber_longitude,subscriber_latitude ]
+ * output_fields: [ first_location ]
+ * parameters:
+ * resolution: 9
+ */
+ UDFContext udfContext = new UDFContext();
+ udfContext.setLookupFields(Arrays.asList("subscriber_longitude", "subscriber_latitude"));
+ udfContext.setOutputFields(Arrays.asList("first_location"));
+ Map<String, Object> parameters = new HashMap<>();
+ parameters.put("resolution", 9);
+ udfContext.setParameters(parameters);
+
+ RuntimeContext runtimeContext = Mockito.mock(RuntimeContext.class);
+
+ /*String content = "cell_id,longitude,latitude\n460-11-630947-1,93.9290001,42.66884";
+ mockKnowledgeBaseHandler(content);*/
+
+ h3CellLookup = new H3CellLookup();
+ h3CellLookup.open(runtimeContext, udfContext);
+ }
+
+ @Test
+ void testValueExpression() throws IOException {
+ Event event = new Event();
+ Map<String, Object> fields = new HashMap<>();
+ fields.put("subscriber_longitude", 116.390249);
+ fields.put("subscriber_latitude", 39.905392);
+ event.setExtractedFields(fields);
+ Event evaluate = h3CellLookup.evaluate(event);
+ assertEquals("8931aa42853ffff", evaluate.getExtractedFields().get("first_location"));
+
+ }
+}
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IcpLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IcpLookupTest.java
index 3158124..c2032e0 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IcpLookupTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IcpLookupTest.java
@@ -24,8 +24,8 @@ class IcpLookupTest {
Map<String, Object> parameters = new HashMap<>();
parameters.put("kb_name", kbName);
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("domain"));
- udfContext.setOutput_fields(Collections.singletonList("domain_icp_company_name"));
+ udfContext.setLookupFields(Collections.singletonList("domain"));
+ udfContext.setOutputFields(Collections.singletonList("domain_icp_company_name"));
RuntimeContext runtimeContext = mockRuntimeContext();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IdcRenterLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IdcRenterLookupTest.java
index b15096b..7409a2f 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IdcRenterLookupTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IdcRenterLookupTest.java
@@ -24,8 +24,8 @@ class IdcRenterLookupTest {
Map<String, Object> parameters = new HashMap<>();
parameters.put("kb_name", kbName);
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("server_ip"));
- udfContext.setOutput_fields(Collections.singletonList("server_idc_renter"));
+ udfContext.setLookupFields(Collections.singletonList("server_ip"));
+ udfContext.setOutputFields(Collections.singletonList("server_idc_renter"));
RuntimeContext runtimeContext = mockRuntimeContext();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IntelligenceIndicatorLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IntelligenceIndicatorLookupTest.java
index 34fef6b..7d643f4 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IntelligenceIndicatorLookupTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IntelligenceIndicatorLookupTest.java
@@ -27,7 +27,7 @@ public class IntelligenceIndicatorLookupTest {
void setUp() {
runtimeContext = mockRuntimeContext();
- String content = "type,ip_addr_format,ip1,ip2,domain,tags\nIP,CIDR,116.178.65.0,25,ali.com,\"阿里1,云服务1\"\nDomain,CIDR,116.178.65.0,25,$ali.com,\"阿里2,云服务2\"\nDomain,CIDR,116.178.65.0,25,*baidu.com,\"阿里3,云服务3\"\nIP,Single,116.178.65.64,116.178.65.64,ali.com,\"test\"";
+ String content = "type,ip_addr_format,ip1,ip2,domain,subscriber,tags\nIP,CIDR,116.178.65.0,25,ali.com,,\"阿里1,云服务1\"\nDomain,CIDR,116.178.65.0,25,$ali.com,,\"阿里2,云服务2\"\nDomain,CIDR,116.178.65.0,25,*baidu.com,,\"阿里3,云服务3\"\nIP,Single,116.178.65.64,116.178.65.64,ali.com,,\"test\"\nSubscriber,Single,116.178.65.64,116.178.65.64,ali.com,1234567,\"test_subscriber\"";
mockKnowledgeBaseHandler(content);
intelligenceIndicatorLookup = new IntelligenceIndicatorLookup();
@@ -40,8 +40,8 @@ public class IntelligenceIndicatorLookupTest {
parameters.put("kb_name", kbName);
parameters.put("option", "IP_TO_TAG");
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("server_ip"));
- udfContext.setOutput_fields(Collections.singletonList("server_ip_tags"));
+ udfContext.setLookupFields(Collections.singletonList("server_ip"));
+ udfContext.setOutputFields(Collections.singletonList("server_ip_tags"));
intelligenceIndicatorLookup.open(runtimeContext, udfContext);
Event event = new Event();
@@ -59,8 +59,8 @@ public class IntelligenceIndicatorLookupTest {
parameters.put("kb_name", kbName);
parameters.put("option", "IP_TO_TAG");
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("server_ip"));
- udfContext.setOutput_fields(Collections.singletonList("server_ip_tags"));
+ udfContext.setLookupFields(Collections.singletonList("server_ip"));
+ udfContext.setOutputFields(Collections.singletonList("server_ip_tags"));
intelligenceIndicatorLookup.open(runtimeContext, udfContext);
Event event = new Event();
@@ -82,8 +82,8 @@ public class IntelligenceIndicatorLookupTest {
parameters.put("kb_name", kbName);
parameters.put("option", "DOMAIN_TO_TAG");
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("domain"));
- udfContext.setOutput_fields(Collections.singletonList("domain_tags"));
+ udfContext.setLookupFields(Collections.singletonList("domain"));
+ udfContext.setOutputFields(Collections.singletonList("domain_tags"));
intelligenceIndicatorLookup.open(runtimeContext, udfContext);
Event event = new Event();
@@ -101,8 +101,8 @@ public class IntelligenceIndicatorLookupTest {
parameters.put("kb_name", kbName);
parameters.put("option", "DOMAIN_TO_TAG");
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("domain"));
- udfContext.setOutput_fields(Collections.singletonList("domain_tags"));
+ udfContext.setLookupFields(Collections.singletonList("domain"));
+ udfContext.setOutputFields(Collections.singletonList("domain_tags"));
intelligenceIndicatorLookup.open(runtimeContext, udfContext);
Event event = new Event();
@@ -124,8 +124,8 @@ public class IntelligenceIndicatorLookupTest {
parameters.put("kb_name", kbName);
parameters.put("option", "DOMAIN_TO_TAG");
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("domain"));
- udfContext.setOutput_fields(Collections.singletonList("domain_tags"));
+ udfContext.setLookupFields(Collections.singletonList("domain"));
+ udfContext.setOutputFields(Collections.singletonList("domain_tags"));
intelligenceIndicatorLookup.open(runtimeContext, udfContext);
Event event = new Event();
@@ -143,8 +143,8 @@ public class IntelligenceIndicatorLookupTest {
parameters.put("kb_name", kbName);
parameters.put("option", "IP_TO_TAG");
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("server_ip"));
- udfContext.setOutput_fields(Collections.singletonList("server_ip_tags"));
+ udfContext.setLookupFields(Collections.singletonList("server_ip"));
+ udfContext.setOutputFields(Collections.singletonList("server_ip_tags"));
intelligenceIndicatorLookup.open(runtimeContext, udfContext);
Event event = new Event();
@@ -155,6 +155,25 @@ public class IntelligenceIndicatorLookupTest {
assertEquals(Arrays.asList("阿里1", "云服务1", "test"), evaluate1.getExtractedFields().get("server_ip_tags"));
}
+ @Test
+ void evaluate7() {
+ UDFContext udfContext = new UDFContext();
+ Map<String, Object> parameters = new HashMap<>();
+ parameters.put("kb_name", kbName);
+ parameters.put("option", "SUBSCRIBER_TO_TAG");
+ udfContext.setParameters(parameters);
+ udfContext.setLookupFields(Collections.singletonList("subscriber_id"));
+ udfContext.setOutputFields(Collections.singletonList("subscriber_tags"));
+ intelligenceIndicatorLookup.open(runtimeContext, udfContext);
+
+ Event event = new Event();
+ Map<String, Object> fields = new HashMap<>();
+ fields.put("subscriber_id", "1234567");
+ event.setExtractedFields(fields);
+ Event evaluate1 = intelligenceIndicatorLookup.evaluate(event);
+ assertEquals(Arrays.asList("test_subscriber"), evaluate1.getExtractedFields().get("subscriber_tags"));
+ }
+
@AfterEach
void afterAll() {
clearState();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IocLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IocLookupTest.java
index f9d3b25..8c01bc7 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IocLookupTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IocLookupTest.java
@@ -37,8 +37,8 @@ class IocLookupTest {
parameters.put("kb_name", kbName);
parameters.put("option", "IP_TO_MALWARE");
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("server_ip"));
- udfContext.setOutput_fields(Collections.singletonList("server_malware"));
+ udfContext.setLookupFields(Collections.singletonList("server_ip"));
+ udfContext.setOutputFields(Collections.singletonList("server_malware"));
iocLookup.open(runtimeContext, udfContext);
Event event = new Event();
@@ -56,8 +56,8 @@ class IocLookupTest {
parameters.put("kb_name", kbName);
parameters.put("option", "DOMAIN_TO_MALWARE");
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("domain"));
- udfContext.setOutput_fields(Collections.singletonList("domain_malware"));
+ udfContext.setLookupFields(Collections.singletonList("domain"));
+ udfContext.setOutputFields(Collections.singletonList("domain_malware"));
iocLookup.open(runtimeContext, udfContext);
Event event = new Event();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IpZoneLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IpZoneLookupTest.java
index abe5ba0..e3024b5 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IpZoneLookupTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IpZoneLookupTest.java
@@ -24,8 +24,8 @@ class IpZoneLookupTest {
Map<String, Object> parameters = new HashMap<>();
parameters.put("kb_name", kbName);
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("client_ip"));
- udfContext.setOutput_fields(Collections.singletonList("client_zone"));
+ udfContext.setLookupFields(Collections.singletonList("client_ip"));
+ udfContext.setOutputFields(Collections.singletonList("client_zone"));
RuntimeContext runtimeContext = mockRuntimeContext();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/LinkDirectionLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/LinkDirectionLookupTest.java
index c0a06fe..4f2f551 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/LinkDirectionLookupTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/LinkDirectionLookupTest.java
@@ -24,8 +24,8 @@ class LinkDirectionLookupTest {
Map<String, Object> parameters = new HashMap<>();
parameters.put("kb_name", kbName);
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("in_link_id"));
- udfContext.setOutput_fields(Collections.singletonList("in_link_direction"));
+ udfContext.setLookupFields(Collections.singletonList("in_link_id"));
+ udfContext.setOutputFields(Collections.singletonList("in_link_direction"));
RuntimeContext runtimeContext = mockRuntimeContext();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/LookupTestUtils.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/LookupTestUtils.java
index 05df41d..bf95c57 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/LookupTestUtils.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/LookupTestUtils.java
@@ -47,6 +47,20 @@ public class LookupTestUtils {
private static MockedStatic<AbstractMultipleKnowledgeBaseHandler> abstractMultipleKnowledgeBaseHandlerMockedStatic = mockStatic(AbstractMultipleKnowledgeBaseHandler.class);
+ /**
+ * mock runtime context
+ * the configuration is set in the global job parameters
+ * the configuration contains the knowledge base configuration
+ * the knowledge base configuration contains the knowledge base name and file path and file type and file list
+ * e.g.:
+ * - name: cn_ip_location
+ * fs_type: http
+ * fs_path: http://192.168.44.55:9999/v1/knowledge_base
+ * files:
+ * - 1
+ *
+ * @return runtime context
+ */
static RuntimeContext mockRuntimeContext() {
RuntimeContext runtimeContext = Mockito.mock(RuntimeContext.class);
ExecutionConfig executionConfig = Mockito.mock(ExecutionConfig.class);
@@ -77,6 +91,12 @@ public class LookupTestUtils {
}
}
+ /**
+ * mock knowledge base handler
+ * the knowledge base handler is used to get the metadata and read the file content
+ *
+ * @param downloadContent download content
+ */
static void mockKnowledgeBaseHandler(String downloadContent) {
checkStaticMock();
KnowLedgeBaseFileMeta knowLedgeBaseFileMeta = new KnowLedgeBaseFileMeta();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/UserDefineTagLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/UserDefineTagLookupTest.java
index 9688199..a586aeb 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/UserDefineTagLookupTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/UserDefineTagLookupTest.java
@@ -37,8 +37,8 @@ class UserDefineTagLookupTest {
parameters.put("kb_name", kbName);
parameters.put("option", "IP_TO_TAG");
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("client_ip"));
- udfContext.setOutput_fields(Collections.singletonList("client_ip_tags"));
+ udfContext.setLookupFields(Collections.singletonList("client_ip"));
+ udfContext.setOutputFields(Collections.singletonList("client_ip_tags"));
userDefineTagLookup.open(runtimeContext, udfContext);
Event event = new Event();
@@ -59,8 +59,8 @@ class UserDefineTagLookupTest {
parameters.put("kb_name", kbName);
parameters.put("option", "DOMAIN_TO_TAG");
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("domain"));
- udfContext.setOutput_fields(Collections.singletonList("domain_tags"));
+ udfContext.setLookupFields(Collections.singletonList("domain"));
+ udfContext.setOutputFields(Collections.singletonList("domain_tags"));
userDefineTagLookup.open(runtimeContext, udfContext);
Event event = new Event();
@@ -81,8 +81,8 @@ class UserDefineTagLookupTest {
parameters.put("kb_name", kbName);
parameters.put("option", "DOMAIN_TO_TAG");
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("domain"));
- udfContext.setOutput_fields(Collections.singletonList("domain_tags"));
+ udfContext.setLookupFields(Collections.singletonList("domain"));
+ udfContext.setOutputFields(Collections.singletonList("domain_tags"));
userDefineTagLookup.open(runtimeContext, udfContext);
Event event = new Event();
@@ -103,8 +103,8 @@ class UserDefineTagLookupTest {
parameters.put("kb_name", kbName);
parameters.put("option", "APP_TO_TAG");
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("app"));
- udfContext.setOutput_fields(Collections.singletonList("app_tags"));
+ udfContext.setLookupFields(Collections.singletonList("app"));
+ udfContext.setOutputFields(Collections.singletonList("app_tags"));
userDefineTagLookup.open(runtimeContext, udfContext);
Event event = new Event();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/VpnLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/VpnLookupTest.java
index 3dd0992..50374f7 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/VpnLookupTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/VpnLookupTest.java
@@ -30,8 +30,8 @@ class VpnLookupTest {
parameters.put("kb_name", kbName);
parameters.put("option", "IP_TO_VPN");
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("server_ip"));
- udfContext.setOutput_fields(Collections.singletonList("server_vpn_service_name"));
+ udfContext.setLookupFields(Collections.singletonList("server_ip"));
+ udfContext.setOutputFields(Collections.singletonList("server_vpn_service_name"));
RuntimeContext runtimeContext = mockRuntimeContext();
@@ -55,8 +55,8 @@ class VpnLookupTest {
parameters.put("kb_name", kbName);
parameters.put("option", "DOMAIN_TO_VPN");
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("domain"));
- udfContext.setOutput_fields(Collections.singletonList("domain_vpn_service_name"));
+ udfContext.setLookupFields(Collections.singletonList("domain"));
+ udfContext.setOutputFields(Collections.singletonList("domain_vpn_service_name"));
RuntimeContext runtimeContext = mockRuntimeContext();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/AsnLookupFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/AsnLookupFunctionTest.java
index 1e5c0a3..f10fe2b 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/AsnLookupFunctionTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/AsnLookupFunctionTest.java
@@ -26,8 +26,8 @@ public class AsnLookupFunctionTest {
udfContext = new UDFContext();
parameters = new HashMap<>();
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("ip"));
- udfContext.setOutput_fields(Collections.singletonList("asn"));
+ udfContext.setLookupFields(Collections.singletonList("ip"));
+ udfContext.setOutputFields(Collections.singletonList("asn"));
}
@@ -36,7 +36,7 @@ public class AsnLookupFunctionTest {
@Test
public void testInit(){
AsnLookup asnLookup = new AsnLookup();
- udfContext.setLookup_fields(new ArrayList<>());
+ udfContext.setLookupFields(new ArrayList<>());
udfContext.setParameters(new HashMap<>());
udfContext.setParameters(null);
Assertions.assertThrows(GrootStreamRuntimeException.class, () -> {
@@ -48,10 +48,10 @@ public class AsnLookupFunctionTest {
asnLookup.open(null, udfContext);
});
- udfContext.setLookup_fields(new ArrayList<>());
- udfContext.getLookup_fields().add("v1");
- udfContext.setOutput_fields(new ArrayList<>());
- udfContext.getOutput_fields().add("v2");
+ udfContext.setLookupFields(new ArrayList<>());
+ udfContext.getLookupFields().add("v1");
+ udfContext.setOutputFields(new ArrayList<>());
+ udfContext.getOutputFields().add("v2");
udfContext.setParameters(new HashMap<>());
udfContext.getParameters().put("option","other");
Assertions.assertThrows(GrootStreamRuntimeException.class, () -> {
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/GeoIpLookupFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/GeoIpLookupFunctionTest.java
index bc67f1a..83f8ab4 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/GeoIpLookupFunctionTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/GeoIpLookupFunctionTest.java
@@ -25,14 +25,14 @@ public class GeoIpLookupFunctionTest {
@BeforeAll
public static void setUp() {
udfContext = new UDFContext();
- udfContext.setLookup_fields(Collections.singletonList("ip"));
- udfContext.setOutput_fields(Collections.singletonList("iplocation"));
+ udfContext.setLookupFields(Collections.singletonList("ip"));
+ udfContext.setOutputFields(Collections.singletonList("iplocation"));
}
@Test
public void testInit(){
GeoIpLookup geoIpLookup = new GeoIpLookup();
- udfContext.setLookup_fields(new ArrayList<>());
+ udfContext.setLookupFields(new ArrayList<>());
udfContext.setParameters(new HashMap<>());
udfContext.setParameters(null);
Assertions.assertThrows(GrootStreamRuntimeException.class, () -> {
@@ -44,10 +44,10 @@ public class GeoIpLookupFunctionTest {
geoIpLookup.open(null, udfContext);
});
- udfContext.setLookup_fields(new ArrayList<>());
- udfContext.getLookup_fields().add("v1");
- udfContext.setOutput_fields(new ArrayList<>());
- udfContext.getOutput_fields().add("v2");
+ udfContext.setLookupFields(new ArrayList<>());
+ udfContext.getLookupFields().add("v1");
+ udfContext.setOutputFields(new ArrayList<>());
+ udfContext.getOutputFields().add("v2");
udfContext.setParameters(new HashMap<>());
udfContext.getParameters().put("option","other");
Assertions.assertThrows(GrootStreamRuntimeException.class, () -> {
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectListTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectListTest.java
index 2bf13a5..16d5cce 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectListTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectListTest.java
@@ -1,18 +1,3 @@
-/**
- * Copyright 2017 Hortonworks.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
-
- * http://www.apache.org/licenses/LICENSE-2.0
-
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.geedgenetworks.core.udf.test.aggregate;
@@ -45,8 +30,8 @@ public class CollectListTest {
private void testMerge(List<String> arr,List<String> arr2) {
UDFContext udfContext = new UDFContext();
- udfContext.setLookup_fields(List.of("field"));
- udfContext.setOutput_fields(Collections.singletonList("field_list"));
+ udfContext.setLookupFields(List.of("field"));
+ udfContext.setOutputFields(Collections.singletonList("field_list"));
CollectList collectList = new CollectList();
Map<String, Object> metricsFields = new HashMap<>();
Accumulator accumulator = new Accumulator();
@@ -83,8 +68,8 @@ public class CollectListTest {
private void testGetResult(List<String> arr) throws ParseException {
UDFContext udfContext = new UDFContext();
- udfContext.setLookup_fields(List.of("field"));
- udfContext.setOutput_fields(Collections.singletonList("field_list"));
+ udfContext.setLookupFields(List.of("field"));
+ udfContext.setOutputFields(Collections.singletonList("field_list"));
CollectList collectList = new CollectList();
Map<String, Object> metricsFields = new HashMap<>();
Accumulator accumulator = new Accumulator();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectSetTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectSetTest.java
index 8e992f6..8909794 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectSetTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectSetTest.java
@@ -1,18 +1,3 @@
-/**
- * Copyright 2017 Hortonworks.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
-
- * http://www.apache.org/licenses/LICENSE-2.0
-
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.geedgenetworks.core.udf.test.aggregate;
@@ -43,8 +28,8 @@ public class CollectSetTest {
private void testMerge(List<String> arr,List<String> arr2) {
UDFContext udfContext = new UDFContext();
- udfContext.setLookup_fields(List.of("field"));
- udfContext.setOutput_fields(Collections.singletonList("field_list"));
+ udfContext.setLookupFields(List.of("field"));
+ udfContext.setOutputFields(Collections.singletonList("field_list"));
CollectSet collectSet = new CollectSet();
Map<String, Object> metricsFields = new HashMap<>();
Accumulator accumulator = new Accumulator();
@@ -80,8 +65,8 @@ public class CollectSetTest {
private static void testGetResult(List<String> arr) throws ParseException {
UDFContext udfContext = new UDFContext();
- udfContext.setLookup_fields(List.of("field"));
- udfContext.setOutput_fields(Collections.singletonList("field_list"));
+ udfContext.setLookupFields(List.of("field"));
+ udfContext.setOutputFields(Collections.singletonList("field_list"));
CollectSet collectSet = new CollectSet();
Map<String, Object> metricsFields = new HashMap<>();
Accumulator accumulator = new Accumulator();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/FirstValueTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/FirstValueTest.java
index 43a9732..0acf1d5 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/FirstValueTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/FirstValueTest.java
@@ -1,18 +1,3 @@
-/**
- * Copyright 2017 Hortonworks.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
-
- * http://www.apache.org/licenses/LICENSE-2.0
-
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.geedgenetworks.core.udf.test.aggregate;
@@ -42,8 +27,8 @@ public class FirstValueTest {
private void testMerge(List<String> arr,List<String> arr2) {
UDFContext udfContext = new UDFContext();
- udfContext.setLookup_fields(List.of("field"));
- udfContext.setOutput_fields(Collections.singletonList("field_first"));
+ udfContext.setLookupFields(List.of("field"));
+ udfContext.setOutputFields(Collections.singletonList("field_first"));
FirstValue firstValue = new FirstValue();
Map<String, Object> metricsFields = new HashMap<>();
Accumulator accumulator = new Accumulator();
@@ -78,8 +63,8 @@ public class FirstValueTest {
private static void testGetResult(List<String> arr) throws ParseException {
UDFContext udfContext = new UDFContext();
- udfContext.setLookup_fields(List.of("field"));
- udfContext.setOutput_fields(Collections.singletonList("field_first"));
+ udfContext.setLookupFields(List.of("field"));
+ udfContext.setOutputFields(Collections.singletonList("field_first"));
FirstValue firstValue = new FirstValue();
Map<String, Object> metricsFields = new HashMap<>();
Accumulator accumulator = new Accumulator();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LastValueTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LastValueTest.java
index e952908..b2c9ceb 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LastValueTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LastValueTest.java
@@ -1,18 +1,3 @@
-/**
- * Copyright 2017 Hortonworks.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
-
- * http://www.apache.org/licenses/LICENSE-2.0
-
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.geedgenetworks.core.udf.test.aggregate;
@@ -45,8 +30,8 @@ public class LastValueTest {
private void testMerge(List<String> arr,List<String> arr2) {
UDFContext udfContext = new UDFContext();
- udfContext.setLookup_fields(List.of("field"));
- udfContext.setOutput_fields(Collections.singletonList("field_last"));
+ udfContext.setLookupFields(List.of("field"));
+ udfContext.setOutputFields(Collections.singletonList("field_last"));
LastValue lastValue = new LastValue();
Map<String, Object> metricsFields = new HashMap<>();
Accumulator accumulator = new Accumulator();
@@ -81,8 +66,8 @@ public class LastValueTest {
private static void testGetResult(List<String> arr) throws ParseException {
UDFContext udfContext = new UDFContext();
- udfContext.setLookup_fields(List.of("field"));
- udfContext.setOutput_fields(Collections.singletonList("field_last"));
+ udfContext.setLookupFields(List.of("field"));
+ udfContext.setOutputFields(Collections.singletonList("field_last"));
LastValue lastValue = new LastValue();
Map<String, Object> metricsFields = new HashMap<>();
Accumulator accumulator = new Accumulator();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LongCountTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LongCountTest.java
index c1dfb9e..54d9dba 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LongCountTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LongCountTest.java
@@ -1,18 +1,3 @@
-/**
- * Copyright 2017 Hortonworks.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
-
- * http://www.apache.org/licenses/LICENSE-2.0
-
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.geedgenetworks.core.udf.test.aggregate;
@@ -46,8 +31,8 @@ public class LongCountTest {
private void testMerge(Number[] arr,Number[] arr2) {
UDFContext udfContext = new UDFContext();
- udfContext.setLookup_fields(List.of("field"));
- udfContext.setOutput_fields(Collections.singletonList("count"));
+ udfContext.setLookupFields(List.of("field"));
+ udfContext.setOutputFields(Collections.singletonList("count"));
LongCount longCount = new LongCount();
Map<String, Object> metricsFields = new HashMap<>();
Accumulator accumulator = new Accumulator();
@@ -81,7 +66,7 @@ public class LongCountTest {
private static void testGetResult(Number[] arr) throws ParseException {
UDFContext udfContext = new UDFContext();
- udfContext.setOutput_fields(Collections.singletonList("count"));
+ udfContext.setOutputFields(Collections.singletonList("count"));
LongCount longCount = new LongCount();
Map<String, Object> metricsFields = new HashMap<>();
Accumulator accumulator = new Accumulator();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MaxTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MaxTest.java
new file mode 100644
index 0000000..311d51f
--- /dev/null
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MaxTest.java
@@ -0,0 +1,144 @@
+package com.geedgenetworks.core.udf.test.aggregate;
+
+
+import com.geedgenetworks.common.Accumulator;
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.udf.UDFContext;
+import com.geedgenetworks.core.udf.udaf.Max;
+import com.geedgenetworks.core.udf.udaf.Min;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.time.LocalDateTime;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+
+public class MaxTest {
+
+ private Max maxFunction;
+ private Accumulator acc;
+ private Event event;
+
+ @BeforeEach
+ void setUp() {
+ maxFunction = new Max();
+ acc = new Accumulator();
+ event = new Event();
+
+ // 初始化上下文
+ UDFContext udfContext = new UDFContext();
+ udfContext.setLookupFields(List.of("value"));
+ udfContext.setOutputFields(List.of("maxValue"));
+ maxFunction.open(udfContext);
+
+ // 初始化累加器的 metricsFields
+ acc.setMetricsFields(new HashMap<>());
+ }
+
+ @Test
+ void testAddMultipleNumericValues() {
+ // 设置事件中的多个数值
+ Map<String, Object> extractedFields = new HashMap<>();
+ extractedFields.put("value", 100);
+ event.setExtractedFields(extractedFields);
+ Accumulator result = maxFunction.add(event, acc);
+
+ // 添加第二个数值
+ extractedFields.put("value", 200);
+ result = maxFunction.add(event, result);
+
+ // 添加第三个数值
+ extractedFields.put("value", 150);
+ result = maxFunction.add(event, result);
+
+ // 验证最大值应该是200
+ assertEquals(200, result.getMetricsFields().get("maxValue"));
+ }
+
+ @Test
+ void testAddMultipleStringValues() {
+ // 设置事件中的多个字符串值
+ Map<String, Object> extractedFields = new HashMap<>();
+ extractedFields.put("value", "abc");
+ event.setExtractedFields(extractedFields);
+ Accumulator result = maxFunction.add(event, acc);
+
+ // 添加第二个字符串值
+ extractedFields.put("value", "def");
+ result = maxFunction.add(event, result);
+
+ // 添加第三个字符串值
+ extractedFields.put("value", "ghi");
+ result = maxFunction.add(event, result);
+
+ // 验证最大字符串值(按字母顺序最大的是 "ghi")
+ assertEquals("ghi", result.getMetricsFields().get("maxValue"));
+ }
+
+ @Test
+ void testAddMultipleLocalDateTimes() {
+ // 设置事件中的多个时间值
+ Map<String, Object> extractedFields = new HashMap<>();
+
+ extractedFields.put("value", LocalDateTime.of(2024, 10, 22, 12, 0));
+ event.setExtractedFields(extractedFields);
+ Accumulator result = maxFunction.add(event, acc);
+
+ // 添加第二个时间值
+ extractedFields.put("value", LocalDateTime.of(2024, 10, 23, 12, 0));
+ result = maxFunction.add(event, result);
+
+ // 添加第三个时间值
+ extractedFields.put("value", LocalDateTime.of(2024, 10, 21, 12, 0));
+ result = maxFunction.add(event, result);
+
+ // 验证最大时间值(最大的是2024年10月23日12点)
+ assertEquals(LocalDateTime.of(2024, 10, 23, 12, 0), result.getMetricsFields().get("maxValue"));
+ }
+
+ @Test
+ void testMergeAccumulatorsForNumber() {
+ Accumulator acc1 = new Accumulator();
+ Accumulator acc2 = new Accumulator();
+ Map map1 = new HashMap();
+ map1.put("maxValue", 10L);
+ Map map2 = new HashMap();
+ map2.put("maxValue", 5.0);
+ acc1.setMetricsFields(map1);
+ acc2.setMetricsFields(map2);
+ Accumulator mergedAcc = maxFunction.merge(acc1, acc2);
+ assertEquals(10L, mergedAcc.getMetricsFields().get("maxValue"));
+ }
+
+ @Test
+ void testMergeAccumulatorsForTime() {
+ Accumulator acc1 = new Accumulator();
+ Accumulator acc2 = new Accumulator();
+ Map map1 = new HashMap();
+ map1.put("maxValue", LocalDateTime.of(2023, 1, 1, 0, 0,1));
+ Map map2 = new HashMap();
+ map2.put("maxValue", LocalDateTime.of(2023, 1, 1, 0, 0,2));
+ acc1.setMetricsFields(map1);
+ acc2.setMetricsFields(map2);
+ Accumulator mergedAcc = maxFunction.merge(acc1, acc2);
+ assertEquals(LocalDateTime.of(2023, 1, 1, 0, 0,2), mergedAcc.getMetricsFields().get("maxValue"));
+ }
+
+ @Test
+ void testMergeAccumulatorsForString() {
+ Accumulator acc1 = new Accumulator();
+ Accumulator acc2 = new Accumulator();
+ Map map1 = new HashMap();
+ map1.put("maxValue", "qwe");
+ Map map2 = new HashMap();
+ map2.put("maxValue", "abc");
+ acc1.setMetricsFields(map1);
+ acc2.setMetricsFields(map2);
+ Accumulator mergedAcc = maxFunction.merge(acc1, acc2);
+ assertEquals("qwe", mergedAcc.getMetricsFields().get("maxValue"));
+ }
+} \ No newline at end of file
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MeanTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MeanTest.java
index cc4eaf0..62efc0a 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MeanTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MeanTest.java
@@ -1,18 +1,3 @@
-/**
- * Copyright 2017 Hortonworks.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
-
- * http://www.apache.org/licenses/LICENSE-2.0
-
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.geedgenetworks.core.udf.test.aggregate;
@@ -46,8 +31,8 @@ public class MeanTest {
}
private void testMerge(Number[] arr1,Number[] arr2,int precision) throws ParseException {
UDFContext udfContext = new UDFContext();
- udfContext.setLookup_fields(List.of("field"));
- udfContext.setOutput_fields(Collections.singletonList("field_mean"));
+ udfContext.setLookupFields(List.of("field"));
+ udfContext.setOutputFields(Collections.singletonList("field_mean"));
udfContext.setParameters(new HashMap<>());
udfContext.getParameters().put("precision", precision);
Mean mean = new Mean();
@@ -62,8 +47,8 @@ public class MeanTest {
}
private Accumulator getMiddleResult(Number[] arr,int precision) throws ParseException {
UDFContext udfContext = new UDFContext();
- udfContext.setLookup_fields(List.of("field"));
- udfContext.setOutput_fields(Collections.singletonList("field_mean"));
+ udfContext.setLookupFields(List.of("field"));
+ udfContext.setOutputFields(Collections.singletonList("field_mean"));
udfContext.setParameters(new HashMap<>());
udfContext.getParameters().put("precision", precision);
Mean mean = new Mean();
@@ -87,8 +72,8 @@ public class MeanTest {
private void testInt(Number[] arr,int precision) throws ParseException {
UDFContext udfContext = new UDFContext();
- udfContext.setLookup_fields(List.of("field"));
- udfContext.setOutput_fields(Collections.singletonList("field_mean"));
+ udfContext.setLookupFields(List.of("field"));
+ udfContext.setOutputFields(Collections.singletonList("field_mean"));
udfContext.setParameters(new HashMap<>());
udfContext.getParameters().put("precision", precision);
Mean mean = new Mean();
@@ -113,8 +98,8 @@ public class MeanTest {
private void testDouble(Number[] arr,int precision) throws ParseException {
UDFContext udfContext = new UDFContext();
- udfContext.setLookup_fields(List.of("field"));
- udfContext.setOutput_fields(Collections.singletonList("field_mean"));
+ udfContext.setLookupFields(List.of("field"));
+ udfContext.setOutputFields(Collections.singletonList("field_mean"));
udfContext.setParameters(new HashMap<>());
udfContext.getParameters().put("precision", precision);
Mean mean = new Mean();
@@ -138,8 +123,8 @@ public class MeanTest {
private void testNoPrecision(Number[] arr) throws ParseException {
UDFContext udfContext = new UDFContext();
- udfContext.setLookup_fields(List.of("field"));
- udfContext.setOutput_fields(Collections.singletonList("field_mean"));
+ udfContext.setLookupFields(List.of("field"));
+ udfContext.setOutputFields(Collections.singletonList("field_mean"));
Mean mean = new Mean();
Map<String, Object> metricsFields = new HashMap<>();
Accumulator accumulator = new Accumulator();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MinTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MinTest.java
new file mode 100644
index 0000000..e5a1615
--- /dev/null
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MinTest.java
@@ -0,0 +1,123 @@
+package com.geedgenetworks.core.udf.test.aggregate;
+
+
+import com.geedgenetworks.common.Accumulator;
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.udf.UDFContext;
+import com.geedgenetworks.core.udf.udaf.Min;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.time.LocalDateTime;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class MinTest {
+
+ private Min minFunction;
+ private UDFContext udfContext;
+ private Accumulator acc;
+ private Event event1, event2, event3;
+
+ @BeforeEach
+ void setUp() {
+ minFunction = new Min();
+ udfContext = new UDFContext();
+ udfContext.setLookupFields(List.of("value"));
+ udfContext.setOutputFields(List.of("minValue"));
+ minFunction.open(udfContext);
+
+ acc = new Accumulator();
+ acc.setMetricsFields(new HashMap<>());
+
+ event1 = new Event();
+ event2 = new Event();
+ event3 = new Event();
+ }
+
+ @Test
+ void testAddNumericValue() {
+ event1.setExtractedFields(Map.of("value", 5.0));
+ event2.setExtractedFields(Map.of("value", 3.0));
+ event3.setExtractedFields(Map.of("value", 10.0));
+
+ acc = minFunction.add(event1, acc);
+ acc = minFunction.add(event2, acc);
+ acc = minFunction.add(event3, acc);
+
+ assertEquals(3.0, acc.getMetricsFields().get("minValue"));
+ }
+
+ @Test
+ void testAddStringValue() {
+ event1.setExtractedFields(Map.of("value", "apple"));
+ event2.setExtractedFields(Map.of("value", "banana"));
+ event3.setExtractedFields(Map.of("value", "cherry"));
+
+ acc = minFunction.add(event1, acc);
+ acc = minFunction.add(event2, acc);
+ acc = minFunction.add(event3, acc);
+
+ assertEquals("apple", acc.getMetricsFields().get("minValue"));
+ }
+
+ @Test
+ void testAddLocalDateTime() {
+ event1.setExtractedFields(Map.of("value", LocalDateTime.of(2023, 1, 1, 0, 0)));
+ event2.setExtractedFields(Map.of("value", LocalDateTime.of(2022, 1, 1, 0, 0)));
+ event3.setExtractedFields(Map.of("value", LocalDateTime.of(2024, 1, 1, 0, 0)));
+
+ acc = minFunction.add(event1, acc);
+ acc = minFunction.add(event2, acc);
+ acc = minFunction.add(event3, acc);
+
+ assertEquals(LocalDateTime.of(2022, 1, 1, 0, 0), acc.getMetricsFields().get("minValue"));
+ }
+
+ @Test
+ void testMergeAccumulatorsForNumber() {
+ Accumulator acc1 = new Accumulator();
+ Accumulator acc2 = new Accumulator();
+ Map map1 = new HashMap();
+ map1.put("minValue", 10L);
+ Map map2 = new HashMap();
+ map2.put("minValue", 5.0);
+ acc1.setMetricsFields(map1);
+ acc2.setMetricsFields(map2);
+ Accumulator mergedAcc = minFunction.merge(acc1, acc2);
+ assertEquals(5.0, mergedAcc.getMetricsFields().get("minValue"));
+ }
+
+ @Test
+ void testMergeAccumulatorsForTime() {
+ Accumulator acc1 = new Accumulator();
+ Accumulator acc2 = new Accumulator();
+ Map map1 = new HashMap();
+ map1.put("minValue", LocalDateTime.of(2023, 1, 1, 0, 0,1));
+ Map map2 = new HashMap();
+ map2.put("minValue", LocalDateTime.of(2023, 1, 1, 0, 0,2));
+ acc1.setMetricsFields(map1);
+ acc2.setMetricsFields(map2);
+ Accumulator mergedAcc = minFunction.merge(acc1, acc2);
+ assertEquals(LocalDateTime.of(2023, 1, 1, 0, 0,1), mergedAcc.getMetricsFields().get("minValue"));
+ }
+
+ @Test
+ void testMergeAccumulatorsForString() {
+ Accumulator acc1 = new Accumulator();
+ Accumulator acc2 = new Accumulator();
+ Map map1 = new HashMap();
+ map1.put("minValue", "qwe");
+ Map map2 = new HashMap();
+ map2.put("minValue", "abc");
+ acc1.setMetricsFields(map1);
+ acc2.setMetricsFields(map2);
+ Accumulator mergedAcc = minFunction.merge(acc1, acc2);
+ assertEquals("abc", mergedAcc.getMetricsFields().get("minValue"));
+ }
+
+
+} \ No newline at end of file
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/NumberSumTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/NumberSumTest.java
index a4072ca..7ccb365 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/NumberSumTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/NumberSumTest.java
@@ -48,8 +48,8 @@ public class NumberSumTest {
private void testMerge(Number[] arr,Number[] arr2) {
UDFContext udfContext = new UDFContext();
- udfContext.setLookup_fields(List.of("field"));
- udfContext.setOutput_fields(Collections.singletonList("field_sum"));
+ udfContext.setLookupFields(List.of("field"));
+ udfContext.setOutputFields(Collections.singletonList("field_sum"));
NumberSum numberSum = new NumberSum();
Map<String, Object> metricsFields = new HashMap<>();
Accumulator accumulator = new Accumulator();
@@ -83,8 +83,8 @@ public class NumberSumTest {
private static void excute(Number[] arr, Class<? extends Number> clazz) throws ParseException {
UDFContext udfContext = new UDFContext();
- udfContext.setLookup_fields(List.of("field"));
- udfContext.setOutput_fields(Collections.singletonList("field_sum"));
+ udfContext.setLookupFields(List.of("field"));
+ udfContext.setOutputFields(Collections.singletonList("field_sum"));
NumberSum numberSum = new NumberSum();
Map<String, Object> metricsFields = new HashMap<>();
Accumulator accumulator = new Accumulator();
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DecodeBase64FunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DecodeBase64FunctionTest.java
index 1d3b863..a5f31f7 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DecodeBase64FunctionTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DecodeBase64FunctionTest.java
@@ -22,8 +22,8 @@ public class DecodeBase64FunctionTest {
@BeforeAll
public static void setUp() {
udfContext = new UDFContext();
- udfContext.setLookup_fields(Arrays.asList("message", "charset"));
- udfContext.setOutput_fields(Collections.singletonList("decodeResult"));
+ udfContext.setLookupFields(Arrays.asList("message", "charset"));
+ udfContext.setOutputFields(Collections.singletonList("decodeResult"));
Map<String,Object> map = new HashMap<>();
map.put("value_field","message");
map.put("charset_field","charset");
@@ -51,8 +51,8 @@ public class DecodeBase64FunctionTest {
DecodeBase64 decodeBase64 = new DecodeBase64();
- udfContext.setLookup_fields(Collections.singletonList("message"));
- udfContext.setOutput_fields(Collections.singletonList("decodeResult"));
+ udfContext.setLookupFields(Collections.singletonList("message"));
+ udfContext.setOutputFields(Collections.singletonList("decodeResult"));
udfContext.getParameters().remove("value_field");
assertThrows(GrootStreamRuntimeException.class,
() -> {
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DomainFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DomainFunctionTest.java
index 2126117..f8076cc 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DomainFunctionTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DomainFunctionTest.java
@@ -24,15 +24,15 @@ public class DomainFunctionTest {
udfContext = new UDFContext();
parameters = new HashMap<>();
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("domain"));
- udfContext.setOutput_fields(Collections.singletonList("domain1"));
+ udfContext.setLookupFields(Collections.singletonList("domain"));
+ udfContext.setOutputFields(Collections.singletonList("domain1"));
}
@Test
public void testInit(){
Domain domain = new Domain();
- udfContext.setLookup_fields(new ArrayList<>());
+ udfContext.setLookupFields(new ArrayList<>());
udfContext.setParameters(new HashMap<>());
udfContext.setParameters(null);
Assertions.assertThrows(GrootStreamRuntimeException.class, () -> {
@@ -44,10 +44,10 @@ public class DomainFunctionTest {
domain.open(null, udfContext);
});
- udfContext.setLookup_fields(new ArrayList<>());
- udfContext.getLookup_fields().add("v1");
- udfContext.setOutput_fields(new ArrayList<>());
- udfContext.getOutput_fields().add("v2");
+ udfContext.setLookupFields(new ArrayList<>());
+ udfContext.getLookupFields().add("v1");
+ udfContext.setOutputFields(new ArrayList<>());
+ udfContext.getOutputFields().add("v2");
udfContext.setParameters(new HashMap<>());
udfContext.getParameters().put("option","other");
Assertions.assertThrows(GrootStreamRuntimeException.class, () -> {
@@ -59,16 +59,16 @@ public class DomainFunctionTest {
@Test
public void testDomainFunctionTopLevelDomain() {
parameters.put("option", "TOP_LEVEL_DOMAIN");
- udfContext.setLookup_fields(Collections.singletonList("domain"));
- udfContext.setOutput_fields(Collections.singletonList("domain1"));
+ udfContext.setLookupFields(Collections.singletonList("domain"));
+ udfContext.setOutputFields(Collections.singletonList("domain1"));
Domain domain = new Domain();
domain.open(null, udfContext);
Event event = new Event();
Map<String, Object> extractedFields = new HashMap<>();
- extractedFields.put("domain", "www.baidu.com");
+ extractedFields.put("domain", "http://www.baidu.com.cn");
event.setExtractedFields(extractedFields);
Event result1 = domain.evaluate(event);
- assertEquals("com", result1.getExtractedFields().get("domain1"));
+ assertEquals("com.cn", result1.getExtractedFields().get("domain1"));
}
@Test
@@ -76,8 +76,8 @@ public class DomainFunctionTest {
parameters.put("option", "FIRST_SIGNIFICANT_SUBDOMAIN");
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("domain"));
- udfContext.setOutput_fields(Collections.singletonList("domain1"));
+ udfContext.setLookupFields(Collections.singletonList("domain"));
+ udfContext.setOutputFields(Collections.singletonList("domain1"));
Domain domain = new Domain();
domain.open(null, udfContext);
Event event = new Event();
@@ -87,4 +87,5 @@ public class DomainFunctionTest {
Event result1 = domain.evaluate(event);
assertEquals("baidu.com", result1.getExtractedFields().get("domain1"));
}
+
}
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DropFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DropFunctionTest.java
index 294a492..027533e 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DropFunctionTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DropFunctionTest.java
@@ -16,6 +16,7 @@ public class DropFunctionTest {
@BeforeAll
public static void setUp() {
udfContext = new UDFContext();
+ udfContext.setFilter("true");
udfContext.setParameters(new HashMap<>());
}
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/EncodeBase64FunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/EncodeBase64FunctionTest.java
index 2bc96b6..2bd6705 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/EncodeBase64FunctionTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/EncodeBase64FunctionTest.java
@@ -21,17 +21,15 @@ public class EncodeBase64FunctionTest {
private static UDFContext udfContext;
- @BeforeAll
- public static void setUp() {
- udfContext = new UDFContext();
- udfContext.setOutput_fields(Collections.singletonList("encodeResult"));
- Map<String,Object> map = new HashMap<>();
- map.put("value_field","name");
- udfContext.setParameters(map);
- }
- @Test
- public void testEncodeBase64Function() {
+ @Test
+ public void testEncodeBase64FunctionForByte() {
+ udfContext = new UDFContext();
+ udfContext.setOutputFields(Collections.singletonList("encodeResult"));
+ udfContext.setLookupFields(Collections.singletonList("name"));
+ Map<String,Object> map = new HashMap<>();
+ map.put("input_type","byte_array");
+ udfContext.setParameters(map);
EncodeBase64 encodeBase64 = new EncodeBase64();
encodeBase64.open(null, udfContext);
Event event = new Event();
@@ -40,11 +38,28 @@ public class EncodeBase64FunctionTest {
event.setExtractedFields(extractedFields);
Event result1 = encodeBase64.evaluate(event);
assertEquals("aGVsbG8=", result1.getExtractedFields().get("encodeResult"));
+
+
+
+ }
+
+ @Test
+ public void testEncodeBase64FunctionForString() {
+
+ udfContext = new UDFContext();
+ udfContext.setOutputFields(Collections.singletonList("encodeResult"));
+ udfContext.setLookupFields(Collections.singletonList("name"));
+ Map<String,Object> map = new HashMap<>();
+ map.put("input_type","string");
+ udfContext.setParameters(map);
+ EncodeBase64 encodeBase64 = new EncodeBase64();
+ encodeBase64.open(null, udfContext);
+ Event event = new Event();
+ Map<String, Object> extractedFields = new HashMap<>();
extractedFields.put("name", "hello");
event.setExtractedFields(extractedFields);
- Event result2 = encodeBase64.evaluate(event);
- assertEquals("", result2.getExtractedFields().get("encodeResult"));
+ Event result1 = encodeBase64.evaluate(event);
+ assertEquals("aGVsbG8=", result1.getExtractedFields().get("encodeResult"));
}
-
}
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/EncryptFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/EncryptFunctionTest.java
new file mode 100644
index 0000000..e9f1698
--- /dev/null
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/EncryptFunctionTest.java
@@ -0,0 +1,245 @@
+package com.geedgenetworks.core.udf.test.simple;
+
+import cn.hutool.core.util.RandomUtil;
+import com.alibaba.fastjson2.JSON;
+import com.geedgenetworks.common.Constants;
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.config.CommonConfig;
+import com.geedgenetworks.common.config.KmsConfig;
+import com.geedgenetworks.common.config.SSLConfig;
+import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
+import com.geedgenetworks.common.udf.UDFContext;
+import com.geedgenetworks.core.pojo.KmsKey;
+import com.geedgenetworks.core.udf.Encrypt;
+import com.geedgenetworks.core.udf.encrypt.EncryptionAlgorithm;
+import com.geedgenetworks.core.utils.EncryptionAlgorithmUtils;
+import com.geedgenetworks.core.utils.HttpClientPoolUtil;
+import com.geedgenetworks.core.utils.KmsUtils;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.SimpleCounter;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.mockito.*;
+
+import java.io.IOException;
+import java.security.Security;
+import java.util.*;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+public class EncryptFunctionTest {
+ private static UDFContext udfContext;
+ private static MockedStatic<HttpClientPoolUtil> httpClientPoolUtilMockedStatic;
+ private static final String DATA = "13812345678";
+
+ @BeforeAll
+ public static void setUp() throws IOException {
+ Security.addProvider(new BouncyCastleProvider());
+ udfContext = new UDFContext();
+ udfContext.setLookupFields(Collections.singletonList("phone_number"));
+ udfContext.setOutputFields(Collections.singletonList("phone_number"));
+ httpClientPoolUtilMockedStatic = mockSensitiveFields();
+ }
+
+ @AfterAll
+ public static void after() {
+ httpClientPoolUtilMockedStatic.close();
+ }
+
+ @Test
+ public void testEncryptByVault() throws Exception {
+ String secretKey = RandomUtil.randomString(32);
+ MockedStatic<KmsUtils> kmsUtilsMockedStatic = Mockito.mockStatic(KmsUtils.class);
+ Mockito.when(KmsUtils.getVaultKey(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any())).thenReturn(new KmsKey(secretKey.getBytes(), 1));
+ RuntimeContext runtimeContext = mockVaultRuntimeContext();
+ Map<String, Object> map = new HashMap<>();
+ map.put("identifier", EncryptionAlgorithmUtils.ALGORITHM_AES_256_GCM96_NAME);
+ udfContext.setParameters(map);
+ Encrypt encrypt = new Encrypt();
+ encrypt.open(runtimeContext, udfContext);
+ Event event = new Event();
+ Map<String, Object> extractedFields = new HashMap<>();
+ extractedFields.put("phone_number", DATA);
+ event.setExtractedFields(extractedFields);
+ Event result = encrypt.evaluate(event);
+ EncryptionAlgorithm encryptionAlgorithm = EncryptionAlgorithmUtils.createEncryptionAlgorithm(EncryptionAlgorithmUtils.ALGORITHM_AES_256_GCM96_NAME);
+ assertNotNull(encryptionAlgorithm);
+ encryptionAlgorithm.setKmsKey(new KmsKey(secretKey.getBytes(), 1));
+ String encrypted = result.getExtractedFields().get("phone_number").toString();
+ assertTrue(encrypted.contains("vault:v1:"));
+ String decrypted = encryptionAlgorithm.decrypt(encrypted.split(":")[2]);
+ assertEquals(DATA, decrypted);
+ encrypt.close();
+ kmsUtilsMockedStatic.close();
+ }
+
+ @Test
+ public void testEncryptByLocal() throws Exception {
+ byte[] secretKey = ".........geedgenetworks.........".getBytes();
+ RuntimeContext runtimeContext = mockLocalRuntimeContext();
+ Map<String, Object> map = new HashMap<>();
+ map.put("identifier", EncryptionAlgorithmUtils.ALGORITHM_AES_256_GCM96_NAME);
+ udfContext.setParameters(map);
+ Encrypt encrypt = new Encrypt();
+ encrypt.open(runtimeContext, udfContext);
+ Event event = new Event();
+ Map<String, Object> extractedFields = new HashMap<>();
+ extractedFields.put("phone_number", DATA);
+ event.setExtractedFields(extractedFields);
+ Event result = encrypt.evaluate(event);
+ EncryptionAlgorithm encryptionAlgorithm = EncryptionAlgorithmUtils.createEncryptionAlgorithm(EncryptionAlgorithmUtils.ALGORITHM_AES_256_GCM96_NAME);
+ assertNotNull(encryptionAlgorithm);
+ encryptionAlgorithm.setKmsKey(new KmsKey(secretKey, 1));
+ String decrypted = encryptionAlgorithm.decrypt((String) result.getExtractedFields().get("phone_number"));
+ assertEquals(DATA, decrypted);
+ encrypt.close();
+ }
+
+ @Test
+ public void testEncryptByIdentifier() {
+ Map<String, Object> map = new HashMap<>();
+ map.put("identifier", EncryptionAlgorithmUtils.ALGORITHM_AES_256_GCM96_NAME);
+ udfContext.setParameters(map);
+ Encrypt encrypt1 = new Encrypt();
+ assertDoesNotThrow(() -> encrypt1.open(mockLocalRuntimeContext(), udfContext));
+ encrypt1.close();
+
+ Encrypt encrypt2 = new Encrypt();
+ map.put("identifier", EncryptionAlgorithmUtils.ALGORITHM_AES_128_GCM96_NAME);
+ udfContext.setParameters(map);
+ assertDoesNotThrow(() -> encrypt2.open(mockLocalRuntimeContext(), udfContext));
+ encrypt2.close();
+
+ Encrypt encrypt3 = new Encrypt();
+ map.put("identifier", EncryptionAlgorithmUtils.ALGORITHM_SM4_GCM96_NAME);
+ udfContext.setParameters(map);
+ assertDoesNotThrow(() -> encrypt3.open(mockLocalRuntimeContext(), udfContext));
+ encrypt3.close();
+ }
+
+ @Test
+ public void testEncryptionAlgorithm() throws Exception {
+ EncryptionAlgorithm encryptionAlgorithm = EncryptionAlgorithmUtils.createEncryptionAlgorithm(EncryptionAlgorithmUtils.ALGORITHM_AES_128_GCM96_NAME);
+ assertNotNull(encryptionAlgorithm);
+ encryptionAlgorithm.setKmsKey(new KmsKey("aaaaaaaaaaaaaaaa".getBytes(), 1));
+ String encryptData = encryptionAlgorithm.encrypt(DATA);
+ String decryptData = encryptionAlgorithm.decrypt(encryptData);
+ assertEquals(DATA, decryptData);
+
+ encryptionAlgorithm = EncryptionAlgorithmUtils.createEncryptionAlgorithm(EncryptionAlgorithmUtils.ALGORITHM_AES_256_GCM96_NAME);
+ assertNotNull(encryptionAlgorithm);
+ encryptionAlgorithm.setKmsKey(new KmsKey("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa".getBytes(), 1));
+ encryptData = encryptionAlgorithm.encrypt(DATA);
+ decryptData = encryptionAlgorithm.decrypt(encryptData);
+ assertEquals(DATA, decryptData);
+
+ encryptionAlgorithm = EncryptionAlgorithmUtils.createEncryptionAlgorithm(EncryptionAlgorithmUtils.ALGORITHM_SM4_GCM96_NAME);
+ assertNotNull(encryptionAlgorithm);
+ encryptionAlgorithm.setKmsKey(new KmsKey("aaaaaaaaaaaaaaaa".getBytes(), 1));
+ encryptData = encryptionAlgorithm.encrypt(DATA);
+ decryptData = encryptionAlgorithm.decrypt(encryptData);
+ assertEquals(DATA, decryptData);
+
+ encryptionAlgorithm = EncryptionAlgorithmUtils.createEncryptionAlgorithm("sm4");
+ assertNull(encryptionAlgorithm);
+ }
+
+ @Test
+ public void testEncryptError() {
+ RuntimeContext runtimeContext = mockLocalRuntimeContext();
+ Encrypt encrypt = new Encrypt();
+ udfContext.setParameters(null);
+ assertThrows(GrootStreamRuntimeException.class, () -> encrypt.open(runtimeContext, udfContext));
+
+ Map<String, Object> map = new HashMap<>();
+ udfContext.setParameters(map);
+ assertThrows(GrootStreamRuntimeException.class, () -> encrypt.open(runtimeContext, udfContext));
+
+ map.put("identifier", "aes");
+ udfContext.setParameters(map);
+ assertThrows(GrootStreamRuntimeException.class, () -> encrypt.open(runtimeContext, udfContext));
+ }
+
+ static RuntimeContext mockLocalRuntimeContext() {
+ RuntimeContext runtimeContext = Mockito.mock(RuntimeContext.class);
+ ExecutionConfig executionConfig = Mockito.mock(ExecutionConfig.class);
+ Mockito.when(runtimeContext.getExecutionConfig()).thenReturn(executionConfig);
+ MetricGroup metricGroup = Mockito.mock(OperatorMetricGroup.class);
+ Mockito.when(runtimeContext.getMetricGroup()).thenReturn(metricGroup);
+ Mockito.when(metricGroup.addGroup(Mockito.anyString())).thenReturn(metricGroup);
+ Mockito.when(metricGroup.counter(Mockito.anyString())).thenReturn(new SimpleCounter());
+ Configuration configuration = new Configuration();
+ CommonConfig commonConfig = new CommonConfig();
+ Map<String, KmsConfig> kmsConfigs = new HashMap<>();
+ KmsConfig kmsConfig = new KmsConfig();
+ kmsConfig.setType(KmsUtils.KMS_TYPE_LOCAL);
+ kmsConfigs.put(KmsUtils.KMS_TYPE_LOCAL, kmsConfig);
+ kmsConfig = new KmsConfig();
+ kmsConfig.setType(KmsUtils.KMS_TYPE_VAULT);
+ kmsConfigs.put(KmsUtils.KMS_TYPE_VAULT, kmsConfig);
+ SSLConfig sslConfig = new SSLConfig();
+ sslConfig.setSkipVerification(true);
+ Map<String, String> propertiesConfig = new HashMap<>();
+ propertiesConfig.put("projection.encrypt.schema.registry.uri", "127.0.0.1:9999/v1/schema/session_record?option=encrypt_fields");
+ commonConfig.setKmsConfig(kmsConfigs);
+ commonConfig.setSslConfig(sslConfig);
+ commonConfig.setPropertiesConfig(propertiesConfig);
+ configuration.setString(Constants.SYSPROP_GROOTSTREAM_CONFIG, JSON.toJSONString(commonConfig));
+ configuration.setString(Constants.SYSPROP_KMS_TYPE_CONFIG, KmsUtils.KMS_TYPE_LOCAL);
+ Mockito.when(executionConfig.getGlobalJobParameters()).thenReturn(configuration);
+ return runtimeContext;
+ }
+
+ static RuntimeContext mockVaultRuntimeContext() {
+ RuntimeContext runtimeContext = Mockito.mock(RuntimeContext.class);
+ ExecutionConfig executionConfig = Mockito.mock(ExecutionConfig.class);
+ Mockito.when(runtimeContext.getExecutionConfig()).thenReturn(executionConfig);
+ MetricGroup metricGroup = Mockito.mock(OperatorMetricGroup.class);
+ Mockito.when(runtimeContext.getMetricGroup()).thenReturn(metricGroup);
+ Mockito.when(metricGroup.addGroup(Mockito.anyString())).thenReturn(metricGroup);
+ Mockito.when(metricGroup.counter(Mockito.anyString())).thenReturn(new SimpleCounter());
+ Configuration configuration = new Configuration();
+ CommonConfig commonConfig = new CommonConfig();
+ Map<String, KmsConfig> kmsConfigs = new HashMap<>();
+ KmsConfig kmsConfig = new KmsConfig();
+ kmsConfig.setType(KmsUtils.KMS_TYPE_VAULT);
+ kmsConfigs.put(KmsUtils.KMS_TYPE_VAULT, kmsConfig);
+ kmsConfig = new KmsConfig();
+ kmsConfig.setType(KmsUtils.KMS_TYPE_LOCAL);
+ kmsConfigs.put(KmsUtils.KMS_TYPE_LOCAL, kmsConfig);
+ SSLConfig sslConfig = new SSLConfig();
+ sslConfig.setSkipVerification(true);
+ Map<String, String> propertiesConfig = new HashMap<>();
+ propertiesConfig.put("projection.encrypt.schema.registry.uri", "127.0.0.1:9999/v1/schema/session_record?option=encrypt_fields");
+ commonConfig.setKmsConfig(kmsConfigs);
+ commonConfig.setSslConfig(sslConfig);
+ commonConfig.setPropertiesConfig(propertiesConfig);
+ configuration.setString(Constants.SYSPROP_GROOTSTREAM_CONFIG, JSON.toJSONString(commonConfig));
+ configuration.setString(Constants.SYSPROP_KMS_TYPE_CONFIG, KmsUtils.KMS_TYPE_VAULT);
+ Mockito.when(executionConfig.getGlobalJobParameters()).thenReturn(configuration);
+ return runtimeContext;
+ }
+
+ static MockedStatic<HttpClientPoolUtil> mockSensitiveFields() throws IOException {
+ String sensitiveFieldsStr = "{\n" +
+ " \"status\": 200,\n" +
+ " \"success\": true,\n" +
+ " \"message\": \"Success\",\n" +
+ " \"data\": [\n" +
+ " \"phone_number\",\n" +
+ " \"server_ip\"\n" +
+ " ]\n" +
+ "}";
+ HttpClientPoolUtil instance = Mockito.mock(HttpClientPoolUtil.class);
+ Mockito.when(instance.httpGet(ArgumentMatchers.any(), ArgumentMatchers.any())).thenReturn(sensitiveFieldsStr);
+ MockedStatic<HttpClientPoolUtil> httpClientPoolUtilMockedStatic = Mockito.mockStatic(HttpClientPoolUtil.class);
+ Mockito.when(HttpClientPoolUtil.getInstance()).thenReturn(instance);
+ return httpClientPoolUtilMockedStatic;
+ }
+}
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/FlattenFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/FlattenFunctionTest.java
index 61c3975..e829b1d 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/FlattenFunctionTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/FlattenFunctionTest.java
@@ -25,7 +25,7 @@ public class FlattenFunctionTest {
UDFContext udfContext = new UDFContext();
- udfContext.setLookup_fields(List.of("parent1", "parent2", "parent3", "parent4","parent5","parent6","parent7","parent8"));
+ udfContext.setLookupFields(List.of("parent1", "parent2", "parent3", "parent4","parent5","parent6","parent7","parent8"));
Map<String, Object> params = new HashMap<>();
params.put("prefix", "prefix");
params.put("depth", "4");
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/FromUnixTimestampTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/FromUnixTimestampTest.java
index d02303b..6cb1bf8 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/FromUnixTimestampTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/FromUnixTimestampTest.java
@@ -19,15 +19,15 @@ public class FromUnixTimestampTest {
@BeforeAll
public static void setUp() {
udfContext = new UDFContext();
- udfContext.setLookup_fields(Arrays.asList("unixTimestamp"));
- udfContext.setOutput_fields(Arrays.asList("timestamp"));
+ udfContext.setLookupFields(Arrays.asList("unixTimestamp"));
+ udfContext.setOutputFields(Arrays.asList("timestamp"));
}
@Test
- public void testFromUnixTimestampFunction() throws Exception {
+ public void testFromUnixTimestampMsFunction() throws Exception {
Map<String, Object> parameters = new HashMap<>();
parameters.put("precision", "seconds");
- parameters.put("timezone", "UTC");
+ parameters.put("timezone", "Asia/Shanghai");
udfContext.setParameters(parameters);
FromUnixTimestamp fromUnixTimestamp = new FromUnixTimestamp();
fromUnixTimestamp.open(null, udfContext);
@@ -36,11 +36,32 @@ public class FromUnixTimestampTest {
extractedFields.put("unixTimestamp", 1577808000000L);
event.setExtractedFields(extractedFields);
Event result1 = fromUnixTimestamp.evaluate(event);
+ assertEquals("2020-01-01 00:00:00", result1.getExtractedFields().get("timestamp"));
+ parameters.put("precision", "milliseconds");
+ fromUnixTimestamp.open(null, udfContext);
+ Event result2 = fromUnixTimestamp.evaluate(event);
+ assertEquals("2020-01-01 00:00:00.000", result2.getExtractedFields().get("timestamp"));
+
+ }
+
+ @Test
+ public void testFromUnixTimestampFunction() throws Exception {
+
+ Map<String, Object> parameters = new HashMap<>();
+ parameters.put("precision", "seconds");
+ udfContext.setParameters(parameters);
+ FromUnixTimestamp fromUnixTimestamp = new FromUnixTimestamp();
+ fromUnixTimestamp.open(null, udfContext);
+ Event event = new Event();
+ Map<String, Object> extractedFields = new HashMap<>();
+ extractedFields.put("unixTimestamp", 1577808000L);
+ event.setExtractedFields(extractedFields);
+ Event result1 = fromUnixTimestamp.evaluate(event);
assertEquals("2019-12-31 16:00:00", result1.getExtractedFields().get("timestamp"));
parameters.put("precision", "milliseconds");
fromUnixTimestamp.open(null, udfContext);
Event result2 = fromUnixTimestamp.evaluate(event);
- assertEquals("2019-12-31 16:00:00:000", result2.getExtractedFields().get("timestamp"));
+ assertEquals("2019-12-31 16:00:00.000", result2.getExtractedFields().get("timestamp"));
}
}
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/GenerateStringArrayFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/GenerateStringArrayFunctionTest.java
index 5490299..1fbe06c 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/GenerateStringArrayFunctionTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/GenerateStringArrayFunctionTest.java
@@ -17,8 +17,8 @@ public class GenerateStringArrayFunctionTest {
@BeforeAll
public static void setUp() {
udfContext = new UDFContext();
- udfContext.setLookup_fields(Arrays.asList("t1", "t2","t3","t4","t5"));
- udfContext.setOutput_fields(Collections.singletonList("result_list"));
+ udfContext.setLookupFields(Arrays.asList("t1", "t2","t3","t4","t5"));
+ udfContext.setOutputFields(Collections.singletonList("result_list"));
}
// 测试方法
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/HmacFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/HmacFunctionTest.java
new file mode 100644
index 0000000..5a4d0d3
--- /dev/null
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/HmacFunctionTest.java
@@ -0,0 +1,136 @@
+package com.geedgenetworks.core.udf.test.simple;
+
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
+import com.geedgenetworks.common.udf.UDFContext;
+import com.geedgenetworks.core.udf.Hmac;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+public class HmacFunctionTest {
+
+ private static final String SECRET_KEY = ".geedgenetworks.";
+ private static final String DATA = "13812345678";
+ private static UDFContext udfContext;
+
+ @BeforeAll
+ public static void setUp() {
+ udfContext = new UDFContext();
+ udfContext.setLookupFields(Collections.singletonList("phone_number"));
+ udfContext.setOutputFields(Collections.singletonList("phone_number_mac"));
+ }
+
+ @Test
+ public void testHmacAsBase64() {
+ Map<String, Object> map = new HashMap<>();
+ map.put("secret_key", SECRET_KEY);
+ map.put("algorithm", "sha256");
+ map.put("output_format", "base64");
+ udfContext.setParameters(map);
+ Hmac hmac = new Hmac();
+ hmac.open(null, udfContext);
+ Event event = new Event();
+ Map<String, Object> extractedFields = new HashMap<>();
+ extractedFields.put("phone_number", DATA);
+ event.setExtractedFields(extractedFields);
+ Event result1 = hmac.evaluate(event);
+ assertEquals("zaj6UKovIsDahIBeRZ2PmgPIfDEr900F2xWu+iQfFrw=", result1.getExtractedFields().get("phone_number_mac"));
+ }
+
+ @Test
+ public void testHmacAsHex() {
+ Map<String, Object> map = new HashMap<>();
+ map.put("secret_key", SECRET_KEY);
+ map.put("algorithm", "sha256");
+ map.put("output_format", "hex");
+ udfContext.setParameters(map);
+ Hmac hmac = new Hmac();
+ hmac.open(null, udfContext);
+ Event event = new Event();
+ Map<String, Object> extractedFields = new HashMap<>();
+ extractedFields.put("phone_number", DATA);
+ event.setExtractedFields(extractedFields);
+ Event result1 = hmac.evaluate(event);
+ assertEquals("cda8fa50aa2f22c0da84805e459d8f9a03c87c312bf74d05db15aefa241f16bc", result1.getExtractedFields().get("phone_number_mac"));
+ }
+
+ @Test
+ public void testHmacAlgorithm() {
+ Map<String, Object> map = new HashMap<>();
+ map.put("secret_key", SECRET_KEY);
+ map.put("algorithm", "sm4");
+ map.put("output_format", "base64");
+ udfContext.setParameters(map);
+ Hmac hmac = new Hmac();
+ hmac.open(null, udfContext);
+ Event event = new Event();
+ Map<String, Object> extractedFields = new HashMap<>();
+ extractedFields.put("phone_number", DATA);
+ event.setExtractedFields(extractedFields);
+ Event result = hmac.evaluate(event);
+ assertEquals("QX1q4Y7y3quYCDje9BuSjg==", result.getExtractedFields().get("phone_number_mac"));
+
+ map = new HashMap<>();
+ map.put("secret_key", SECRET_KEY);
+ map.put("algorithm", "sha1");
+ map.put("output_format", "base64");
+ udfContext.setParameters(map);
+ hmac = new Hmac();
+ hmac.open(null, udfContext);
+ event.setExtractedFields(extractedFields);
+ result = hmac.evaluate(event);
+ assertEquals("NB1b1TsVZ95/0sE+d/6kdtyUFh0=", result.getExtractedFields().get("phone_number_mac"));
+
+ map = new HashMap<>();
+ map.put("secret_key", SECRET_KEY);
+ map.put("algorithm", "sm3");
+ map.put("output_format", "base64");
+ udfContext.setParameters(map);
+ hmac = new Hmac();
+ hmac.open(null, udfContext);
+ event.setExtractedFields(extractedFields);
+ result = hmac.evaluate(event);
+ assertEquals("BbQNpwLWE3rkaI1WlPBJgYeD14UyL2OwTxiEoTNA3UU=", result.getExtractedFields().get("phone_number_mac"));
+
+ map = new HashMap<>();
+ map.put("secret_key", SECRET_KEY);
+ map.put("algorithm", "md5");
+ map.put("output_format", "base64");
+ udfContext.setParameters(map);
+ hmac = new Hmac();
+ hmac.open(null, udfContext);
+ event.setExtractedFields(extractedFields);
+ result = hmac.evaluate(event);
+ assertEquals("BQZzRqD3ZR/nJsDIOO4dBg==", result.getExtractedFields().get("phone_number_mac"));
+
+ map = new HashMap<>();
+ map.put("secret_key", SECRET_KEY);
+ map.put("algorithm", "sha512");
+ map.put("output_format", "base64");
+ udfContext.setParameters(map);
+ hmac = new Hmac();
+ hmac.open(null, udfContext);
+ event.setExtractedFields(extractedFields);
+ result = hmac.evaluate(event);
+ assertEquals("DWrndzlcqf2qvFTbuDC1gZCGmRhuAUayfsxEqr2ZlpY/QOr9HgGUZNOfytRfA4VT8OZK0BwHwcAg5pgGBvPQ4A==", result.getExtractedFields().get("phone_number_mac"));
+ }
+
+ @Test
+ public void testHmacError() {
+ Map<String, Object> map = new HashMap<>();
+ map.put("secret_key", SECRET_KEY);
+ map.put("algorithm", "sha256");
+ map.put("output_format", "hex");
+ udfContext.setParameters(map);
+ Hmac hmac = new Hmac();
+ udfContext.getParameters().remove("secret_key");
+ assertThrows(GrootStreamRuntimeException.class, () -> hmac.open(null, udfContext));
+ }
+}
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/JsonExtractFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/JsonExtractFunctionTest.java
index 99f3f96..dd661de 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/JsonExtractFunctionTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/JsonExtractFunctionTest.java
@@ -25,7 +25,7 @@ public class JsonExtractFunctionTest {
@Test
public void testInit(){
JsonExtract jsonExtract = new JsonExtract();
- udfContext.setLookup_fields(new ArrayList<>());
+ udfContext.setLookupFields(new ArrayList<>());
udfContext.setParameters(new HashMap<>());
udfContext.setParameters(null);
Assertions.assertThrows(GrootStreamRuntimeException.class, () -> {
@@ -52,8 +52,8 @@ public class JsonExtractFunctionTest {
Map<String, Object> parameters = new HashMap<>();
parameters.put("value_expression","$.tags[?(@.tag=='device_group')][0].value");
udfContext.setParameters(parameters);
- udfContext.setLookup_fields(Collections.singletonList("device_tag"));
- udfContext.setOutput_fields(Collections.singletonList("device_group"));
+ udfContext.setLookupFields(Collections.singletonList("device_tag"));
+ udfContext.setOutputFields(Collections.singletonList("device_group"));
jsonExtract.open(null, udfContext);
Event event = new Event();
String jsonString = "{\"device_tag\":\"{\\\"tags\\\":[{\\\"tag\\\":\\\"data_center\\\",\\\"value\\\":\\\"center-xxg-tsgx\\\"},{\\\"tag\\\":\\\"device_group\\\",\\\"value\\\":\\\"group-xxg-tsgx\\\"}]}\"}";
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/StringJoinerFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/StringJoinerFunctionTest.java
index d80eb97..e9cde8a 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/StringJoinerFunctionTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/StringJoinerFunctionTest.java
@@ -20,8 +20,8 @@ public class StringJoinerFunctionTest {
@BeforeAll
public static void setUp() {
udfContext = new UDFContext();
- udfContext.setLookup_fields(Arrays.asList("server_ip", "client_ip"));
- udfContext.setOutput_fields(Collections.singletonList("ip_string"));
+ udfContext.setLookupFields(Arrays.asList("server_ip", "client_ip"));
+ udfContext.setOutputFields(Collections.singletonList("ip_string"));
Map<String, Object> params = new HashMap<>();
params.put("separator",",");
params.put("prefix","[");
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/UUIDTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/UUIDTest.java
new file mode 100644
index 0000000..534569b
--- /dev/null
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/UUIDTest.java
@@ -0,0 +1,139 @@
+package com.geedgenetworks.core.udf.test.simple;
+
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.exception.GrootStreamRuntimeException;
+import com.geedgenetworks.common.udf.UDFContext;
+import com.geedgenetworks.core.udf.uuid.UUID;
+import com.geedgenetworks.core.udf.uuid.UUIDv5;
+import com.geedgenetworks.core.udf.uuid.UUIDv7;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.*;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+
+public class UUIDTest {
+
+ private UDFContext udfContext;
+ private Map<String, Object> parameters ;
+
+
+ @Test
+ public void testInit(){
+ udfContext = new UDFContext();
+ UUIDv5 uuidv5 = new UUIDv5();
+ parameters = new HashMap<>();
+ udfContext.setParameters(parameters);
+ udfContext.setLookupFields(List.of("client_ip","server_ip"));
+ udfContext.setOutputFields(Collections.singletonList("uuid"));
+ parameters.put("namespace","NAMESPACE_IP_1");
+ Assertions.assertThrows(GrootStreamRuntimeException.class, () -> {
+ uuidv5.open(null, udfContext);
+ });
+
+ }
+
+ @Test
+ public void testUUID() {
+ udfContext = new UDFContext();
+ UUID uuid = new UUID();
+ parameters = new HashMap<>();
+ udfContext.setParameters(parameters);
+ udfContext.setOutputFields(Collections.singletonList("uuid"));
+ uuid.open(null, udfContext);
+ Event event = new Event();
+ Map<String, Object> extractedFields = new HashMap<>();
+ event.setExtractedFields(extractedFields);
+ Event result1 = uuid.evaluate(event);
+ assertEquals(36, result1.getExtractedFields().get("uuid").toString().length());
+ }
+ @Test
+ public void testUUIDV7() {
+ udfContext = new UDFContext();
+ UUIDv7 uuid = new UUIDv7();
+ parameters = new HashMap<>();
+ udfContext.setParameters(parameters);
+ udfContext.setOutputFields(Collections.singletonList("uuid"));
+ uuid.open(null, udfContext);
+ Event event = new Event();
+ Map<String, Object> extractedFields = new HashMap<>();
+ event.setExtractedFields(extractedFields);
+ Event result1 = uuid.evaluate(event);
+ assertEquals(36, result1.getExtractedFields().get("uuid").toString().length());
+ }
+ @Test
+ public void testUUIDV5ForNameSpaceIp() {
+ udfContext = new UDFContext();
+ UUIDv5 uuidv5 = new UUIDv5();
+ parameters = new HashMap<>();
+ udfContext.setParameters(parameters);
+ udfContext.setLookupFields(List.of("client_ip", "server_ip"));
+ udfContext.setOutputFields(Collections.singletonList("uuid"));
+ parameters.put("namespace","NAMESPACE_IP");
+ uuidv5.open(null, udfContext);
+ Event event = new Event();
+ Map<String, Object> extractedFields = new HashMap<>();
+ extractedFields.put("client_ip", "1.1.1.1");
+ extractedFields.put("server_ip", "");
+ event.setExtractedFields(extractedFields);
+ Event result = uuidv5.evaluate(event);
+ System.out.printf("uuid: %s\n", result.getExtractedFields().get("uuid").toString());
+ assertEquals("5394a6a8-b9b8-5147-b5b2-01365f158acb", result.getExtractedFields().get("uuid").toString());
+ assertNotEquals("ecc67867-1f76-580c-a4c1-6a3d16ad6d02", result.getExtractedFields().get("uuid").toString());
+ }
+
+ @Test
+ public void testUUIDV5ForNameSpaceDomain() {
+ udfContext = new UDFContext();
+ UUIDv5 uuidv5 = new UUIDv5();
+ parameters = new HashMap<>();
+ udfContext.setParameters(parameters);
+ udfContext.setLookupFields(List.of("domain"));
+ udfContext.setOutputFields(Collections.singletonList("uuid"));
+ parameters.put("namespace","NAMESPACE_DOMAIN");
+ uuidv5.open(null, udfContext);
+ Event event = new Event();
+ Map<String, Object> extractedFields = new HashMap<>();
+ extractedFields.put("domain", "baidu");
+ event.setExtractedFields(extractedFields);
+ Event result1 = uuidv5.evaluate(event);
+ assertEquals("fd67cec1-6b33-5def-835c-fbe32f1ce4a4", result1.getExtractedFields().get("uuid").toString());
+ }
+ @Test
+ public void testUUIDv5ForNameSpaceApp() {
+ udfContext = new UDFContext();
+ UUIDv5 uuidv5 = new UUIDv5();
+ parameters = new HashMap<>();
+ udfContext.setParameters(parameters);
+ udfContext.setLookupFields(List.of("app"));
+ udfContext.setOutputFields(Collections.singletonList("uuid"));
+ parameters.put("namespace","NAMESPACE_APP");
+ uuidv5.open(null, udfContext);
+ Event event = new Event();
+ Map<String, Object> extractedFields = new HashMap<>();
+ extractedFields.put("app", "54321");
+ event.setExtractedFields(extractedFields);
+ Event result1 = uuidv5.evaluate(event);
+ assertEquals("194b8304-64f0-599e-bf7a-66cbc21bf6ee", result1.getExtractedFields().get("uuid").toString());
+ }
+
+ @Test
+ public void testUUIDV5ForNameSpaceSubscriberID() {
+ udfContext = new UDFContext();
+ UUIDv5 uuidv5 = new UUIDv5();
+ parameters = new HashMap<>();
+ udfContext.setParameters(parameters);
+ udfContext.setLookupFields(List.of("subscriber_id"));
+ udfContext.setOutputFields(Collections.singletonList("uuid"));
+ parameters.put("namespace","NAMESPACE_SUBSCRIBER");
+ uuidv5.open(null, udfContext);
+ Event event = new Event();
+ Map<String, Object> extractedFields = new HashMap<>();
+ extractedFields.put("subscriber_id", "test1");
+ event.setExtractedFields(extractedFields);
+ Event result1 = uuidv5.evaluate(event);
+ assertEquals("9b154520-3c29-541c-bb81-f649354dae67", result1.getExtractedFields().get("uuid").toString());
+ }
+}
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/UnixTimestampConverterTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/UnixTimestampConverterTest.java
index 12a2093..a0d70d7 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/UnixTimestampConverterTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/UnixTimestampConverterTest.java
@@ -19,8 +19,8 @@ public class UnixTimestampConverterTest {
@BeforeAll
public static void setUp() {
udfContext = new UDFContext();
- udfContext.setLookup_fields(Arrays.asList("input"));
- udfContext.setOutput_fields(Arrays.asList("output"));
+ udfContext.setLookupFields(Arrays.asList("input"));
+ udfContext.setOutputFields(Arrays.asList("output"));
}
@Test
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/JsonUnrollFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/JsonUnrollFunctionTest.java
index 02f0b66..3749eb1 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/JsonUnrollFunctionTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/JsonUnrollFunctionTest.java
@@ -52,8 +52,8 @@ public class JsonUnrollFunctionTest {
JsonUnroll unroll = new JsonUnroll();
Event event = new Event();
event.setExtractedFields(nestedMap);
- udfContext.setLookup_fields(List.of("k1"));
- udfContext.setOutput_fields(List.of("newk1"));
+ udfContext.setLookupFields(List.of("k1"));
+ udfContext.setOutputFields(List.of("newk1"));
unroll.open(null, udfContext);
List<Event> result3 = unroll.evaluate(event);
assertEquals(2, result3.size());
@@ -75,8 +75,8 @@ public class JsonUnrollFunctionTest {
udfContext.setParameters(params);
params.put("path", "$.k3_1.k3_1_1");
event.setExtractedFields(nestedMap);
- udfContext.setLookup_fields(List.of("k3"));
- udfContext.setOutput_fields(List.of("newk3"));
+ udfContext.setLookupFields(List.of("k3"));
+ udfContext.setOutputFields(List.of("newk3"));
unroll.open(null, udfContext);
List<Event> result2 = unroll.evaluate(event);
assertEquals(2, result2.size());
@@ -95,8 +95,8 @@ public class JsonUnrollFunctionTest {
udfContext.setParameters(params);
params.put("path", "$.k4_1.k4_1_1");
event.setExtractedFields(nestedMap);
- udfContext.setLookup_fields(List.of("k4"));
- udfContext.setOutput_fields(List.of("newk4"));
+ udfContext.setLookupFields(List.of("k4"));
+ udfContext.setOutputFields(List.of("newk4"));
unroll.open(null, udfContext);
List<Event> result2 = unroll.evaluate(event);
assertEquals(1, result2.size());
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnrollFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnrollFunctionTest.java
index 2f4da76..db66e55 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnrollFunctionTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnrollFunctionTest.java
@@ -34,8 +34,8 @@ public class UnrollFunctionTest {
UDFContext udfContext = new UDFContext();
- udfContext.setLookup_fields(List.of("k1"));
- udfContext.setOutput_fields(List.of("newk1"));
+ udfContext.setLookupFields(List.of("k1"));
+ udfContext.setOutputFields(List.of("newk1"));
Unroll unroll = new Unroll();
unroll.open(null, udfContext);
Event event = new Event();
@@ -60,8 +60,8 @@ public class UnrollFunctionTest {
udfContext.setParameters(params);
udfContext.setParameters(params);
event.setExtractedFields(nestedMap);
- udfContext.setLookup_fields(List.of("k2"));
- udfContext.setOutput_fields(List.of("k2"));
+ udfContext.setLookupFields(List.of("k2"));
+ udfContext.setOutputFields(List.of("k2"));
unroll.open(null, udfContext);
List<Event> result2 = unroll.evaluate(event);
assertEquals(3, result2.size());
@@ -76,8 +76,8 @@ public class UnrollFunctionTest {
Unroll unroll = new Unroll();
Event event = new Event();
event.setExtractedFields(nestedMap);
- udfContext.setLookup_fields(List.of("k3"));
- udfContext.setOutput_fields(List.of("newk3"));
+ udfContext.setLookupFields(List.of("k3"));
+ udfContext.setOutputFields(List.of("newk3"));
unroll.open(null, udfContext);
List<Event> result2 = unroll.evaluate(event);
assertEquals(1, result2.size());
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantileTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantileTest.java
index 33f7bad..990186d 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantileTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantileTest.java
@@ -1,89 +1,89 @@
-package com.geedgenetworks.core.udf.udaf.HdrHistogram;
-
-import com.geedgenetworks.common.Accumulator;
-import com.geedgenetworks.common.Event;
-import com.geedgenetworks.common.udf.AggregateFunction;
-import com.geedgenetworks.common.udf.UDFContext;
-import com.geedgenetworks.sketch.util.StringUtils;
-import org.HdrHistogram.ArrayHistogram;
-import org.junit.jupiter.api.Test;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.junit.jupiter.api.Assertions.*;
-
-public class HdrHistogramQuantileTest {
- AggregateFunction agg;
- Accumulator acc;
- Event event;
-
- @Test
- public void inputRegular() {
- double probability = 0.5;
- initData( "regular", 2, probability);
- long count = 100000;
- Map<String, Object> fields = event.getExtractedFields();
- for (int i = 1; i <= count; i++) {
- fields.put("ms", i);
- agg.add(event, acc);
- }
-
- long expect = (long) (count * probability);
- long rst = (long)agg.getResult(acc).getMetricsFields().get("ms_his");
- double error = Math.abs(rst - expect) / (double) expect;
- System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error));
- assertTrue(error <= 0.05);
- }
-
- @Test
- public void inputSketch() {
- double probability = 0.5;
- initData( "sketch", 2, probability);
- long count = 100000;
- Map<String, Object> fields = event.getExtractedFields();
-
- ArrayHistogram his = new ArrayHistogram(2);
- for (int i = 1; i <= count; i++) {
- his.recordValue(i);
- }
- fields.put("ms", StringUtils.encodeBase64String(his.toBytes()));
- agg.add(event, acc);
-
- his = new ArrayHistogram(2);
- for (int i = 1; i <= count; i++) {
- his.recordValue(i);
- }
- fields.put("ms", his.toBytes());
- agg.add(event, acc);
-
- long expect = (long) (count * probability);
- long rst = (long)agg.getResult(acc).getMetricsFields().get("ms_his");
- double error = Math.abs(rst - expect) / (double) expect;
- System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error));
- assertTrue(error <= 0.05);
- }
-
- private void initData(String input_type, int numberOfSignificantValueDigits, double probability){
- agg = new HdrHistogramQuantile();
- UDFContext c = new UDFContext();
- Map<String, Object> parameters = new HashMap<>();
- parameters.put("input_type", input_type);
- parameters.put("numberOfSignificantValueDigits", numberOfSignificantValueDigits);
- parameters.put("probability", probability);
- c.setParameters(parameters);
- c.setLookup_fields(Collections.singletonList("ms"));
- c.setOutput_fields(Collections.singletonList("ms_his"));
-
- agg.open(c);
- Map<String, Object> map = new HashMap<>();
- acc = new Accumulator();
- acc.setMetricsFields(map);
- agg.initAccumulator(acc);
-
- event = new Event();
- Map<String, Object> fields = new HashMap<>();
- event.setExtractedFields(fields);
- }
+package com.geedgenetworks.core.udf.udaf.HdrHistogram;
+
+import com.geedgenetworks.common.Accumulator;
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.udf.AggregateFunction;
+import com.geedgenetworks.common.udf.UDFContext;
+import com.geedgenetworks.sketch.util.StringUtils;
+import org.HdrHistogram.ArrayHistogram;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+public class HdrHistogramQuantileTest {
+ AggregateFunction agg;
+ Accumulator acc;
+ Event event;
+
+ @Test
+ public void inputRegular() {
+ double probability = 0.5;
+ initData( "regular", 2, probability);
+ long count = 100000;
+ Map<String, Object> fields = event.getExtractedFields();
+ for (int i = 1; i <= count; i++) {
+ fields.put("ms", i);
+ agg.add(event, acc);
+ }
+
+ long expect = (long) (count * probability);
+ long rst = (long)agg.getResult(acc).getMetricsFields().get("ms_his");
+ double error = Math.abs(rst - expect) / (double) expect;
+ System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error));
+ assertTrue(error <= 0.05);
+ }
+
+ @Test
+ public void inputSketch() {
+ double probability = 0.5;
+ initData( "sketch", 2, probability);
+ long count = 100000;
+ Map<String, Object> fields = event.getExtractedFields();
+
+ ArrayHistogram his = new ArrayHistogram(2);
+ for (int i = 1; i <= count; i++) {
+ his.recordValue(i);
+ }
+ fields.put("ms", StringUtils.encodeBase64String(his.toBytes()));
+ agg.add(event, acc);
+
+ his = new ArrayHistogram(2);
+ for (int i = 1; i <= count; i++) {
+ his.recordValue(i);
+ }
+ fields.put("ms", his.toBytes());
+ agg.add(event, acc);
+
+ long expect = (long) (count * probability);
+ long rst = (long)agg.getResult(acc).getMetricsFields().get("ms_his");
+ double error = Math.abs(rst - expect) / (double) expect;
+ System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error));
+ assertTrue(error <= 0.05);
+ }
+
+ private void initData(String input_type, int numberOfSignificantValueDigits, double probability){
+ agg = new HdrHistogramQuantile();
+ UDFContext c = new UDFContext();
+ Map<String, Object> parameters = new HashMap<>();
+ parameters.put("input_type", input_type);
+ parameters.put("numberOfSignificantValueDigits", numberOfSignificantValueDigits);
+ parameters.put("probability", probability);
+ c.setParameters(parameters);
+ c.setLookupFields(Collections.singletonList("ms"));
+ c.setOutputFields(Collections.singletonList("ms_his"));
+
+ agg.open(c);
+ Map<String, Object> map = new HashMap<>();
+ acc = new Accumulator();
+ acc.setMetricsFields(map);
+ agg.initAccumulator(acc);
+
+ event = new Event();
+ Map<String, Object> fields = new HashMap<>();
+ event.setExtractedFields(fields);
+ }
} \ No newline at end of file
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantilesTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantilesTest.java
index 4eefd9a..a57645d 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantilesTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantilesTest.java
@@ -1,98 +1,98 @@
-package com.geedgenetworks.core.udf.udaf.HdrHistogram;
-
-import com.geedgenetworks.common.Accumulator;
-import com.geedgenetworks.common.Event;
-import com.geedgenetworks.common.udf.AggregateFunction;
-import com.geedgenetworks.common.udf.UDFContext;
-import com.geedgenetworks.sketch.util.StringUtils;
-import org.HdrHistogram.ArrayHistogram;
-import org.junit.jupiter.api.Test;
-
-import java.util.*;
-
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-public class HdrHistogramQuantilesTest {
- AggregateFunction agg;
- Accumulator acc;
- Event event;
-
- @Test
- public void inputRegular() {
- double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1};
- initData( "regular", 2, probabilities);
- long count = 100000;
- Map<String, Object> fields = event.getExtractedFields();
- for (int i = 1; i <= count; i++) {
- fields.put("ms", i);
- agg.add(event, acc);
- }
-
- long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray();
-
- List<Long> rsts = (List<Long>)agg.getResult(acc).getMetricsFields().get("ms_his");
- for (int i = 0; i < expects.length; i++) {
- long rst = rsts.get(i);
- long expect = expects[i];
- double probability = probabilities[i];
- double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect;
- System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error));
- assertTrue(error <= 0.05);
- }
- }
-
- @Test
- public void inputSketch() {
- double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1};
- initData( "sketch", 2, probabilities);
- long count = 100000;
- Map<String, Object> fields = event.getExtractedFields();
- long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray();
-
- ArrayHistogram his = new ArrayHistogram(2);
- for (int i = 1; i <= count; i++) {
- his.recordValue(i);
- }
- fields.put("ms", StringUtils.encodeBase64String(his.toBytes()));
- agg.add(event, acc);
-
- his = new ArrayHistogram(2);
- for (int i = 1; i <= count; i++) {
- his.recordValue(i);
- }
- fields.put("ms", his.toBytes());
- agg.add(event, acc);
-
- List<Long> rsts = (List<Long>)agg.getResult(acc).getMetricsFields().get("ms_his");
- for (int i = 0; i < expects.length; i++) {
- long rst = rsts.get(i);
- long expect = expects[i];
- double probability = probabilities[i];
- double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect;
- System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error));
- assertTrue(error <= 0.05);
- }
- }
-
- private void initData(String input_type, int numberOfSignificantValueDigits, double[] probabilities){
- agg = new HdrHistogramQuantiles();
- UDFContext c = new UDFContext();
- Map<String, Object> parameters = new HashMap<>();
- parameters.put("input_type", input_type);
- parameters.put("numberOfSignificantValueDigits", numberOfSignificantValueDigits);
- parameters.put("probabilities", probabilities);
- c.setParameters(parameters);
- c.setLookup_fields(Collections.singletonList("ms"));
- c.setOutput_fields(Collections.singletonList("ms_his"));
-
- agg.open(c);
- Map<String, Object> map = new HashMap<>();
- acc = new Accumulator();
- acc.setMetricsFields(map);
- agg.initAccumulator(acc);
-
- event = new Event();
- Map<String, Object> fields = new HashMap<>();
- event.setExtractedFields(fields);
- }
+package com.geedgenetworks.core.udf.udaf.HdrHistogram;
+
+import com.geedgenetworks.common.Accumulator;
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.udf.AggregateFunction;
+import com.geedgenetworks.common.udf.UDFContext;
+import com.geedgenetworks.sketch.util.StringUtils;
+import org.HdrHistogram.ArrayHistogram;
+import org.junit.jupiter.api.Test;
+
+import java.util.*;
+
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class HdrHistogramQuantilesTest {
+ AggregateFunction agg;
+ Accumulator acc;
+ Event event;
+
+ @Test
+ public void inputRegular() {
+ double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1};
+ initData( "regular", 2, probabilities);
+ long count = 100000;
+ Map<String, Object> fields = event.getExtractedFields();
+ for (int i = 1; i <= count; i++) {
+ fields.put("ms", i);
+ agg.add(event, acc);
+ }
+
+ long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray();
+
+ List<Long> rsts = (List<Long>)agg.getResult(acc).getMetricsFields().get("ms_his");
+ for (int i = 0; i < expects.length; i++) {
+ long rst = rsts.get(i);
+ long expect = expects[i];
+ double probability = probabilities[i];
+ double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect;
+ System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error));
+ assertTrue(error <= 0.05);
+ }
+ }
+
+ @Test
+ public void inputSketch() {
+ double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1};
+ initData( "sketch", 2, probabilities);
+ long count = 100000;
+ Map<String, Object> fields = event.getExtractedFields();
+ long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray();
+
+ ArrayHistogram his = new ArrayHistogram(2);
+ for (int i = 1; i <= count; i++) {
+ his.recordValue(i);
+ }
+ fields.put("ms", StringUtils.encodeBase64String(his.toBytes()));
+ agg.add(event, acc);
+
+ his = new ArrayHistogram(2);
+ for (int i = 1; i <= count; i++) {
+ his.recordValue(i);
+ }
+ fields.put("ms", his.toBytes());
+ agg.add(event, acc);
+
+ List<Long> rsts = (List<Long>)agg.getResult(acc).getMetricsFields().get("ms_his");
+ for (int i = 0; i < expects.length; i++) {
+ long rst = rsts.get(i);
+ long expect = expects[i];
+ double probability = probabilities[i];
+ double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect;
+ System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error));
+ assertTrue(error <= 0.05);
+ }
+ }
+
+ private void initData(String input_type, int numberOfSignificantValueDigits, double[] probabilities){
+ agg = new HdrHistogramQuantiles();
+ UDFContext c = new UDFContext();
+ Map<String, Object> parameters = new HashMap<>();
+ parameters.put("input_type", input_type);
+ parameters.put("numberOfSignificantValueDigits", numberOfSignificantValueDigits);
+ parameters.put("probabilities", probabilities);
+ c.setParameters(parameters);
+ c.setLookupFields(Collections.singletonList("ms"));
+ c.setOutputFields(Collections.singletonList("ms_his"));
+
+ agg.open(c);
+ Map<String, Object> map = new HashMap<>();
+ acc = new Accumulator();
+ acc.setMetricsFields(map);
+ agg.initAccumulator(acc);
+
+ event = new Event();
+ Map<String, Object> fields = new HashMap<>();
+ event.setExtractedFields(fields);
+ }
} \ No newline at end of file
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramTest.java
index f177ca5..5905138 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramTest.java
@@ -1,102 +1,102 @@
-package com.geedgenetworks.core.udf.udaf.HdrHistogram;
-
-import com.geedgenetworks.common.Accumulator;
-import com.geedgenetworks.common.Event;
-import com.geedgenetworks.common.udf.AggregateFunction;
-import com.geedgenetworks.common.udf.UDFContext;
-import com.geedgenetworks.sketch.util.StringUtils;
-import org.HdrHistogram.ArrayHistogram;
-import org.junit.jupiter.api.Test;
-
-import java.nio.charset.StandardCharsets;
-import java.util.*;
-
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-public class HdrHistogramTest {
- AggregateFunction agg;
- Accumulator acc;
- Event event;
-
- @Test
- public void inputRegular() {
- double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1};
- initData( "regular", 2, "base64");
- long count = 100000;
- Map<String, Object> fields = event.getExtractedFields();
- for (int i = 1; i <= count; i++) {
- fields.put("ms", i);
- agg.add(event, acc);
- }
-
- long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray();
- String str = (String) agg.getResult(acc).getMetricsFields().get("ms_his");
- ArrayHistogram his = ArrayHistogram.fromBytes(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
-
- for (int i = 0; i < expects.length; i++) {
- long rst = his.getValueAtPercentile(probabilities[i] * 100);
- long expect = expects[i];
- double probability = probabilities[i];
- double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect;
- System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error));
- assertTrue(error <= 0.05);
- }
- }
-
- @Test
- public void inputSketch() {
- double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1};
- initData( "sketch", 2, "binary");
- long count = 100000;
- Map<String, Object> fields = event.getExtractedFields();
- long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray();
-
- ArrayHistogram his = new ArrayHistogram(2);
- for (int i = 1; i <= count; i++) {
- his.recordValue(i);
- }
- fields.put("ms", StringUtils.encodeBase64String(his.toBytes()));
- agg.add(event, acc);
-
- his = new ArrayHistogram(2);
- for (int i = 1; i <= count; i++) {
- his.recordValue(i);
- }
- fields.put("ms", his.toBytes());
- agg.add(event, acc);
-
- byte[] bytes = (byte[]) agg.getResult(acc).getMetricsFields().get("ms_his");
- ArrayHistogram h = ArrayHistogram.fromBytes(bytes);
-
- for (int i = 0; i < expects.length; i++) {
- long rst = h.getValueAtPercentile(probabilities[i] * 100);
- long expect = expects[i];
- double probability = probabilities[i];
- double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect;
- System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error));
- assertTrue(error <= 0.05);
- }
- }
-
- private void initData(String input_type, int numberOfSignificantValueDigits, String output_format){
- agg = new HdrHistogram();
- UDFContext c = new UDFContext();
- Map<String, Object> parameters = new HashMap<>();
- parameters.put("input_type", input_type);
- parameters.put("numberOfSignificantValueDigits", numberOfSignificantValueDigits);
- parameters.put("output_format", output_format);
- c.setParameters(parameters);
- c.setLookup_fields(Collections.singletonList("ms"));
- c.setOutput_fields(Collections.singletonList("ms_his"));
-
- agg.open(c);
- Map<String, Object> map = new HashMap<>();
- acc = new Accumulator();
- acc.setMetricsFields(map);
- agg.initAccumulator(acc);
-
- event = new Event();
- Map<String, Object> fields = new HashMap<>();
- event.setExtractedFields(fields);
- }
+package com.geedgenetworks.core.udf.udaf.HdrHistogram;
+
+import com.geedgenetworks.common.Accumulator;
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.udf.AggregateFunction;
+import com.geedgenetworks.common.udf.UDFContext;
+import com.geedgenetworks.sketch.util.StringUtils;
+import org.HdrHistogram.ArrayHistogram;
+import org.junit.jupiter.api.Test;
+
+import java.nio.charset.StandardCharsets;
+import java.util.*;
+
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class HdrHistogramTest {
+ AggregateFunction agg;
+ Accumulator acc;
+ Event event;
+
+ @Test
+ public void inputRegular() {
+ double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1};
+ initData( "regular", 2, "base64");
+ long count = 100000;
+ Map<String, Object> fields = event.getExtractedFields();
+ for (int i = 1; i <= count; i++) {
+ fields.put("ms", i);
+ agg.add(event, acc);
+ }
+
+ long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray();
+ String str = (String) agg.getResult(acc).getMetricsFields().get("ms_his");
+ ArrayHistogram his = ArrayHistogram.fromBytes(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
+
+ for (int i = 0; i < expects.length; i++) {
+ long rst = his.getValueAtPercentile(probabilities[i] * 100);
+ long expect = expects[i];
+ double probability = probabilities[i];
+ double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect;
+ System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error));
+ assertTrue(error <= 0.05);
+ }
+ }
+
+ @Test
+ public void inputSketch() {
+ double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1};
+ initData( "sketch", 2, "binary");
+ long count = 100000;
+ Map<String, Object> fields = event.getExtractedFields();
+ long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray();
+
+ ArrayHistogram his = new ArrayHistogram(2);
+ for (int i = 1; i <= count; i++) {
+ his.recordValue(i);
+ }
+ fields.put("ms", StringUtils.encodeBase64String(his.toBytes()));
+ agg.add(event, acc);
+
+ his = new ArrayHistogram(2);
+ for (int i = 1; i <= count; i++) {
+ his.recordValue(i);
+ }
+ fields.put("ms", his.toBytes());
+ agg.add(event, acc);
+
+ byte[] bytes = (byte[]) agg.getResult(acc).getMetricsFields().get("ms_his");
+ ArrayHistogram h = ArrayHistogram.fromBytes(bytes);
+
+ for (int i = 0; i < expects.length; i++) {
+ long rst = h.getValueAtPercentile(probabilities[i] * 100);
+ long expect = expects[i];
+ double probability = probabilities[i];
+ double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect;
+ System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error));
+ assertTrue(error <= 0.05);
+ }
+ }
+
+ private void initData(String input_type, int numberOfSignificantValueDigits, String output_format){
+ agg = new HdrHistogram();
+ UDFContext c = new UDFContext();
+ Map<String, Object> parameters = new HashMap<>();
+ parameters.put("input_type", input_type);
+ parameters.put("numberOfSignificantValueDigits", numberOfSignificantValueDigits);
+ parameters.put("output_format", output_format);
+ c.setParameters(parameters);
+ c.setLookupFields(Collections.singletonList("ms"));
+ c.setOutputFields(Collections.singletonList("ms_his"));
+
+ agg.open(c);
+ Map<String, Object> map = new HashMap<>();
+ acc = new Accumulator();
+ acc.setMetricsFields(map);
+ agg.initAccumulator(acc);
+
+ event = new Event();
+ Map<String, Object> fields = new HashMap<>();
+ event.setExtractedFields(fields);
+ }
} \ No newline at end of file
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldApproxCountDistinctTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldApproxCountDistinctTest.java
index eae356d..b80d782 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldApproxCountDistinctTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldApproxCountDistinctTest.java
@@ -1,87 +1,87 @@
-package com.geedgenetworks.core.udf.udaf.hlld;
-
-
-import com.geedgenetworks.common.Accumulator;
-import com.geedgenetworks.common.Event;
-import com.geedgenetworks.common.udf.AggregateFunction;
-import com.geedgenetworks.common.udf.UDFContext;
-import com.geedgenetworks.sketch.hlld.Hll;
-import com.geedgenetworks.sketch.util.StringUtils;
-import org.junit.jupiter.api.Test;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-
-public class HlldApproxCountDistinctTest {
- AggregateFunction agg;
- Accumulator acc;
- Event event;
-
-
- @Test
- public void inputRegular() {
- initData(14, "regular");
- long count = 100000;
- Map<String, Object> fields = event.getExtractedFields();
- for (int i = 0; i < count; i++) {
- fields.put("ip", i);
- agg.add(event, acc);
- }
-
- long rst = (long)agg.getResult(acc).getMetricsFields().get("ip_cnt");
- double error = Math.abs(rst - count) / (double) count;
- System.out.println(String.format("%d,%d,%.4f", count , rst , error));
- assertTrue(error <= 0.05);
- }
-
- @Test
- public void inputSketch() {
- initData(14, "sketch");
- long count = 150000;
- Map<String, Object> fields = event.getExtractedFields();
-
- Hll hll = new Hll(12);
- for (int i = 0; i < 100000; i++) {
- hll.add(i);
- }
- fields.put("ip", StringUtils.encodeBase64String(hll.toBytes()));
- agg.add(event, acc);
-
- hll = new Hll(13);
- for (int i = 50000; i < 150000; i++) {
- hll.add(i);
- }
- fields.put("ip", hll.toBytes());
- agg.add(event, acc);
-
- long rst = (long)agg.getResult(acc).getMetricsFields().get("ip_cnt");
- double error = Math.abs(rst - count) / (double) count;
- System.out.println(String.format("%d,%d,%.4f", count , rst , error));
- assertTrue(error <= 0.05);
- }
-
- private void initData(int precision, String input_type){
- agg = new HlldApproxCountDistinct();
- UDFContext c = new UDFContext();
- Map<String, Object> parameters = new HashMap<>();
- parameters.put("precision", precision);
- parameters.put("input_type", input_type);
- c.setParameters(parameters);
- c.setLookup_fields(Collections.singletonList("ip"));
- c.setOutput_fields(Collections.singletonList("ip_cnt"));
-
- agg.open(c);
- Map<String, Object> map = new HashMap<>();
- acc = new Accumulator();
- acc.setMetricsFields(map);
- agg.initAccumulator(acc);
-
- event = new Event();
- Map<String, Object> fields = new HashMap<>();
- event.setExtractedFields(fields);
- }
+package com.geedgenetworks.core.udf.udaf.hlld;
+
+
+import com.geedgenetworks.common.Accumulator;
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.udf.AggregateFunction;
+import com.geedgenetworks.common.udf.UDFContext;
+import com.geedgenetworks.sketch.hlld.Hll;
+import com.geedgenetworks.sketch.util.StringUtils;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+
+public class HlldApproxCountDistinctTest {
+ AggregateFunction agg;
+ Accumulator acc;
+ Event event;
+
+
+ @Test
+ public void inputRegular() {
+ initData(14, "regular");
+ long count = 100000;
+ Map<String, Object> fields = event.getExtractedFields();
+ for (int i = 0; i < count; i++) {
+ fields.put("ip", i);
+ agg.add(event, acc);
+ }
+
+ long rst = (long)agg.getResult(acc).getMetricsFields().get("ip_cnt");
+ double error = Math.abs(rst - count) / (double) count;
+ System.out.println(String.format("%d,%d,%.4f", count , rst , error));
+ assertTrue(error <= 0.05);
+ }
+
+ @Test
+ public void inputSketch() {
+ initData(14, "sketch");
+ long count = 150000;
+ Map<String, Object> fields = event.getExtractedFields();
+
+ Hll hll = new Hll(12);
+ for (int i = 0; i < 100000; i++) {
+ hll.add(i);
+ }
+ fields.put("ip", StringUtils.encodeBase64String(hll.toBytes()));
+ agg.add(event, acc);
+
+ hll = new Hll(13);
+ for (int i = 50000; i < 150000; i++) {
+ hll.add(i);
+ }
+ fields.put("ip", hll.toBytes());
+ agg.add(event, acc);
+
+ long rst = (long)agg.getResult(acc).getMetricsFields().get("ip_cnt");
+ double error = Math.abs(rst - count) / (double) count;
+ System.out.println(String.format("%d,%d,%.4f", count , rst , error));
+ assertTrue(error <= 0.05);
+ }
+
+ private void initData(int precision, String input_type){
+ agg = new HlldApproxCountDistinct();
+ UDFContext c = new UDFContext();
+ Map<String, Object> parameters = new HashMap<>();
+ parameters.put("precision", precision);
+ parameters.put("input_type", input_type);
+ c.setParameters(parameters);
+ c.setLookupFields(Collections.singletonList("ip"));
+ c.setOutputFields(Collections.singletonList("ip_cnt"));
+
+ agg.open(c);
+ Map<String, Object> map = new HashMap<>();
+ acc = new Accumulator();
+ acc.setMetricsFields(map);
+ agg.initAccumulator(acc);
+
+ event = new Event();
+ Map<String, Object> fields = new HashMap<>();
+ event.setExtractedFields(fields);
+ }
} \ No newline at end of file
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldTest.java
index f489ee4..d6ed4c1 100644
--- a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldTest.java
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldTest.java
@@ -1,86 +1,86 @@
-package com.geedgenetworks.core.udf.udaf.hlld;
-
-import com.geedgenetworks.common.Accumulator;
-import com.geedgenetworks.common.Event;
-import com.geedgenetworks.common.udf.AggregateFunction;
-import com.geedgenetworks.common.udf.UDFContext;
-import com.geedgenetworks.sketch.hlld.Hll;
-import com.geedgenetworks.sketch.util.StringUtils;
-import org.junit.jupiter.api.Test;
-
-import java.nio.charset.StandardCharsets;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-public class HlldTest {
- AggregateFunction agg;
- Accumulator acc;
- Event event;
-
- @Test
- public void inputRegular() {
- initData(14, "regular", "base64");
- long count = 100000;
- Map<String, Object> fields = event.getExtractedFields();
- for (int i = 0; i < count; i++) {
- fields.put("ip", i);
- agg.add(event, acc);
- }
-
- String hllStr = (String)agg.getResult(acc).getMetricsFields().get("ip_cnt");
- long rst = (long) Hll.fromBytes(StringUtils.decodeBase64(hllStr.getBytes(StandardCharsets.UTF_8))).size();
- double error = Math.abs(rst - count) / (double) count;
- System.out.println(String.format("%d,%d,%.4f", count , rst , error));
- assertTrue(error <= 0.05);
- }
-
- @Test
- public void inputSketch() {
- initData(14, "sketch", "binary");
- long count = 150000;
- Map<String, Object> fields = event.getExtractedFields();
- for (int i = 0; i < 100000; i++) {
- Hll hll = new Hll(12);
- hll.add(i);
- fields.put("ip", StringUtils.encodeBase64String(hll.toBytes()));
- agg.add(event, acc);
- }
- for (int i = 50000; i < 150000; i++) {
- Hll hll = new Hll(13);
- hll.add(i);
- fields.put("ip", hll.toBytes());
- agg.add(event, acc);
- }
-
- byte[] hllBytes = (byte[])agg.getResult(acc).getMetricsFields().get("ip_cnt");
- long rst = (long) Hll.fromBytes(hllBytes).size();
- double error = Math.abs(rst - count) / (double) count;
- System.out.println(String.format("%d,%d,%.4f", count , rst , error));
- assertTrue(error <= 0.05);
- }
-
- private void initData(int precision, String input_type, String output_format){
- agg = new Hlld();
- UDFContext c = new UDFContext();
- Map<String, Object> parameters = new HashMap<>();
- parameters.put("precision", precision);
- parameters.put("input_type", input_type);
- parameters.put("output_format", output_format);
- c.setParameters(parameters);
- c.setLookup_fields(Collections.singletonList("ip"));
- c.setOutput_fields(Collections.singletonList("ip_cnt"));
-
- agg.open(c);
- Map<String, Object> map = new HashMap<>();
- acc = new Accumulator();
- acc.setMetricsFields(map);
- agg.initAccumulator(acc);
-
- event = new Event();
- Map<String, Object> fields = new HashMap<>();
- event.setExtractedFields(fields);
- }
-}
+package com.geedgenetworks.core.udf.udaf.hlld;
+
+import com.geedgenetworks.common.Accumulator;
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.udf.AggregateFunction;
+import com.geedgenetworks.common.udf.UDFContext;
+import com.geedgenetworks.sketch.hlld.Hll;
+import com.geedgenetworks.sketch.util.StringUtils;
+import org.junit.jupiter.api.Test;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class HlldTest {
+ AggregateFunction agg;
+ Accumulator acc;
+ Event event;
+
+ @Test
+ public void inputRegular() {
+ initData(14, "regular", "base64");
+ long count = 100000;
+ Map<String, Object> fields = event.getExtractedFields();
+ for (int i = 0; i < count; i++) {
+ fields.put("ip", i);
+ agg.add(event, acc);
+ }
+
+ String hllStr = (String)agg.getResult(acc).getMetricsFields().get("ip_cnt");
+ long rst = (long) Hll.fromBytes(StringUtils.decodeBase64(hllStr.getBytes(StandardCharsets.UTF_8))).size();
+ double error = Math.abs(rst - count) / (double) count;
+ System.out.println(String.format("%d,%d,%.4f", count , rst , error));
+ assertTrue(error <= 0.05);
+ }
+
+ @Test
+ public void inputSketch() {
+ initData(14, "sketch", "binary");
+ long count = 150000;
+ Map<String, Object> fields = event.getExtractedFields();
+ for (int i = 0; i < 100000; i++) {
+ Hll hll = new Hll(12);
+ hll.add(i);
+ fields.put("ip", StringUtils.encodeBase64String(hll.toBytes()));
+ agg.add(event, acc);
+ }
+ for (int i = 50000; i < 150000; i++) {
+ Hll hll = new Hll(13);
+ hll.add(i);
+ fields.put("ip", hll.toBytes());
+ agg.add(event, acc);
+ }
+
+ byte[] hllBytes = (byte[])agg.getResult(acc).getMetricsFields().get("ip_cnt");
+ long rst = (long) Hll.fromBytes(hllBytes).size();
+ double error = Math.abs(rst - count) / (double) count;
+ System.out.println(String.format("%d,%d,%.4f", count , rst , error));
+ assertTrue(error <= 0.05);
+ }
+
+ private void initData(int precision, String input_type, String output_format){
+ agg = new Hlld();
+ UDFContext c = new UDFContext();
+ Map<String, Object> parameters = new HashMap<>();
+ parameters.put("precision", precision);
+ parameters.put("input_type", input_type);
+ parameters.put("output_format", output_format);
+ c.setParameters(parameters);
+ c.setLookupFields(Collections.singletonList("ip"));
+ c.setOutputFields(Collections.singletonList("ip_cnt"));
+
+ agg.open(c);
+ Map<String, Object> map = new HashMap<>();
+ acc = new Accumulator();
+ acc.setMetricsFields(map);
+ agg.initAccumulator(acc);
+
+ event = new Event();
+ Map<String, Object> fields = new HashMap<>();
+ event.setExtractedFields(fields);
+ }
+}
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/udtf/UnrollTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/udtf/UnrollTest.java
new file mode 100644
index 0000000..3320f38
--- /dev/null
+++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/udtf/UnrollTest.java
@@ -0,0 +1,109 @@
+package com.geedgenetworks.core.udf.udtf;
+
+import com.alibaba.fastjson2.JSON;
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.common.udf.UDFContext;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+
+public class UnrollTest {
+ PathUnroll pathUnroll;
+ Event event;
+
+ @Test
+ public void explodePathWithNoFileField() {
+ init("path", "out_path", ".");
+ Map<String, Object> fields = event.getExtractedFields();
+ fields.put("path", "ETHERNET.IPv4.TCP.ssl");
+ String[] excepted = new String[]{"ETHERNET","ETHERNET.IPv4","ETHERNET.IPv4.TCP","ETHERNET.IPv4.TCP.ssl"};
+ String[] outPaths = pathUnroll.evaluate(event).stream().map(x -> (String)x.getExtractedFields().get("out_path")).toArray(String[]::new);
+ System.out.println(JSON.toJSONString(outPaths));
+ assertArrayEquals(outPaths, excepted);
+ // 忽略结尾的分隔符
+ fields.put("path", "ETHERNET.IPv4.TCP.ssl.");
+ outPaths = pathUnroll.evaluate(event).stream().map(x -> (String)x.getExtractedFields().get("out_path")).toArray(String[]::new);
+ System.out.println(JSON.toJSONString(outPaths));
+ assertArrayEquals(outPaths, excepted);
+ // 空路径不输出
+ fields.put("path", "");
+ outPaths = pathUnroll.evaluate(event).stream().map(x -> (String)x.getExtractedFields().get("out_path")).toArray(String[]::new);
+ System.out.println(JSON.toJSONString(outPaths));
+ assertTrue(outPaths.length == 0);
+
+ init("path", "out_path", "/");
+ fields = event.getExtractedFields();
+ fields.put("path", "ETHERNET/IPv4/TCP/ssl");
+ excepted = new String[]{"ETHERNET","ETHERNET/IPv4","ETHERNET/IPv4/TCP","ETHERNET/IPv4/TCP/ssl"};
+ outPaths = pathUnroll.evaluate(event).stream().map(x -> (String)x.getExtractedFields().get("out_path")).toArray(String[]::new);
+ System.out.println(JSON.toJSONString(outPaths));
+ assertArrayEquals(outPaths, excepted);
+ // 忽略结尾的分隔符
+ fields.put("path", "ETHERNET/IPv4/TCP/ssl/");
+ outPaths = pathUnroll.evaluate(event).stream().map(x -> (String)x.getExtractedFields().get("out_path")).toArray(String[]::new);
+ System.out.println(JSON.toJSONString(outPaths));
+ assertArrayEquals(outPaths, excepted);
+ }
+
+ @Test
+ public void explodePathWithFileField() {
+ init("path", "file", "out_path", "out_file", ".");
+ Map<String, Object> fields = event.getExtractedFields();
+ fields.put("path", "ETHERNET.IPv4.TCP.ssl");
+ fields.put("file", "ssl");
+ String[] excepted = new String[]{"ETHERNET", "ETHERNET.IPv4", "ETHERNET.IPv4.TCP", "ETHERNET.IPv4.TCP.ssl"};
+ String[] exceptedFile = new String[]{null, null, null, "ssl"};
+ String[] outPaths = pathUnroll.evaluate(event).stream().map(x -> (String) x.getExtractedFields().get("out_path")).toArray(String[]::new);
+ String[] outFiles = pathUnroll.evaluate(event).stream().map(x -> (String) x.getExtractedFields().get("out_file")).toArray(String[]::new);
+ System.out.println(JSON.toJSONString(outPaths));
+ System.out.println(JSON.toJSONString(outFiles));
+ assertArrayEquals(outPaths, excepted);
+ assertArrayEquals(outFiles, exceptedFile);
+ // 忽略结尾的分隔符
+ fields.put("path", "ETHERNET.IPv4.TCP.ssl.");
+ fields.put("file", "ssl");
+ outPaths = pathUnroll.evaluate(event).stream().map(x -> (String) x.getExtractedFields().get("out_path")).toArray(String[]::new);
+ outFiles = pathUnroll.evaluate(event).stream().map(x -> (String) x.getExtractedFields().get("out_file")).toArray(String[]::new);
+ System.out.println(JSON.toJSONString(outPaths));
+ System.out.println(JSON.toJSONString(outFiles));
+ assertArrayEquals(outPaths, excepted);
+ assertArrayEquals(outFiles, exceptedFile);
+
+ fields.put("path", "ETHERNET.IPv4.TCP.ssl");
+ fields.put("file", "ssl.aa");
+ excepted = new String[]{"ETHERNET", "ETHERNET.IPv4", "ETHERNET.IPv4.TCP", "ETHERNET.IPv4.TCP.ssl", "ETHERNET.IPv4.TCP.ssl.ssl.aa"};
+ exceptedFile = new String[]{null, null, null, null,"ssl.aa"};
+ outPaths = pathUnroll.evaluate(event).stream().map(x -> (String) x.getExtractedFields().get("out_path")).toArray(String[]::new);
+ outFiles = pathUnroll.evaluate(event).stream().map(x -> (String) x.getExtractedFields().get("out_file")).toArray(String[]::new);
+ System.out.println(JSON.toJSONString(outPaths));
+ System.out.println(JSON.toJSONString(outFiles));
+ assertArrayEquals(outPaths, excepted);
+ assertArrayEquals(outFiles, exceptedFile);
+ }
+
+ private void init(String pathField, String outputPathField, String separator){
+ init(pathField, null, outputPathField, null, separator);
+ }
+
+ private void init(String pathField, String fileField, String outputPathField, String outputFileField, String separator){
+ pathUnroll = new PathUnroll();
+ UDFContext c = new UDFContext();
+ Map<String, Object> parameters = new HashMap<>();
+ parameters.put("separator", separator);
+ c.setParameters(parameters);
+ c.setLookupFields(Arrays.asList(pathField, fileField).stream().filter(x -> x != null).collect(Collectors.toList()));
+ c.setOutputFields(Arrays.asList(outputPathField, outputFileField).stream().filter(x -> x != null).collect(Collectors.toList()));
+
+ pathUnroll.open(null, c);
+ event = new Event();
+ Map<String, Object> fields = new HashMap<>();
+ event.setExtractedFields(fields);
+ }
+} \ No newline at end of file
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/utils/LoadIntervalDataUtilTest.java b/groot-core/src/test/java/com/geedgenetworks/core/utils/LoadIntervalDataUtilTest.java
new file mode 100644
index 0000000..b7c6306
--- /dev/null
+++ b/groot-core/src/test/java/com/geedgenetworks/core/utils/LoadIntervalDataUtilTest.java
@@ -0,0 +1,80 @@
+package com.geedgenetworks.core.utils;
+
+
+import java.sql.Timestamp;
+
+public class LoadIntervalDataUtilTest {
+
+ public static void main(String[] args) throws Exception{
+ //testNoError();
+ //testNotUpdateDataOnStart();
+ //testWithErrorAndNotFail();
+ testWithErrorAndFail();
+ }
+
+ public static void testNoError() throws Exception{
+ LoadIntervalDataUtil<Timestamp> util = LoadIntervalDataUtil.newInstance(() -> new Timestamp(System.currentTimeMillis()),
+ LoadIntervalDataOptions.defaults("time", 3000));
+
+ System.out.println(new Timestamp(System.currentTimeMillis()) + " - " + util.data());
+
+ for (int i = 0; i < 10; i++) {
+ Thread.sleep(1000);
+ System.out.println(new Timestamp(System.currentTimeMillis()) + " - " + util.data());
+ }
+
+ util.stop();
+ }
+
+ public static void testNotUpdateDataOnStart() throws Exception{
+ LoadIntervalDataUtil<Timestamp> util = LoadIntervalDataUtil.newInstance(() -> new Timestamp(System.currentTimeMillis()),
+ LoadIntervalDataOptions.builder().withName("time").withIntervalMs(3000).withUpdateDataOnStart(false).build());
+
+ System.out.println(new Timestamp(System.currentTimeMillis()) + " - " + util.data());
+
+ for (int i = 0; i < 10; i++) {
+ Thread.sleep(1000);
+ System.out.println(new Timestamp(System.currentTimeMillis()) + " - " + util.data());
+ }
+
+ util.stop();
+ }
+
+ public static void testWithErrorAndNotFail() throws Exception{
+ final long start = System.currentTimeMillis();
+ LoadIntervalDataUtil<Timestamp> util = LoadIntervalDataUtil.newInstance(() -> {
+ if(System.currentTimeMillis() - start >= 5000){
+ throw new RuntimeException(new Timestamp(System.currentTimeMillis()).toString());
+ }
+ return new Timestamp(System.currentTimeMillis());
+ }, LoadIntervalDataOptions.defaults("time", 3000));
+
+ System.out.println(new Timestamp(System.currentTimeMillis()) + " - " + util.data());
+
+ for (int i = 0; i < 10; i++) {
+ Thread.sleep(1000);
+ System.out.println(new Timestamp(System.currentTimeMillis()) + " - " + util.data());
+ }
+
+ util.stop();
+ }
+
+ public static void testWithErrorAndFail() throws Exception{
+ final long start = System.currentTimeMillis();
+ LoadIntervalDataUtil<Timestamp> util = LoadIntervalDataUtil.newInstance(() -> {
+ if(System.currentTimeMillis() - start >= 5000){
+ throw new RuntimeException(new Timestamp(System.currentTimeMillis()).toString());
+ }
+ return new Timestamp(System.currentTimeMillis());
+ }, LoadIntervalDataOptions.builder().withName("time").withIntervalMs(3000).withFailOnException(true).build());
+
+ System.out.println(new Timestamp(System.currentTimeMillis()) + " - " + util.data());
+
+ for (int i = 0; i < 10; i++) {
+ Thread.sleep(1000);
+ System.out.println(new Timestamp(System.currentTimeMillis()) + " - " + util.data());
+ }
+
+ util.stop();
+ }
+} \ No newline at end of file
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/utils/SingleValueMapTest.java b/groot-core/src/test/java/com/geedgenetworks/core/utils/SingleValueMapTest.java
new file mode 100644
index 0000000..f5f1e7c
--- /dev/null
+++ b/groot-core/src/test/java/com/geedgenetworks/core/utils/SingleValueMapTest.java
@@ -0,0 +1,98 @@
+package com.geedgenetworks.core.utils;
+
+import org.junit.jupiter.api.Assertions;
+
+import java.sql.Timestamp;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class SingleValueMapTest {
+
+ public static void main(String[] args) throws Exception {
+ //testSingleValue();
+ testSingleValueWithLoadIntervalDataUtil();
+ }
+
+ public static void testSingleValue() throws Exception {
+ Thread[] threads = new Thread[20];
+ for (int i = 0; i < threads.length; i++) {
+ threads[i] = new Thread(() -> {
+ SingleValueMap.Data<ConnDada> connDada = null;
+ try {
+ connDada = SingleValueMap.acquireData("conn_data", () -> new ConnDada(), x -> {
+ System.out.println("close conn");
+ });
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+
+ try {
+ Thread.sleep(ThreadLocalRandom.current().nextInt(5) * 10);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+
+ connDada.release();
+ }, "Thread-" + i);
+ }
+
+ for (int i = 0; i < threads.length; i++) {
+ threads[i].start();
+ }
+
+ for (int i = 0; i < threads.length; i++) {
+ threads[i].join();
+ }
+
+ System.out.println("initCnt:" + ConnDada.initCnt.get());
+ Assertions.assertEquals(ConnDada.initCnt.get(), 1);
+ }
+
+ public static void testSingleValueWithLoadIntervalDataUtil() throws Exception {
+ Thread[] threads = new Thread[20];
+ for (int i = 0; i < threads.length; i++) {
+ threads[i] = new Thread(() -> {
+ SingleValueMap.Data<LoadIntervalDataUtil<Timestamp>> util = null;
+ try {
+ util = SingleValueMap.acquireData("LoadIntervalDataUtil",
+ () -> LoadIntervalDataUtil.newInstance(() -> new Timestamp(System.currentTimeMillis()), LoadIntervalDataOptions.defaults("time", 3000)),
+ LoadIntervalDataUtil::stop);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+
+
+ try {
+ for (int j = 0; j < 10; j++) {
+ Thread.sleep(1000);
+ System.out.println(Thread.currentThread().getName() + " - " + new Timestamp(System.currentTimeMillis()) + " - " + util.getData().data());
+ }
+
+ Thread.sleep(ThreadLocalRandom.current().nextInt(5) * 10);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+
+ util.release();
+ }, "Thread-" + i);
+ }
+
+ for (int i = 0; i < threads.length; i++) {
+ threads[i].start();
+ }
+
+ for (int i = 0; i < threads.length; i++) {
+ threads[i].join();
+ }
+
+ }
+
+ public static class ConnDada {
+ static AtomicInteger initCnt = new AtomicInteger(0);
+ public ConnDada(){
+ System.out.println("ConnDada init");
+ initCnt.incrementAndGet();
+ }
+
+ }
+} \ No newline at end of file
diff --git a/groot-examples/cn-udf-example/pom.xml b/groot-examples/cn-udf-example/pom.xml
index 38ae4ea..4ec1f18 100644
--- a/groot-examples/cn-udf-example/pom.xml
+++ b/groot-examples/cn-udf-example/pom.xml
@@ -9,7 +9,7 @@
<version>${revision}</version>
</parent>
- <artifactId>cn-udf-example</artifactId>
+ <artifactId>cn-scalarFunction-example</artifactId>
<name>Groot : Examples : CN-UDF</name>
<properties>
<maven.install.skip>true</maven.install.skip>
diff --git a/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java b/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java
index 9b58289..5e64962 100644
--- a/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java
+++ b/groot-examples/end-to-end-example/src/main/java/com/geedgenetworks/example/GrootStreamExample.java
@@ -14,7 +14,7 @@ import java.util.List;
public class GrootStreamExample {
public static void main(String[] args) throws FileNotFoundException, URISyntaxException {
- String configPath = args.length > 0 ? args[0] : "/examples/inline_to_print.yaml";
+ String configPath = args.length > 0 ? args[0] : "/examples/inline_to_print_test.yaml";
String configFile = getTestConfigFile(configPath);
ExecuteCommandArgs executeCommandArgs = new ExecuteCommandArgs();
executeCommandArgs.setConfigFile(configFile);
diff --git a/groot-examples/end-to-end-example/src/main/resources/examples/grootstream_job_split_test.yaml b/groot-examples/end-to-end-example/src/main/resources/examples/grootstream_job_split_test.yaml
new file mode 100644
index 0000000..9bb2900
--- /dev/null
+++ b/groot-examples/end-to-end-example/src/main/resources/examples/grootstream_job_split_test.yaml
@@ -0,0 +1,97 @@
+sources:
+ inline_source:
+ type : inline
+ fields: # [array of object] Field List, if not set, all fields(Map<String, Object>) will be output.
+ properties:
+ data: '[{"sessions":1,"mail_attachment_name_charset":"GBK","mail_attachment_name":"aGVsbG8=","packet_capture_file":"test","ssl_sni":"www.google.com","decoded_as":"BASE","ssl_san":"www.google.com","__timestamp":1705568517095,"client_ip":"255.255.255.255","server_ip":"2600:1015:b002::"},{"sessions":1,"decoded_as":"HTTP","log_id": 1, "recv_time":"111", "client_ip":"192.168.0.2","server_ip":"2600:1015:b002::"},{"sessions":1,"decoded_as":"DNS","log_id": 1, "recv_time":"111", "client_ip":"192.168.0.2","server_ip":"2600:1015:b002::"},{"sessions":1,"decoded_as":"DNS","log_id": 1, "recv_time":"111", "client_ip":"192.168.0.2","server_ip":"2600:1015:b002::"}]'
+ interval.per.row: 1s # 可选
+ repeat.count: 1 # 可选
+ format: json
+ json.ignore.parse.errors: false
+sinks:
+ collect_sink:
+ type: collect
+ properties:
+ format: json
+splits:
+ test_split:
+ type: split
+ rules:
+ - name: table_processor
+ expression: event.decoded_as == 'HTTP'
+ - name: pre_etl_processor
+ expression: event.decoded_as == 'DNS'
+
+postprocessing_pipelines:
+ pre_etl_processor: # [object] Processing Pipeline
+ type: com.geedgenetworks.core.processor.projection.ProjectionProcessorImpl
+ remove_fields: [fields,tags]
+ output_fields:
+ functions: # [array of object] Function List
+
+ - function: FLATTEN
+ lookup_fields: [ fields,tags ]
+ output_fields: [ ]
+ parameters:
+ #prefix: ""
+ depth: 3
+ # delimiter: "."
+
+ - function: RENAME
+ lookup_fields: [ '' ]
+ output_fields: [ '' ]
+ filter:
+ parameters:
+ # parent_fields: [tags]
+ # rename_fields:
+ # tags: tags
+ rename_expression: key =string.replace_all(key,'tags.','');key =string.replace_all(key,'fields.','');return key;
+
+
+ - function: UNIX_TIMESTAMP_CONVERTER
+ lookup_fields: [ timestamp_ms ]
+ output_fields: [ recv_time ]
+ parameters:
+ precision: seconds
+ interval: 300
+ #
+
+ aggregate_processor:
+ type: aggregate
+ group_by_fields: [decoded_as]
+ window_type: tumbling_processing_time # tumbling_event_time,sliding_processing_time,sliding_event_time
+ window_size: 5
+ window_timestamp_field: test_time
+ functions:
+ - function: NUMBER_SUM
+ lookup_fields: [ sessions ]
+
+ table_processor:
+ type: table
+ functions:
+ - function: JSON_UNROLL
+ lookup_fields: [ encapsulation ]
+ output_fields: [ new_name ]
+
+application: # [object] Application Configuration
+ env: # [object] Environment Variables
+ name: groot-stream-job # [string] Job Name
+ pipeline:
+ object-reuse: true # [boolean] Object Reuse, default is false
+ topology: # [array of object] Node List. It will be used build data flow for job dag graph.
+ - name: inline_source # [string] Node Name, must be unique. It will be used as the name of the corresponding Flink operator. eg. kafka_source the processor type as SOURCE.
+ parallelism: 1 # [number] Operator-Level Parallelism.
+ downstream: [test_split,collect_sink]
+ - name: test_split
+ parallelism: 1
+ downstream: [ table_processor,pre_etl_processor ]
+ - name: pre_etl_processor
+ parallelism: 1
+ downstream: [ collect_sink ]
+ - name: table_processor
+ parallelism: 1
+ downstream: [ collect_sink ]
+ - name: collect_sink
+ parallelism: 1
+
+
diff --git a/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_test.yaml b/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_test.yaml
index fb51a0e..e0cbb17 100644
--- a/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_test.yaml
+++ b/groot-examples/end-to-end-example/src/main/resources/examples/inline_to_print_test.yaml
@@ -23,10 +23,8 @@ sources:
type: string
- name: device_tag
type: string
- - name: sent_bytes
- type: bigint
- - name: received_bytes
- type: bigint
+ - name: http_host
+ type: string
properties:
data: '{"tcp_rtt_ms":128,"decoded_as":"HTTP", "http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.ct.cn","http_url":"www.ct.cn/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.11.22.22","server_ip":"8.8.8.8","client_port":42751,"server_port":80,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":5575,"sent_pkts":97,"sent_bytes":5892,"received_pkts":250,"received_bytes":333931,"tcp_c2s_ip_fragments":0,"tcp_s2c_ip_fragments":0,"tcp_c2s_rtx_pkts":0,"tcp_c2s_rtx_bytes":0,"tcp_s2c_rtx_pkts":0,"tcp_s2c_rtx_bytes":0,"tcp_c2s_o3_pkts":0,"tcp_s2c_o3_pkts":0,"tcp_c2s_lost_bytes":0,"tcp_s2c_lost_bytes":0,"flags":26418,"flags_identify_info":[100,1,100,60,150,100,1,2],"app_transition":"http.1111.test_1_1","decoded_as":"HTTP","server_fqdn":"www.ct.cn","app":"test_1_1","decoded_path":"ETHERNET.IPv4.TCP.http","fqdn_category_list":[1767],"t_vsys_id":1,"vsys_id":1,"session_id":290538039798223400,"tcp_handshake_latency_ms":41,"client_os_desc":"Windows","server_os_desc":"Linux","data_center":"center-xxg-tsgx","device_group":"group-xxg-tsgx","device_tag":"{\"tags\":[{\"tag\":\"data_center\",\"value\":\"center-xxg-tsgx\"},{\"tag\":\"device_group\",\"value\":\"group-xxg-tsgx\"}]}","device_id":"9800165603247024","sled_ip":"192.168.40.39","dup_traffic_flag":0}'
format: json
@@ -47,12 +45,19 @@ processing_pipelines:
session_record_processor:
type: projection
remove_fields: [device_tag]
- output_fields: [log_id, renamed_client_ip, c2s_bytes]
+ #output_fields: [log_id, device_tag, client_ip, client_geolocation, client_asn, server_domain, server_ip, server_geolocation, server_asn, log_uuid, log_uuid_v7, ip_uuid]
functions:
- function: DROP
lookup_fields: []
output_fields: []
filter: event.client_ip == '192.168.10.100'
+
+ - function: DOMAIN
+ lookup_fields: [ http_host, ssl_sni, quic_sni ]
+ output_fields: [ server_domain ]
+ parameters:
+ option: FIRST_SIGNIFICANT_SUBDOMAIN
+
- function: SNOWFLAKE_ID
lookup_fields: []
output_fields: [log_id]
@@ -87,6 +92,21 @@ processing_pipelines:
kb_name: tsg_ip_location
option: IP_TO_DETAIL
+ - function: GEOIP_LOOKUP
+ lookup_fields: [ server_ip ]
+ output_fields: [ ]
+ parameters:
+ kb_name: tsg_ip_location
+ option: IP_TO_OBJECT
+ geolocation_field_mapping:
+ COUNTRY: server_country
+ PROVINCE: server_super_administrative_area
+ CITY: server_administrative_area
+ LONGITUDE: server_longitude
+ LATITUDE: server_latitude
+ ISP: server_isp
+ ORGANIZATION: server_organization
+
- function: JSON_EXTRACT
lookup_fields: [ device_tag ]
output_fields: [ device_group ]
@@ -101,14 +121,22 @@ processing_pipelines:
output_fields: [ processing_time_str ]
parameters:
precision: milliseconds
+
- function: RENAME
parameters:
rename_fields:
- client_ip: renamed_client_ip
- - function: EVAL
- output_fields: [ c2s_bytes ]
+ device_tag: renamed_device_tag
+
+ - function: UUIDv5
+ lookup_fields: [ client_ip, server_ip ]
+ output_fields: [ ip_uuid ]
parameters:
- value_expression: sent_bytes
+ namespace: NAMESPACE_IP
+ - function: UUIDv7
+ output_fields: [ log_uuid_v7 ]
+ - function: UUID
+ output_fields: [ log_uuid ]
+
sinks:
print_sink:
@@ -120,6 +148,7 @@ application:
env:
name: example-inline-to-print
parallelism: 3
+ kms.type: local
pipeline:
object-reuse: true
topology:
diff --git a/groot-examples/end-to-end-example/src/main/resources/grootstream.yaml b/groot-examples/end-to-end-example/src/main/resources/grootstream.yaml
index 67e1dd6..2c352a2 100644
--- a/groot-examples/end-to-end-example/src/main/resources/grootstream.yaml
+++ b/groot-examples/end-to-end-example/src/main/resources/grootstream.yaml
@@ -10,6 +10,21 @@ grootstream:
fs_path: ./config/dat
files:
- ip_builtin.mmdb
+ kms:
+ local:
+ type: local
+ vault:
+ type: vault
+ url: <vault-url>
+ token: <vault-token>
+ key_path: <vault-key-path>
+
+ ssl:
+ enabled: true
+ cert_file: ./config/ssl/cert.pem
+ key_file: ./config/ssl/key.pem
+ require_client_auth: true
+
properties:
hos.path: http://192.168.44.12:9098/hos
hos.bucket.name.traffic_file: traffic_file_bucket
diff --git a/groot-examples/pom.xml b/groot-examples/pom.xml
index 6184bda..46ccaaa 100644
--- a/groot-examples/pom.xml
+++ b/groot-examples/pom.xml
@@ -127,12 +127,21 @@
</dependency>
<dependency>
+ <groupId>com.fasterxml.uuid</groupId>
+ <artifactId>java-uuid-generator</artifactId>
+ <version>${uuid-generator.version}</version>
+ <scope>${scope}</scope>
+ </dependency>
+
+ <dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils_${scala.version}</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
</dependency>
+
+
</dependencies>
diff --git a/groot-formats/format-csv/pom.xml b/groot-formats/format-csv/pom.xml
new file mode 100644
index 0000000..4940bcf
--- /dev/null
+++ b/groot-formats/format-csv/pom.xml
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>com.geedgenetworks</groupId>
+ <artifactId>groot-formats</artifactId>
+ <version>${revision}</version>
+ </parent>
+
+ <artifactId>format-csv</artifactId>
+ <name>Groot : Formats : Format-Csv </name>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-csv</artifactId>
+ <version>${flink.version}</version>
+ <scope>${flink.scope}</scope>
+ </dependency>
+ </dependencies>
+</project> \ No newline at end of file
diff --git a/groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvEventDeserializationSchema.java b/groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvEventDeserializationSchema.java
new file mode 100644
index 0000000..cae823f
--- /dev/null
+++ b/groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvEventDeserializationSchema.java
@@ -0,0 +1,54 @@
+package com.geedgenetworks.formats.csv;
+
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.core.connector.format.MapDeserialization;
+import com.geedgenetworks.core.types.StructType;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+
+public class CsvEventDeserializationSchema implements DeserializationSchema<Event>, MapDeserialization {
+ private final StructType dataType;
+ private final CsvSchema csvSchema;
+ private final boolean ignoreParseErrors;
+ private final CsvToMapDataConverter converter;
+
+ public CsvEventDeserializationSchema(StructType dataType, CsvSchema csvSchema, boolean ignoreParseErrors) {
+ this.dataType = dataType;
+ this.csvSchema = csvSchema;
+ this.ignoreParseErrors = ignoreParseErrors;
+ this.converter = new CsvToMapDataConverter(dataType, csvSchema, ignoreParseErrors);
+ }
+
+ @Override
+ public Event deserialize(byte[] bytes) throws IOException {
+ Map<String, Object> map = deserializeToMap(bytes);
+ if (map == null) {
+ return null;
+ }
+ Event event = new Event();
+ event.setExtractedFields(map);
+ return event;
+ }
+
+ @Override
+ public Map<String, Object> deserializeToMap(byte[] bytes) throws IOException {
+ String message = new String(bytes, StandardCharsets.UTF_8);
+ return converter.convert(message);
+ }
+
+ @Override
+ public boolean isEndOfStream(Event nextElement) {
+ return false;
+ }
+
+ @Override
+ public TypeInformation<Event> getProducedType() {
+ return null;
+ }
+
+}
diff --git a/groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvEventSerializationSchema.java b/groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvEventSerializationSchema.java
new file mode 100644
index 0000000..1df31bb
--- /dev/null
+++ b/groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvEventSerializationSchema.java
@@ -0,0 +1,27 @@
+package com.geedgenetworks.formats.csv;
+
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.core.types.StructType;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema;
+
+public class CsvEventSerializationSchema implements SerializationSchema<Event> {
+ private final StructType dataType;
+ private final CsvSchema csvSchema;
+ private final CsvSerializer serializer;
+
+ public CsvEventSerializationSchema(StructType dataType, CsvSchema csvSchema) {
+ this.dataType = dataType;
+ this.csvSchema = csvSchema.withLineSeparator("");
+ this.serializer = new CsvSerializer(dataType, this.csvSchema);
+ }
+
+ @Override
+ public byte[] serialize(Event element) {
+ try {
+ return serializer.serialize(element.getExtractedFields());
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+}
diff --git a/groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvFormatFactory.java b/groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvFormatFactory.java
new file mode 100644
index 0000000..7e5db4a
--- /dev/null
+++ b/groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvFormatFactory.java
@@ -0,0 +1,190 @@
+package com.geedgenetworks.formats.csv;
+
+import com.geedgenetworks.core.connector.format.DecodingFormat;
+import com.geedgenetworks.core.connector.format.EncodingFormat;
+import com.geedgenetworks.core.factories.DecodingFormatFactory;
+import com.geedgenetworks.core.factories.EncodingFormatFactory;
+import com.geedgenetworks.core.factories.FactoryUtil;
+import com.geedgenetworks.core.factories.TableFactory;
+import com.geedgenetworks.core.types.*;
+import org.apache.commons.lang3.StringEscapeUtils;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+import static com.geedgenetworks.formats.csv.CsvFormatOptions.*;
+
+public class CsvFormatFactory implements DecodingFormatFactory, EncodingFormatFactory {
+ public static final String IDENTIFIER = "csv";
+
+ @Override
+ public String factoryIdentifier() {
+ return IDENTIFIER;
+ }
+
+ @Override
+ public DecodingFormat createDecodingFormat(TableFactory.Context context, ReadableConfig formatOptions) {
+ FactoryUtil.validateFactoryOptions(this, formatOptions);
+ validateFormatOptions(formatOptions);
+ final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS);
+ return dataType -> {
+ Preconditions.checkNotNull(dataType, "csv format require schema");
+ CsvSchema csvSchema = getCsvSchema(dataType, formatOptions);
+ return new CsvEventDeserializationSchema(dataType, csvSchema, ignoreParseErrors);
+ };
+ }
+
+ @Override
+ public EncodingFormat createEncodingFormat(TableFactory.Context context, ReadableConfig formatOptions) {
+ FactoryUtil.validateFactoryOptions(this, formatOptions);
+ validateFormatOptions(formatOptions);
+ return dataType -> {
+ Preconditions.checkNotNull(dataType, "csv format require schema");
+ CsvSchema csvSchema = getCsvSchema(dataType, formatOptions);
+ return new CsvEventSerializationSchema(dataType, csvSchema);
+ };
+ }
+
+ @Override
+ public Set<ConfigOption<?>> requiredOptions() {
+ return Collections.emptySet();
+ }
+
+ @Override
+ public Set<ConfigOption<?>> optionalOptions() {
+ Set<ConfigOption<?>> options = new HashSet<>();
+ options.add(FIELD_DELIMITER);
+ options.add(DISABLE_QUOTE_CHARACTER);
+ options.add(QUOTE_CHARACTER);
+ options.add(ALLOW_COMMENTS);
+ options.add(IGNORE_PARSE_ERRORS);
+ options.add(ARRAY_ELEMENT_DELIMITER);
+ options.add(ESCAPE_CHARACTER);
+ options.add(NULL_LITERAL);
+ return options;
+ }
+
+ static CsvSchema getCsvSchema(StructType dataType, ReadableConfig options){
+ CsvSchema.Builder builder = convert(dataType).rebuild();
+
+ options.getOptional(FIELD_DELIMITER)
+ .map(delimiter -> StringEscapeUtils.unescapeJava(delimiter).charAt(0))
+ .ifPresent(builder::setColumnSeparator);
+
+ if (options.get(DISABLE_QUOTE_CHARACTER)) {
+ builder.disableQuoteChar();
+ } else {
+ options.getOptional(QUOTE_CHARACTER)
+ .map(quote -> quote.charAt(0))
+ .ifPresent(builder::setQuoteChar);
+ }
+
+ options.getOptional(ARRAY_ELEMENT_DELIMITER).ifPresent(builder::setArrayElementSeparator);
+
+ options.getOptional(ESCAPE_CHARACTER).map(quote -> quote.charAt(0)).ifPresent(builder::setEscapeChar);
+
+ Optional.ofNullable(options.get(NULL_LITERAL)).ifPresent(builder::setNullValue);
+
+ CsvSchema csvSchema = builder.build();
+
+ return csvSchema;
+ }
+
+ public static CsvSchema convert(StructType schema) {
+ CsvSchema.Builder builder = new CsvSchema.Builder();
+ StructType.StructField[] fields = schema.fields;
+ for (int i = 0; i < fields.length; i++) {
+ String fieldName = fields[i].name;
+ DataType dataType = fields[i].dataType;
+ builder.addColumn(new CsvSchema.Column(i, fieldName, convertType(fieldName, dataType)));
+ }
+ return builder.build();
+ }
+
+ private static CsvSchema.ColumnType convertType(String fieldName, DataType dataType) {
+ if (dataType instanceof StringType) {
+ return CsvSchema.ColumnType.STRING;
+ } else if (dataType instanceof IntegerType || dataType instanceof LongType || dataType instanceof FloatType || dataType instanceof DoubleType) {
+ return CsvSchema.ColumnType.NUMBER;
+ } else if (dataType instanceof BooleanType) {
+ return CsvSchema.ColumnType.BOOLEAN;
+ } else if (dataType instanceof ArrayType) {
+ validateNestedField(fieldName, ((ArrayType) dataType).elementType);
+ return CsvSchema.ColumnType.ARRAY;
+ } else if (dataType instanceof StructType) {
+ StructType rowType = (StructType) dataType;
+ for (StructType.StructField field : rowType.fields) {
+ validateNestedField(fieldName, field.dataType);
+ }
+ return CsvSchema.ColumnType.ARRAY;
+ } else {
+ throw new IllegalArgumentException(
+ "Unsupported type '" + dataType + "' for field '" + fieldName + "'.");
+ }
+ }
+
+ private static void validateNestedField(String fieldName, DataType dataType) {
+ if (!(dataType instanceof StringType || dataType instanceof IntegerType || dataType instanceof LongType ||
+ dataType instanceof FloatType || dataType instanceof DoubleType || dataType instanceof BooleanType)) {
+ throw new IllegalArgumentException(
+ "Only simple types are supported in the second level nesting of fields '"
+ + fieldName
+ + "' but was: "
+ + dataType);
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Validation
+ // ------------------------------------------------------------------------
+
+ static void validateFormatOptions(ReadableConfig tableOptions) {
+ final boolean hasQuoteCharacter = tableOptions.getOptional(QUOTE_CHARACTER).isPresent();
+ final boolean isDisabledQuoteCharacter = tableOptions.get(DISABLE_QUOTE_CHARACTER);
+ if (isDisabledQuoteCharacter && hasQuoteCharacter) {
+ throw new ValidationException(
+ "Format cannot define a quote character and disabled quote character at the same time.");
+ }
+ // Validate the option value must be a single char.
+ validateCharacterVal(tableOptions, FIELD_DELIMITER, true);
+ validateCharacterVal(tableOptions, ARRAY_ELEMENT_DELIMITER);
+ validateCharacterVal(tableOptions, QUOTE_CHARACTER);
+ validateCharacterVal(tableOptions, ESCAPE_CHARACTER);
+ }
+
+ /** Validates the option {@code option} value must be a Character. */
+ private static void validateCharacterVal(
+ ReadableConfig tableOptions, ConfigOption<String> option) {
+ validateCharacterVal(tableOptions, option, false);
+ }
+
+ /**
+ * Validates the option {@code option} value must be a Character.
+ *
+ * @param tableOptions the table options
+ * @param option the config option
+ * @param unescape whether to unescape the option value
+ */
+ private static void validateCharacterVal(
+ ReadableConfig tableOptions, ConfigOption<String> option, boolean unescape) {
+ if (tableOptions.getOptional(option).isPresent()) {
+ final String value =
+ unescape
+ ? StringEscapeUtils.unescapeJava(tableOptions.get(option))
+ : tableOptions.get(option);
+ if (value.length() != 1) {
+ throw new ValidationException(
+ String.format(
+ "Option '%s.%s' must be a string with single character, but was: %s",
+ IDENTIFIER, option.key(), tableOptions.get(option)));
+ }
+ }
+ }
+}
diff --git a/groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvFormatOptions.java b/groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvFormatOptions.java
new file mode 100644
index 0000000..6d67c20
--- /dev/null
+++ b/groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvFormatOptions.java
@@ -0,0 +1,58 @@
+package com.geedgenetworks.formats.csv;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+
+public class CsvFormatOptions {
+ public static final ConfigOption<String> FIELD_DELIMITER = ConfigOptions.key("field.delimiter")
+ .stringType()
+ .defaultValue(",")
+ .withDescription("Optional field delimiter character (',' by default)");
+
+ public static final ConfigOption<Boolean> DISABLE_QUOTE_CHARACTER = ConfigOptions.key("disable.quote.character")
+ .booleanType()
+ .defaultValue(false)
+ .withDescription(
+ "Optional flag to disabled quote character for enclosing field values (false by default)\n"
+ + "if true, quote-character can not be set");
+
+ public static final ConfigOption<String> QUOTE_CHARACTER = ConfigOptions.key("quote.character")
+ .stringType()
+ .defaultValue("\"")
+ .withDescription(
+ "Optional quote character for enclosing field values ('\"' by default)");
+
+ public static final ConfigOption<Boolean> ALLOW_COMMENTS = ConfigOptions.key("allow.comments")
+ .booleanType()
+ .defaultValue(false)
+ .withDescription(
+ "Optional flag to ignore comment lines that start with \"#\"\n"
+ + "(disabled by default);\n"
+ + "if enabled, make sure to also ignore parse errors to allow empty rows");
+
+ public static final ConfigOption<Boolean> IGNORE_PARSE_ERRORS = ConfigOptions.key("ignore.parse.errors")
+ .booleanType()
+ .defaultValue(false)
+ .withDescription(
+ "Optional flag to skip fields and rows with parse errors instead of failing;\n"
+ + "fields are set to null in case of errors");
+
+ public static final ConfigOption<String> ARRAY_ELEMENT_DELIMITER = ConfigOptions.key("array.element.delimiter")
+ .stringType()
+ .defaultValue(";")
+ .withDescription(
+ "Optional array element delimiter string for separating\n"
+ + "array and row element values (\";\" by default)");
+
+ public static final ConfigOption<String> ESCAPE_CHARACTER = ConfigOptions.key("escape.character")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("Optional escape character for escaping values (disabled by default)");
+
+ public static final ConfigOption<String> NULL_LITERAL = ConfigOptions.key("null.literal")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("Optional null literal string that is interpreted as a\n"
+ + "null value (disabled by default)");
+
+}
diff --git a/groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvSerializer.java b/groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvSerializer.java
new file mode 100644
index 0000000..170a2b6
--- /dev/null
+++ b/groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvSerializer.java
@@ -0,0 +1,181 @@
+package com.geedgenetworks.formats.csv;
+
+import com.geedgenetworks.core.types.*;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectWriter;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.*;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+public class CsvSerializer implements Serializable {
+ private final StructType dataType;
+ private final CsvSchema csvSchema;
+ private final ValueConverter[] fieldConverters;
+ private final String[] fields;
+ private transient CsvMapper csvMapper;
+ private transient ObjectNode root;
+ private transient ObjectWriter objectWriter;
+
+ public CsvSerializer(StructType dataType, CsvSchema csvSchema) {
+ this.dataType = dataType;
+ this.csvSchema = csvSchema;
+ this.fieldConverters = Arrays.stream(dataType.fields).map(f -> this.makeConverter(f.dataType)).toArray(ValueConverter[]::new);
+ this.fields = Arrays.stream(dataType.fields).map(f -> f.name).toArray(String[]::new);
+ }
+
+ public byte[] serialize(Map<String, Object> data) throws IOException {
+ if (root == null) {
+ csvMapper = new CsvMapper();
+ root = csvMapper.createObjectNode();
+ objectWriter = csvMapper.writer(csvSchema);
+ }
+
+ String field;
+ Object value;
+ for (int i = 0; i < fields.length; i++) {
+ field = fields[i];
+ value = data.get(field);
+ if (value == null) {
+ root.set(field, NullNode.getInstance());
+ }else{
+ root.set(field, fieldConverters[i].convert(value));
+ }
+ }
+
+ return objectWriter.writeValueAsBytes(root);
+ }
+
+ private ValueConverter makeConverter(DataType dataType) {
+ if (dataType instanceof StringType) {
+ return this::convertString;
+ }
+
+ if (dataType instanceof IntegerType) {
+ return this::convertInteger;
+ }
+
+ if (dataType instanceof LongType) {
+ return this::convertLong;
+ }
+
+ if (dataType instanceof FloatType) {
+ return this::convertFloat;
+ }
+
+ if (dataType instanceof DoubleType) {
+ return this::convertDouble;
+ }
+
+ if (dataType instanceof BooleanType) {
+ return this::convertBoolean;
+ }
+
+ if (dataType instanceof StructType) {
+ final ValueConverter[] fieldConverters = Arrays.stream(((StructType) dataType).fields).map(f -> this.makeConverter(f.dataType)).toArray(ValueConverter[]::new);
+ final String[] fields = Arrays.stream(((StructType) dataType).fields).map(f -> f.name).toArray(String[]::new);
+ return obj -> {
+ Map<String, Object> map = (Map<String, Object>) obj;
+ // nested rows use array node container
+ final ArrayNode arrayNode = csvMapper.createArrayNode();
+ String field;
+ Object value;
+ for (int i = 0; i < fields.length; i++) {
+ field = fields[i];
+ value = map.get(field);
+ if (value == null) {
+ arrayNode.add(NullNode.getInstance());
+ }else{
+ arrayNode.add(fieldConverters[i].convert(value));
+ }
+ }
+ return arrayNode;
+ };
+ }
+
+ if (dataType instanceof ArrayType) {
+ final ValueConverter elementConverter = this.makeConverter(((ArrayType) dataType).elementType);
+ return obj -> {
+ List<Object> list = (List<Object>) obj;
+ Object element;
+ ArrayNode arrayNode = csvMapper.createArrayNode();
+ for (int i = 0; i < list.size(); i++) {
+ element = list.get(i);
+ if (element == null) {
+ arrayNode.add(NullNode.getInstance());
+ } else {
+ arrayNode.add(elementConverter.convert(element));
+ }
+ }
+ return arrayNode;
+ };
+ }
+
+ throw new UnsupportedOperationException("unsupported dataType: " + dataType);
+ }
+
+ private JsonNode convertString(Object obj) {
+ return TextNode.valueOf(obj.toString());
+ }
+
+ private JsonNode convertInteger(Object obj) {
+ if(obj instanceof Number){
+ return IntNode.valueOf(((Number) obj).intValue());
+ } else if(obj instanceof String){
+ return IntNode.valueOf(Integer.parseInt((String) obj));
+ } else {
+ throw new IllegalArgumentException(String.format("can not convert %s to int", obj));
+ }
+ }
+
+ private JsonNode convertLong(Object obj) {
+ if(obj instanceof Number){
+ return LongNode.valueOf(((Number) obj).longValue());
+ } else if(obj instanceof String){
+ return LongNode.valueOf(Long.parseLong((String) obj));
+ } else {
+ throw new IllegalArgumentException(String.format("can not convert %s to long", obj));
+ }
+ }
+
+ private JsonNode convertFloat(Object obj) {
+ if(obj instanceof Number){
+ return FloatNode.valueOf(((Number) obj).floatValue());
+ } else if(obj instanceof String){
+ return FloatNode.valueOf(Float.parseFloat((String) obj));
+ } else {
+ throw new IllegalArgumentException(String.format("can not convert %s to float", obj));
+ }
+ }
+
+ private JsonNode convertDouble(Object obj) {
+ if(obj instanceof Number){
+ return DoubleNode.valueOf(((Number) obj).doubleValue());
+ } else if(obj instanceof String){
+ return DoubleNode.valueOf(Double.parseDouble((String) obj));
+ } else {
+ throw new IllegalArgumentException(String.format("can not convert %s to double", obj));
+ }
+ }
+
+ private JsonNode convertBoolean(Object obj) {
+ if(obj instanceof Boolean){
+ return BooleanNode.valueOf((Boolean) obj);
+ } else if(obj instanceof String){
+ return BooleanNode.valueOf(Boolean.parseBoolean((String) obj));
+ } else {
+ throw new IllegalArgumentException(String.format("can not convert %s to double", obj));
+ }
+ }
+
+ @FunctionalInterface
+ public interface ValueConverter extends Serializable {
+ JsonNode convert(Object obj);
+ }
+
+}
diff --git a/groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvToMapDataConverter.java b/groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvToMapDataConverter.java
new file mode 100644
index 0000000..f0d2e79
--- /dev/null
+++ b/groot-formats/format-csv/src/main/java/com/geedgenetworks/formats/csv/CsvToMapDataConverter.java
@@ -0,0 +1,222 @@
+package com.geedgenetworks.formats.csv;
+
+import com.geedgenetworks.core.types.*;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectReader;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.util.*;
+
+public class CsvToMapDataConverter implements Serializable {
+ private static final Logger LOG = LoggerFactory.getLogger(CsvToMapDataConverter.class);
+ private final StructType dataType;
+ private final CsvSchema csvSchema;
+ private final boolean ignoreParseErrors;
+ private final ValueConverter valueConverter;
+ private transient ObjectReader objectReader;
+
+ public CsvToMapDataConverter(StructType dataType, CsvSchema csvSchema, boolean ignoreParseErrors) {
+ this.dataType = dataType;
+ this.csvSchema = csvSchema;
+ this.ignoreParseErrors = ignoreParseErrors;
+ this.valueConverter = createRowConverter(dataType, true);
+ this.objectReader = new CsvMapper().readerFor(JsonNode.class).with(csvSchema);
+ }
+
+ public Map<String, Object> convert(String message) {
+ if (objectReader == null) {
+ this.objectReader = new CsvMapper().readerFor(JsonNode.class).with(csvSchema);
+ }
+ try {
+ final JsonNode root = objectReader.readValue(message);
+ return (Map<String, Object>) valueConverter.convert(root);
+ } catch (Throwable t) {
+ if (ignoreParseErrors) {
+ LOG.error(String.format("CSV Parse Errors:%s", message), t);
+ return null;
+ }
+ throw new UnsupportedOperationException(String.format("CSV Parse Errors:%s", message), t);
+ }
+ }
+
+ private ValueConverter createRowConverter(StructType rowType, boolean isTopLevel) {
+ final ValueConverter[] fieldConverters = Arrays.stream(rowType.fields).map(f -> makeConverter(f.dataType)).toArray(ValueConverter[]::new);
+ final String[] fields = Arrays.stream(rowType.fields).map(f -> f.name).toArray(String[]::new);
+ final int arity = fields.length;
+ return node -> {
+ int nodeSize = node.size();
+
+ if (nodeSize != 0) {
+ validateArity(arity, nodeSize, ignoreParseErrors);
+ } else {
+ return null;
+ }
+
+ Map<String, Object> obj = new HashMap<>();
+ Object value;
+ for (int i = 0; i < arity; i++) {
+ JsonNode field;
+ // Jackson only supports mapping by name in the first level
+ if (isTopLevel) {
+ field = node.get(fields[i]);
+ } else {
+ field = node.get(i);
+ }
+ if (field != null && !field.isNull()) {
+ value = fieldConverters[i].convert(field);
+ if (value != null) {
+ obj.put(fields[i], value);
+ }
+ }
+ }
+ return obj;
+ };
+ }
+
+ private ValueConverter createArrayConverter(ArrayType arrayType) {
+ final ValueConverter converter = makeConverter(arrayType.elementType);
+ return node -> {
+ final ArrayNode arrayNode = (ArrayNode) node;
+ if (arrayNode.size() == 0) {
+ return null;
+ }
+ List<Object> objs = new ArrayList<>(arrayNode.size());
+ for (int i = 0; i < arrayNode.size(); i++) {
+ final JsonNode innerNode = arrayNode.get(i);
+ if (innerNode == null || innerNode.isNull()) {
+ objs.add(null);
+ }else{
+ objs.add(converter.convert(innerNode));
+ }
+ }
+ return objs;
+ };
+ }
+
+ private ValueConverter makeConverter(DataType dataType) {
+ if (dataType instanceof StringType) {
+ return this::convertToString;
+ }
+
+ if (dataType instanceof IntegerType) {
+ return this::convertToInteger;
+ }
+
+ if (dataType instanceof LongType) {
+ return this::convertToLong;
+ }
+
+ if (dataType instanceof FloatType) {
+ return this::convertToFloat;
+ }
+
+ if (dataType instanceof DoubleType) {
+ return this::convertToDouble;
+ }
+
+ if (dataType instanceof BooleanType) {
+ return this::convertToBoolean;
+ }
+
+ if (dataType instanceof StructType) {
+ return createRowConverter((StructType) dataType, false);
+ }
+
+ if (dataType instanceof ArrayType) {
+ return createArrayConverter((ArrayType) dataType);
+ }
+
+ throw new UnsupportedOperationException("unsupported dataType: " + dataType);
+ }
+
+ private String convertToString(JsonNode node) {
+ return node.asText();
+ }
+
+ private Integer convertToInteger(JsonNode node) {
+ if (node.canConvertToInt()) {
+ // avoid redundant toString and parseInt, for better performance
+ return node.asInt();
+ } else {
+ String text = node.asText().trim();
+ if (StringUtils.isBlank(text)) {
+ return null;
+ }
+ return Integer.parseInt(text);
+ }
+ }
+
+ private Long convertToLong(JsonNode node) {
+ if (node.canConvertToLong()) {
+ // avoid redundant toString and parseLong, for better performance
+ return node.asLong();
+ } else {
+ String text = node.asText().trim();
+ if (StringUtils.isBlank(text)) {
+ return null;
+ }
+ return Long.parseLong(text);
+ }
+ }
+
+ private Float convertToFloat(JsonNode node) {
+ if (node.isDouble()) {
+ // avoid redundant toString and parseDouble, for better performance
+ return (float) node.asDouble();
+ } else {
+ String text = node.asText().trim();
+ if (StringUtils.isBlank(text)) {
+ return null;
+ }
+ return Float.parseFloat(text);
+ }
+ }
+
+ private Double convertToDouble(JsonNode node) {
+ if (node.isDouble()) {
+ // avoid redundant toString and parseDouble, for better performance
+ return node.asDouble();
+ } else {
+ String text = node.asText().trim();
+ if (StringUtils.isBlank(text)) {
+ return null;
+ }
+ return Double.parseDouble(text);
+ }
+ }
+
+ private Boolean convertToBoolean(JsonNode node) {
+ if (node.isBoolean()) {
+ // avoid redundant toString and parseBoolean, for better performance
+ return node.asBoolean();
+ } else {
+ String text = node.asText().trim();
+ if (StringUtils.isBlank(text)) {
+ return null;
+ }
+ return Boolean.parseBoolean(text);
+ }
+ }
+
+ private static void validateArity(int expected, int actual, boolean ignoreParseErrors) {
+ if (expected > actual && !ignoreParseErrors) {
+ throw new RuntimeException(
+ "Row length mismatch. "
+ + expected
+ + " fields expected but was "
+ + actual
+ + ".");
+ }
+ }
+
+ @FunctionalInterface
+ public interface ValueConverter extends Serializable {
+ Object convert(JsonNode node) throws Exception;
+ }
+}
diff --git a/groot-formats/format-csv/src/main/resources/META-INF/services/com.geedgenetworks.core.factories.Factory b/groot-formats/format-csv/src/main/resources/META-INF/services/com.geedgenetworks.core.factories.Factory
new file mode 100644
index 0000000..e417fa4
--- /dev/null
+++ b/groot-formats/format-csv/src/main/resources/META-INF/services/com.geedgenetworks.core.factories.Factory
@@ -0,0 +1 @@
+com.geedgenetworks.formats.csv.CsvFormatFactory
diff --git a/groot-formats/format-csv/src/test/java/com/geedgenetworks/formats/csv/CsvEventSerDeSchemaTest.java b/groot-formats/format-csv/src/test/java/com/geedgenetworks/formats/csv/CsvEventSerDeSchemaTest.java
new file mode 100644
index 0000000..5142646
--- /dev/null
+++ b/groot-formats/format-csv/src/test/java/com/geedgenetworks/formats/csv/CsvEventSerDeSchemaTest.java
@@ -0,0 +1,219 @@
+package com.geedgenetworks.formats.csv;
+
+import com.geedgenetworks.common.Event;
+import com.geedgenetworks.core.connector.format.MapDeserialization;
+import com.geedgenetworks.core.connector.schema.Schema;
+import com.geedgenetworks.core.factories.DecodingFormatFactory;
+import com.geedgenetworks.core.factories.EncodingFormatFactory;
+import com.geedgenetworks.core.factories.FactoryUtil;
+import com.geedgenetworks.core.factories.TableFactory;
+import com.geedgenetworks.core.types.StructType;
+import com.geedgenetworks.core.types.Types;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.configuration.Configuration;
+import org.junit.jupiter.api.Test;
+
+import java.nio.charset.StandardCharsets;
+import java.util.*;
+
+public class CsvEventSerDeSchemaTest {
+
+ @Test
+ public void testSimpleSerializeDeserialize() throws Exception {
+ StructType dataType = Types.parseStructType("int:int,bigint:bigint,double:double,string:string");
+ Map<String, String> options = new HashMap<>();
+ TableFactory.Context context = new TableFactory.Context(Schema.newSchema(dataType), options, Configuration.fromMap(options));
+
+ // 获取deserialization和serialization
+ DeserializationSchema<Event> deserialization = FactoryUtil.discoverDecodingFormatFactory(DecodingFormatFactory.class, "csv")
+ .createDecodingFormat(context, context.getConfiguration()).createRuntimeDecoder(dataType);
+ SerializationSchema<Event> serialization = FactoryUtil.discoverEncodingFormatFactory(EncodingFormatFactory.class, "csv")
+ .createEncodingFormat(context, context.getConfiguration()).createRuntimeEncoder(dataType);
+
+ deserialization.open(null);
+ serialization.open(null);
+
+ Map<String, Object> map = new HashMap<>();
+ map.put("int", 1);
+ map.put("bigint", "2");
+ map.put("double", "10.2");
+ map.put("string", "utf-8字符串");
+ Event row = new Event();
+ row.setExtractedFields(map);
+
+ byte[] bytes = serialization.serialize(row);
+ System.out.println(map);
+ System.out.println(new String(bytes, StandardCharsets.UTF_8));
+ Map<String, Object> rst = deserialization.deserialize(bytes).getExtractedFields();
+ System.out.println(rst);
+
+ // 反序列成map
+ if(deserialization instanceof MapDeserialization){
+ MapDeserialization mapDeserialization = (MapDeserialization) deserialization;
+ Map<String, Object> rstMap = mapDeserialization.deserializeToMap(bytes);
+ System.out.println(rstMap);
+ }
+ }
+
+ @Test
+ public void testSerializeDeserialize() throws Exception {
+ StructType dataType = Types.parseStructType("int:int,bigint:bigint,double:double,string:string,int_array:array<int>,struct:struct<int:int,string:string>");
+ Map<String, String> options = new HashMap<>();
+ TableFactory.Context context = new TableFactory.Context(Schema.newSchema(dataType), options, Configuration.fromMap(options));
+
+ DeserializationSchema<Event> deserialization = FactoryUtil.discoverDecodingFormatFactory(DecodingFormatFactory.class, "csv")
+ .createDecodingFormat(context, context.getConfiguration()).createRuntimeDecoder(dataType);
+ SerializationSchema<Event> serialization = FactoryUtil.discoverEncodingFormatFactory(EncodingFormatFactory.class, "csv")
+ .createEncodingFormat(context, context.getConfiguration()).createRuntimeEncoder(dataType);
+
+ deserialization.open(null);
+ serialization.open(null);
+
+ Map<String, Object> map = new HashMap<>();
+ map.put("int", 1);
+ map.put("bigint", "2");
+ map.put("double", "10.2");
+ map.put("string", "utf-8字符串");
+ map.put("int_array", Arrays.asList(1 , "2", 3));
+ map.put("struct", Map.of("int", "1", "string", 22));
+ Event row = new Event();
+ row.setExtractedFields(map);
+
+ byte[] bytes = serialization.serialize(row);
+ System.out.println(map);
+ System.out.println(new String(bytes, StandardCharsets.UTF_8));
+ Map<String, Object> rst = deserialization.deserialize(bytes).getExtractedFields();
+ System.out.println(rst);
+
+ System.out.println(StringUtils.repeat('*', 60));
+
+ map = new HashMap<>();
+ row = new Event();
+ map.put("int", 1);
+ map.put("double", "10.2");
+ map.put("int_array", Arrays.asList(1 , null, null));
+ map.put("struct", Map.of( "string", 22));
+ row.setExtractedFields(map);
+
+ bytes = serialization.serialize(row);
+ System.out.println(map);
+ System.out.println(new String(bytes, StandardCharsets.UTF_8));
+ rst = deserialization.deserialize(bytes).getExtractedFields();
+ System.out.println(rst);
+
+
+ System.out.println(StringUtils.repeat('*', 60));
+
+ map = new HashMap<>();
+ row = new Event();
+ row.setExtractedFields(map);
+
+ bytes = serialization.serialize(row);
+ System.out.println(map);
+ System.out.println(new String(bytes, StandardCharsets.UTF_8));
+ rst = deserialization.deserialize(bytes).getExtractedFields();
+ System.out.println(rst);
+
+ System.out.println(StringUtils.repeat('*', 60));
+
+ map = new HashMap<>();
+ row = new Event();
+ map.put("int", 1);
+ map.put("bigint", "2");
+ map.put("double", "10.2");
+ map.put("string", "utf-8字符串");
+ map.put("int_array", List.of(1 , "2", 3));
+ map.put("struct", Map.of("int", "1", "string", 22));
+ row.setExtractedFields(map);
+
+ bytes = serialization.serialize(row);
+ System.out.println(map);
+ System.out.println(new String(bytes, StandardCharsets.UTF_8));
+ rst = deserialization.deserialize(bytes).getExtractedFields();
+ System.out.println(rst);
+ }
+
+
+ @Test
+ public void testNullableFieldSerializeDeserialize() throws Exception {
+ StructType dataType = Types.parseStructType("int:int,bigint:bigint,double:double,string:string,int_array:array<int>,struct:struct<int:int,string:string>");
+ Map<String, String> options = new HashMap<>();
+ options.put(CsvFormatOptions.NULL_LITERAL.key(), "null");
+ options.put(CsvFormatOptions.IGNORE_PARSE_ERRORS.key(), "true");
+ TableFactory.Context context = new TableFactory.Context(Schema.newSchema(dataType), options, Configuration.fromMap(options));
+
+ DeserializationSchema<Event> deserialization = FactoryUtil.discoverDecodingFormatFactory(DecodingFormatFactory.class, "csv")
+ .createDecodingFormat(context, context.getConfiguration()).createRuntimeDecoder(dataType);
+ SerializationSchema<Event> serialization = FactoryUtil.discoverEncodingFormatFactory(EncodingFormatFactory.class, "csv")
+ .createEncodingFormat(context, context.getConfiguration()).createRuntimeEncoder(dataType);
+
+ deserialization.open(null);
+ serialization.open(null);
+
+ Map<String, Object> map = new HashMap<>();
+ map.put("int", 1);
+ map.put("bigint", "2");
+ map.put("double", "10.2");
+ map.put("string", "utf-8字符串");
+ map.put("int_array", Arrays.asList(1 , "2", 3));
+ map.put("struct", Map.of("int", "1", "string", 22));
+ Event row = new Event();
+ row.setExtractedFields(map);
+
+ byte[] bytes = serialization.serialize(row);
+ System.out.println(map);
+ System.out.println(new String(bytes, StandardCharsets.UTF_8));
+ Map<String, Object> rst = deserialization.deserialize(bytes).getExtractedFields();
+ System.out.println(rst);
+
+ System.out.println(StringUtils.repeat('*', 60));
+
+ map = new HashMap<>();
+ row = new Event();
+ map.put("int", 1);
+ map.put("double", "10.2");
+ map.put("int_array", Arrays.asList(1 , null, null));
+ map.put("struct", Map.of( "string", 22));
+ row.setExtractedFields(map);
+
+ bytes = serialization.serialize(row);
+ System.out.println(map);
+ System.out.println(new String(bytes, StandardCharsets.UTF_8));
+ rst = deserialization.deserialize(bytes).getExtractedFields();
+ System.out.println(rst);
+
+
+ System.out.println(StringUtils.repeat('*', 60));
+
+ map = new HashMap<>();
+ row = new Event();
+ row.setExtractedFields(map);
+
+ bytes = serialization.serialize(row);
+ System.out.println(map);
+ System.out.println(new String(bytes, StandardCharsets.UTF_8));
+ rst = deserialization.deserialize(bytes).getExtractedFields();
+ System.out.println(rst);
+
+ System.out.println(StringUtils.repeat('*', 60));
+
+ map = new HashMap<>();
+ row = new Event();
+ map.put("int", 1);
+ map.put("bigint", "2");
+ map.put("double", "10.2");
+ map.put("string", "utf-8字符串");
+ map.put("int_array", List.of(1 , "2", 3));
+ map.put("struct", Map.of("int", "1", "string", 22));
+ row.setExtractedFields(map);
+
+ bytes = serialization.serialize(row);
+ System.out.println(map);
+ System.out.println(new String(bytes, StandardCharsets.UTF_8));
+ rst = deserialization.deserialize(bytes).getExtractedFields();
+ System.out.println(rst);
+ }
+
+} \ No newline at end of file
diff --git a/groot-formats/format-json/src/main/java/com/geedgenetworks/formats/json/JsonEventDeserializationSchema.java b/groot-formats/format-json/src/main/java/com/geedgenetworks/formats/json/JsonEventDeserializationSchema.java
index 89d68e0..2f7c352 100644
--- a/groot-formats/format-json/src/main/java/com/geedgenetworks/formats/json/JsonEventDeserializationSchema.java
+++ b/groot-formats/format-json/src/main/java/com/geedgenetworks/formats/json/JsonEventDeserializationSchema.java
@@ -2,6 +2,7 @@ package com.geedgenetworks.formats.json;
import com.alibaba.fastjson2.JSON;
import com.geedgenetworks.common.Event;
+import com.geedgenetworks.core.connector.format.MapDeserialization;
import com.geedgenetworks.core.types.StructType;
import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -12,7 +13,7 @@ import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Map;
-public class JsonEventDeserializationSchema implements DeserializationSchema<Event> {
+public class JsonEventDeserializationSchema implements DeserializationSchema<Event>, MapDeserialization {
private static final Logger LOG = LoggerFactory.getLogger(JsonEventDeserializationSchema.class);
private static final int MAX_CHARS_LENGTH = 1024 * 32;
private final StructType dataType;
@@ -28,6 +29,18 @@ public class JsonEventDeserializationSchema implements DeserializationSchema<Eve
@Override
public Event deserialize(byte[] bytes) throws IOException {
+ Map<String, Object> map = deserializeToMap(bytes);
+ if (map == null) {
+ return null;
+ }
+
+ Event event = new Event();
+ event.setExtractedFields(map);
+ return event;
+ }
+
+ @Override
+ public Map<String, Object> deserializeToMap(byte[] bytes) throws IOException {
Map<String, Object> map;
String message = decodeUTF8(bytes, 0, bytes.length);
@@ -50,9 +63,7 @@ public class JsonEventDeserializationSchema implements DeserializationSchema<Eve
map = converter.convert(message);
}
- Event event = new Event();
- event.setExtractedFields(map);
- return event;
+ return map;
}
private String decodeUTF8(byte[] input, int offset, int byteLen) {
diff --git a/groot-formats/format-msgpack/src/main/java/com/geedgenetworks/formats/msgpack/MessagePackEventDeserializationSchema.java b/groot-formats/format-msgpack/src/main/java/com/geedgenetworks/formats/msgpack/MessagePackEventDeserializationSchema.java
index c7783b7..2fc9c64 100644
--- a/groot-formats/format-msgpack/src/main/java/com/geedgenetworks/formats/msgpack/MessagePackEventDeserializationSchema.java
+++ b/groot-formats/format-msgpack/src/main/java/com/geedgenetworks/formats/msgpack/MessagePackEventDeserializationSchema.java
@@ -1,6 +1,7 @@
package com.geedgenetworks.formats.msgpack;
import com.geedgenetworks.common.Event;
+import com.geedgenetworks.core.connector.format.MapDeserialization;
import com.geedgenetworks.core.types.StructType;
import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -9,7 +10,7 @@ import org.apache.flink.util.StringUtils;
import java.io.IOException;
import java.util.Map;
-public class MessagePackEventDeserializationSchema implements DeserializationSchema<Event> {
+public class MessagePackEventDeserializationSchema implements DeserializationSchema<Event>, MapDeserialization {
private final StructType dataType;
private final MessagePackDeserializer deserializer;
@@ -31,6 +32,15 @@ public class MessagePackEventDeserializationSchema implements DeserializationSch
}
@Override
+ public Map<String, Object> deserializeToMap(byte[] bytes) throws IOException {
+ try {
+ return deserializer.deserialize(bytes);
+ } catch (Exception e) {
+ throw new IOException(StringUtils.byteToHexString(bytes), e);
+ }
+ }
+
+ @Override
public boolean isEndOfStream(Event nextElement) {
return false;
}
diff --git a/groot-formats/format-protobuf/src/main/java/com/geedgenetworks/formats/protobuf/ProtobufEventDeserializationSchema.java b/groot-formats/format-protobuf/src/main/java/com/geedgenetworks/formats/protobuf/ProtobufEventDeserializationSchema.java
index c599445..0e477a1 100644
--- a/groot-formats/format-protobuf/src/main/java/com/geedgenetworks/formats/protobuf/ProtobufEventDeserializationSchema.java
+++ b/groot-formats/format-protobuf/src/main/java/com/geedgenetworks/formats/protobuf/ProtobufEventDeserializationSchema.java
@@ -1,6 +1,7 @@
package com.geedgenetworks.formats.protobuf;
import com.geedgenetworks.common.Event;
+import com.geedgenetworks.core.connector.format.MapDeserialization;
import com.geedgenetworks.core.types.StructType;
import com.geedgenetworks.shaded.com.google.protobuf.Descriptors.Descriptor;
import org.apache.flink.api.common.serialization.DeserializationSchema;
@@ -12,7 +13,7 @@ import java.io.IOException;
import java.util.Base64;
import java.util.Map;
-public class ProtobufEventDeserializationSchema implements DeserializationSchema<Event> {
+public class ProtobufEventDeserializationSchema implements DeserializationSchema<Event>, MapDeserialization {
private static final Logger LOG = LoggerFactory.getLogger(ProtobufEventDeserializationSchema.class);
private final String messageName;
private final byte[] binaryFileDescriptorSet;
@@ -37,10 +38,6 @@ public class ProtobufEventDeserializationSchema implements DeserializationSchema
@Override
public Event deserialize(byte[] message) throws IOException {
- if(message == null){
- return null;
- }
-
try {
Map<String, Object> map = converter.converter(message);
Event event = new Event();
@@ -57,6 +54,20 @@ public class ProtobufEventDeserializationSchema implements DeserializationSchema
}
@Override
+ public Map<String, Object> deserializeToMap(byte[] message) throws IOException {
+ try {
+ return converter.converter(message);
+ } catch (Exception e) {
+ if(ignoreParseErrors){
+ LOG.error(String.format("proto解析失败for:%s", Base64.getEncoder().encodeToString(message)), e);
+ return null;
+ }else{
+ throw new IOException(String.format("proto解析失败for:%s", Base64.getEncoder().encodeToString(message)), e);
+ }
+ }
+ }
+
+ @Override
public boolean isEndOfStream(Event nextElement) {
return false;
}
diff --git a/groot-formats/pom.xml b/groot-formats/pom.xml
index 7a6295e..31f15a1 100644
--- a/groot-formats/pom.xml
+++ b/groot-formats/pom.xml
@@ -17,6 +17,7 @@
<module>format-protobuf</module>
<module>format-msgpack</module>
<module>format-raw</module>
+ <module>format-csv</module>
</modules>
<dependencies>
diff --git a/groot-release/pom.xml b/groot-release/pom.xml
index 8a78efa..8cdf1c9 100644
--- a/groot-release/pom.xml
+++ b/groot-release/pom.xml
@@ -121,13 +121,13 @@
<scope>provided</scope>
</dependency>
- <!--Hbase Jars -->
<dependency>
<groupId>com.geedgenetworks</groupId>
- <artifactId>hbase-client-shaded</artifactId>
+ <artifactId>connector-starrocks</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
+
<!--Format Json -->
<dependency>
<groupId>com.geedgenetworks</groupId>
@@ -149,6 +149,12 @@
</dependency>
<dependency>
<groupId>com.geedgenetworks</groupId>
+ <artifactId>format-csv</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>com.geedgenetworks</groupId>
<artifactId>format-raw</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
diff --git a/groot-release/src/main/assembly/assembly-bin-ci.xml b/groot-release/src/main/assembly/assembly-bin-ci.xml
index 4402809..fac9023 100644
--- a/groot-release/src/main/assembly/assembly-bin-ci.xml
+++ b/groot-release/src/main/assembly/assembly-bin-ci.xml
@@ -134,10 +134,10 @@
<useTransitiveDependencies>true</useTransitiveDependencies>
<unpack>false</unpack>
<includes>
- <include>com.geedgenetworks:hbase-client-shaded:jar</include>
<include>com.geedgenetworks:format-json:jar</include>
<include>com.geedgenetworks:format-protobuf:jar</include>
<include>com.geedgenetworks:format-msgpack:jar</include>
+ <include>com.geedgenetworks:format-csv:jar</include>
<include>com.geedgenetworks:format-raw:jar</include>
</includes>
<outputFileNameMapping>${artifact.file.name}</outputFileNameMapping>
diff --git a/groot-tests/test-common/src/test/java/com/geedgenetworks/test/common/container/AbstractTestFlinkContainer.java b/groot-tests/test-common/src/test/java/com/geedgenetworks/test/common/container/AbstractTestFlinkContainer.java
index b833115..4ac3d03 100644
--- a/groot-tests/test-common/src/test/java/com/geedgenetworks/test/common/container/AbstractTestFlinkContainer.java
+++ b/groot-tests/test-common/src/test/java/com/geedgenetworks/test/common/container/AbstractTestFlinkContainer.java
@@ -81,6 +81,10 @@ public abstract class AbstractTestFlinkContainer extends AbstractTestContainer {
".*Successful registration at resource manager.*")
.withStartupTimeout(Duration.ofMinutes(2)));
+ // Copy groot-stream bootstrap and some other files to the container
+ copyGrootStreamStarterToContainer(taskManager);
+ copyGrootStreamStarterLoggingToContainer(taskManager);
+
Startables.deepStart(Stream.of(jobManager)).join();
Startables.deepStart(Stream.of(taskManager)).join();
// execute extra commands
diff --git a/groot-tests/test-common/src/test/resources/grootstream.yaml b/groot-tests/test-common/src/test/resources/grootstream.yaml
index 2eb105b..0def444 100644
--- a/groot-tests/test-common/src/test/resources/grootstream.yaml
+++ b/groot-tests/test-common/src/test/resources/grootstream.yaml
@@ -11,11 +11,4 @@ grootstream:
files:
- ip_builtin.mmdb
properties:
- hos.path: http://192.168.44.12:9098/hos
- hos.bucket.name.traffic_file: traffic_file_bucket
- hos.bucket.name.troubleshooting_file: troubleshooting_file_bucket
scheduler.knowledge_base.update.interval.minutes: 5
- hos.bucket.name.rtp_file: traffic_rtp_file_bucket
- hos.bucket.name.http_file: traffic_http_file_bucket
- hos.bucket.name.eml_file: traffic_eml_file_bucket
- hos.bucket.name.policy_capture_file: traffic_policy_capture_file_bucket
diff --git a/groot-tests/test-e2e-base/src/test/java/com/geedgenetworks/test/e2e/base/InlineToPrintIT.java b/groot-tests/test-e2e-base/src/test/java/com/geedgenetworks/test/e2e/base/InlineToPrintIT.java
index 1c1e777..fdba36f 100644
--- a/groot-tests/test-e2e-base/src/test/java/com/geedgenetworks/test/e2e/base/InlineToPrintIT.java
+++ b/groot-tests/test-e2e-base/src/test/java/com/geedgenetworks/test/e2e/base/InlineToPrintIT.java
@@ -27,30 +27,31 @@ import static org.awaitility.Awaitility.await;
disabledReason = "Only flink adjusts the parameter configuration rules")
public class InlineToPrintIT extends TestSuiteBase {
+
@TestTemplate
- public void testInlineToPrint(AbstractTestFlinkContainer container) throws IOException, InterruptedException {
+ public void testJobExecution(AbstractTestFlinkContainer container) throws IOException, InterruptedException {
CompletableFuture.supplyAsync(
() -> {
try {
- List<String> variables = List.of(
- "hos.bucket.name.rtp_file=cli_job_level_traffic_rtp_file_bucket",
- "hos.bucket.name.http_file=cli_job_level_traffic_http_file_bucket");
- return container.executeJob("/inline_to_print.yaml", variables);
+ return container.executeJob("/inline_to_print.yaml");
} catch (Exception e) {
- log.error("Commit task exception :" + e.getMessage());
throw new RuntimeException(e);
}
});
AtomicReference<String> taskMangerID = new AtomicReference<>();
+
await().atMost(300000, TimeUnit.MILLISECONDS)
.untilAsserted(
() -> {
Map<String, Object> taskMangerInfo = JSON.parseObject(container.executeJobManagerInnerCommand(
"curl http://localhost:8081/taskmanagers"), new TypeReference<Map<String, Object>>() {
});
+
+ @SuppressWarnings("unchecked")
List<Map<String, Object>> taskManagers =
(List<Map<String, Object>>) taskMangerInfo.get("taskmanagers");
+
if (!CollectionUtils.isEmpty(taskManagers)) {
taskMangerID.set(taskManagers.get(0).get("id").toString());
}
@@ -64,6 +65,7 @@ public class InlineToPrintIT extends TestSuiteBase {
Map<String, Object> jobInfo = JSON.parseObject(container.executeJobManagerInnerCommand(
"curl http://localhost:8081/jobs/overview"), new TypeReference<Map<String, Object>>() {
});
+ @SuppressWarnings("unchecked")
List<Map<String, Object>> jobs =
(List<Map<String, Object>>) jobInfo.get("jobs");
if (!CollectionUtils.isEmpty(jobs)) {
@@ -71,6 +73,7 @@ public class InlineToPrintIT extends TestSuiteBase {
}
Assertions.assertNotNull(jobId.get());
});
+
//Obtain job metrics
AtomicReference<List<Map<String, Object>>> jobNumRestartsReference = new AtomicReference<>();
await().atMost(60000, TimeUnit.MILLISECONDS)
@@ -78,8 +81,8 @@ public class InlineToPrintIT extends TestSuiteBase {
() -> {
Thread.sleep(5000);
String result = container.executeJobManagerInnerCommand(
- String.format(
- "curl http://localhost:8081/jobs/%s/metrics?get=numRestarts", jobId.get()));
+ String.format(
+ "curl http://localhost:8081/jobs/%s/metrics?get=numRestarts", jobId.get()));
List<Map<String, Object>> jobNumRestartsInfo = JSON.parseObject(result, new TypeReference<List<Map<String, Object>>>() {
});
if (!CollectionUtils.isEmpty(jobNumRestartsInfo)) {
@@ -90,12 +93,57 @@ public class InlineToPrintIT extends TestSuiteBase {
});
+
+ }
+
+ @TestTemplate
+ public void testUserDefinedJobVariables(AbstractTestFlinkContainer container) throws IOException, InterruptedException {
+
+ CompletableFuture.supplyAsync(
+ () -> {
+ try {
+ List<String> variables = List.of(
+ "hos.bucket.name.rtp_file=cli_job_level_traffic_rtp_file_bucket",
+ "hos.bucket.name.http_file=cli_job_level_traffic_http_file_bucket");
+ return container.executeJob("/inline_to_print.yaml", variables);
+ } catch (Exception e) {
+ log.error("Commit task exception : {} ", e.getMessage());
+ throw new RuntimeException(e);
+ }
+ });
+
+
await().atMost(300000, TimeUnit.MILLISECONDS)
.untilAsserted(
() -> {
String logs = container.getServerLogs();
Assertions.assertTrue(StringUtils.countMatches(logs, "cli_job_level_traffic_rtp_file_bucket/test_pcap_file") > 10);
Assertions.assertTrue(StringUtils.countMatches(logs, "cli_job_level_traffic_http_file_bucket/test_http_req_file") > 10);
+ // Test server_ip filter -> output logs not contains 4.4.4.4 of server_ip
+ Assertions.assertTrue(StringUtils.containsIgnoreCase(logs, "PrintSinkFunction ") && !StringUtils.contains(logs, "\"server_ip\":\"4.4.4.4\""));
+ // Test Drop function -> output logs not contains 5.5.5.5 of server_ip
+ Assertions.assertTrue(StringUtils.containsIgnoreCase(logs, "PrintSinkFunction ") && !StringUtils.contains(logs, "\"server_ip\":\"5.5.5.5\""));
+
+ // Output logs contains server_asn
+ Assertions.assertTrue(StringUtils.containsIgnoreCase(logs, "PrintSinkFunction ") && StringUtils.contains(logs, "\"server_asn\""));
+ // Output logs contains server_domain
+ Assertions.assertTrue(StringUtils.containsIgnoreCase(logs, "PrintSinkFunction ") && StringUtils.contains(logs, "\"server_domain\""));
+
+ // Output logs contains server_country
+ Assertions.assertTrue(StringUtils.containsIgnoreCase(logs, "PrintSinkFunction ") && StringUtils.contains(logs, "\"server_country\""));
+ // Output logs contains mail_attachment_name equals 中文测试
+ Assertions.assertTrue(StringUtils.containsIgnoreCase(logs, "PrintSinkFunction ") && StringUtils.contains(logs, "\"mail_attachment_name\":\"中文测试\""));
+ // Test EVAL function -> output logs contains direction equals c2s
+ Assertions.assertTrue(StringUtils.containsIgnoreCase(logs, "PrintSinkFunction ") && StringUtils.contains(logs, "\"direction\":\"c2s\""));
+ // Test JSON Extract function -> output logs contains device_group equals XXG-TSG-BJ
+ Assertions.assertTrue(StringUtils.containsIgnoreCase(logs, "PrintSinkFunction ") && StringUtils.contains(logs, "\"device_group\":\"XXG-TSG-BJ\""));
+
+ Assertions.assertTrue(StringUtils.containsIgnoreCase(logs, "PrintSinkFunction ") && StringUtils.contains(logs, "client_ip_list"));
+
+
+
+
+
});
diff --git a/groot-tests/test-e2e-base/src/test/resources/inline_to_print.yaml b/groot-tests/test-e2e-base/src/test/resources/inline_to_print.yaml
index b4773a1..2908ffb 100644
--- a/groot-tests/test-e2e-base/src/test/resources/inline_to_print.yaml
+++ b/groot-tests/test-e2e-base/src/test/resources/inline_to_print.yaml
@@ -2,48 +2,205 @@ sources:
inline_source:
type: inline
properties:
- data: '[{"tcp_rtt_ms":128,"decoded_as":"HTTP","rtp_pcap_path":"test_pcap_file","http_request_body":"test_http_req_file","http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.ct.cn","http_url":"www.ct.cn/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.11.22.22","server_ip":"8.8.8.8","client_port":42751,"server_port":80,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":5575,"sent_pkts":97,"sent_bytes":5892,"received_pkts":250,"received_bytes":333931},{"tcp_rtt_ms":256,"decoded_as":"HTTP","http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.abc.cn","http_url":"www.cabc.cn/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.168.10.198","server_ip":"4.4.4.4","client_port":42751,"server_port":80,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":2575,"sent_pkts":197,"sent_bytes":5892,"received_pkts":350,"received_bytes":533931}]'
+ data: [{"tcp_rtt_ms":128,"decoded_as":"DNS","rtp_pcap_path":"test_pcap_file", "security_rule_id_list": [1,10,100,300], "http_request_body":"test_http_req_file","http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.ct.cn","http_url":"www.ct.cn/","ssl_sni":"www.ct.cn", "http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"flags":8192, "address_type":4,"mail_subject":"中文标题测试","mail_attachment_name":"5Lit5paH5rWL6K+V","mail_attachment_name_charset": "utf8","device_tag": "{\"tags\":[{\"tag\":\"data_center\",\"value\":\"XXG-TSG-BJ\"},{\"tag\":\"device_group\",\"value\":\"XXG-TSG-BJ\"}]}", "client_ip":"192.11.22.22","server_ip":"8.8.8.8","client_port":42751,"server_port":80,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":5575,"sent_pkts":97,"sent_bytes":5892,"received_pkts":250,"received_bytes":333931},{"tcp_rtt_ms":256,"decoded_as":"HTTP","http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.abc.cn","http_url":"www.cabc.cn/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.168.10.198","mail_subject":"中文标题测试","server_ip":"4.4.4.4","client_port":42751,"server_port":80,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":2575,"sent_pkts":197,"sent_bytes":5892,"received_pkts":350,"received_bytes":533931},{"tcp_rtt_ms":256,"decoded_as":"HTTP","http_version":"http1","mail_subject":"english subject test","http_request_line":"GET / HTTP/1.1","http_host":"www.5555.com","http_url":"www.5555.com/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.168.10.1","server_ip":"5.5.5.5","client_port":42751,"server_port":53,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":2575,"sent_pkts":197,"sent_bytes":5892,"received_pkts":350,"received_bytes":533931},{"tcp_rtt_ms":256,"decoded_as":"HTTP","http_version":"http1","http_request_line":"GET / HTTP/1.1","http_host":"www.6666.cn","http_url":"www.6666.cn/","http_user_agent":"curl/8.0.1","http_status_code":200,"http_response_line":"HTTP/1.1 200 OK","http_response_content_type":"text/html; charset=UTF-8","http_response_latency_ms":31,"http_session_duration_ms":5451,"in_src_mac":"ba:bb:a7:3c:67:1c","in_dest_mac":"86:dd:7a:8f:ae:e2","mail_subject":"中文标题测试","out_src_mac":"86:dd:7a:8f:ae:e2","out_dest_mac":"ba:bb:a7:3c:67:1c","tcp_client_isn":678677906,"tcp_server_isn":1006700307,"address_type":4,"client_ip":"192.168.100.1","server_ip":"6.6.6.6","client_port":42751,"server_port":53,"in_link_id":65535,"out_link_id":65535,"start_timestamp_ms":1703646546127,"end_timestamp_ms":1703646551702,"duration_ms":2575,"sent_pkts":197,"sent_bytes":5892,"received_pkts":350,"received_bytes":533931}]
format: json
json.ignore.parse.errors: false
filters:
- filter_operator:
- type: com.geedgenetworks.core.filter.AviatorFilter
+ server_ip_filter:
+ type: aviator
properties:
- expression: event.server_ip != '12.12.12.12'
+ expression: event.server_ip != '4.4.4.4'
+
+splits:
+ decoded_as_split:
+ type: split
+ rules:
+ - tag: http_tag
+ expression: event.decoded_as == 'HTTP'
+ - tag: dns_tag
+ expression: event.decoded_as == 'DNS'
+
processing_pipelines:
projection_processor:
type: projection
remove_fields: [http_request_line, http_response_line, http_response_content_type]
functions:
+
- function: DROP
- filter: event.server_ip == '4.4.4.4'
+ filter: event.server_ip == '5.5.5.5'
+
+ - function: SNOWFLAKE_ID
+ output_fields: [ log_id ]
+ parameters:
+ data_center_id_num: 1
+
+ - function: UUID
+ output_fields: [ log_uuid ]
+
+ - function: UUIDv5
+ lookup_fields: [ client_ip, server_ip ]
+ output_fields: [ ip_uuid ]
+ parameters:
+ namespace: NAMESPACE_IP
+ - function: UUIDv7
+ output_fields: [ log_uuid_v7 ]
+
+ - function: ASN_LOOKUP
+ lookup_fields: [ server_ip ]
+ output_fields: [ server_asn ]
+ parameters:
+ kb_name: tsg_ip_asn
+ option: IP_TO_ASN
+
+ - function: GEOIP_LOOKUP
+ lookup_fields: [ server_ip ]
+ output_fields: [ ]
+ parameters:
+ kb_name: tsg_ip_location
+ option: IP_TO_OBJECT
+ geolocation_field_mapping:
+ COUNTRY: server_country
+ PROVINCE: server_super_administrative_area
+ CITY: server_administrative_area
+ LONGITUDE: server_longitude
+ LATITUDE: server_latitude
+ ISP: server_isp
+ ORGANIZATION: server_organization
+
+ - function : BASE64_ENCODE_TO_STRING
+ lookup_fields: [ mail_subject ]
+ output_fields: [ mail_subject_base64 ]
+ parameters:
+ input_type: string
+
+ - function: BASE64_DECODE_TO_STRING
+ output_fields: [ mail_attachment_name ]
+ parameters:
+ value_field: mail_attachment_name
+ charset_field: mail_attachment_name_charset
+ - function: CURRENT_UNIX_TIMESTAMP
+ output_fields: [ current_unix_timestamp_ms ]
+ parameters:
+ precision: milliseconds
+
+ - function: DOMAIN
+ lookup_fields: [ http_host, ssl_sni, quic_sni ]
+ output_fields: [ server_domain ]
+ parameters:
+ option: FIRST_SIGNIFICANT_SUBDOMAIN
+
+ - function: EVAL
+ output_fields: [ recv_time ]
+ parameters:
+ value_expression: current_unix_timestamp_ms
+
+ - function: EVAL
+ output_fields: [ direction ]
+ parameters:
+ value_expression: "(flags & 24576) == 24576 ? 'double' : ((flags & 8192) == 8192 ? 'c2s' : ((flags & 16384) == 16384 ? 's2c' : 'unknown'))"
+
+ - function: EVAL
+ output_fields: [ constant_value ]
+ parameters:
+ value_expression: "'abc'"
+
+ - function: JSON_EXTRACT
+ lookup_fields: [ device_tag ]
+ output_fields: [ device_group ]
+ parameters:
+ value_expression: $.tags[?(@.tag=='device_group')][0].value
+
+ - function: FLATTEN
+ lookup_fields: [ device_tag ]
+ parameters:
+ prefix: olap
+ json_string_keys: [device_tag]
+
+ - function: FROM_UNIX_TIMESTAMP
+ lookup_fields: [ current_unix_timestamp_ms ]
+ output_fields: [ current_time_str ]
+ parameters:
+ precision: milliseconds
+
+ - function: GENERATE_STRING_ARRAY
+ lookup_fields: [server_ip, server_port]
+ output_fields: [server_ip_port]
+
- function: PATH_COMBINE
lookup_fields: [ rtp_pcap_path ]
output_fields: [ rtp_pcap_path ]
parameters:
path: [ props.hos.path, props.hos.bucket.name.rtp_file, rtp_pcap_path ]
+
- function: PATH_COMBINE
lookup_fields: [ http_request_body ]
output_fields: [ http_request_body ]
parameters:
path: [ props.hos.path, props.hos.bucket.name.http_file, http_request_body ]
+ - function: RENAME
+ parameters:
+ rename_fields:
+ current_unix_timestamp_ms: processing_time_ms
+ rename_expression: key = string.replace_all(key,'olap.device_tag.tags','device_tags'); return key;
+
+ - function: UNIX_TIMESTAMP_CONVERTER
+ lookup_fields: [ __timestamp ]
+ output_fields: [stat_time_minute]
+ parameters:
+ precision: minutes
+
+ dns_table_processor:
+ type: table
+ functions:
+ - function: UNROLL
+ lookup_fields: [ security_rule_id_list ]
+ output_fields: [ security_rule_id ]
+
+ dns_aggregate_processor:
+ type: aggregate
+ group_by_fields: [ decoded_as ]
+ window_type: tumbling_processing_time
+ window_size: 5
+ functions:
+ - function: LONG_COUNT
+ output_fields: [ count ]
+ - function: COLLECT_LIST
+ lookup_fields: [ client_ip ]
+ output_fields: [ client_ip_list ]
+
+
+
sinks:
- print_sink:
+ global_print_sink:
+ type: print
+ properties:
+ format: json
+ mode: log_warn
+ dns_print_sink:
type: print
properties:
format: json
mode: log_warn
+ http_print_sink:
+ type: print
+ properties:
+ format: json
+ mode: log_warn
+
application:
env:
name: example-inline-to-print
- parallelism: 3
+ parallelism: 1
pipeline:
object-reuse: true
+
properties:
+ hos.path: http://192.168.44.12:9098/hos
+ hos.bucket.name.troubleshooting_file: troubleshooting_file_bucket
hos.bucket.name.rtp_file: job_level_traffic_rtp_file_bucket
hos.bucket.name.http_file: job_level_traffic_http_file_bucket
hos.bucket.name.eml_file: job_level_traffic_eml_file_bucket
@@ -51,10 +208,25 @@ application:
topology:
- name: inline_source
- downstream: [filter_operator]
- - name: filter_operator
+ downstream: [server_ip_filter]
+ - name: server_ip_filter
downstream: [ projection_processor ]
- name: projection_processor
- downstream: [ print_sink ]
- - name: print_sink
+ downstream: [ global_print_sink, decoded_as_split ]
+ parallelism: 2
+ - name: decoded_as_split
+ tags: [http_tag, dns_tag]
+ downstream: [ http_print_sink, dns_table_processor ]
+ parallelism: 2
+ - name: dns_table_processor
+ downstream: [ dns_aggregate_processor ]
+ parallelism: 2
+ - name: dns_aggregate_processor
+ downstream: [ dns_print_sink ]
+ parallelism: 2
+ - name: global_print_sink
+ downstream: []
+ - name: http_print_sink
+ downstream: []
+ - name: dns_print_sink
downstream: [] \ No newline at end of file
diff --git a/pom.xml b/pom.xml
index f1fb003..f25bfd5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -23,7 +23,7 @@
</modules>
<properties>
- <revision>1.6.0</revision>
+ <revision>1.7.0</revision>
<java.version>11</java.version>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<maven.compiler.source>${java.version}</maven.compiler.source>
@@ -55,6 +55,7 @@
<jsonpath.version>2.4.0</jsonpath.version>
<fastjson2.version>2.0.32</fastjson2.version>
<hutool.version>5.8.22</hutool.version>
+ <uuid-generator.version>5.1.0</uuid-generator.version>
<bouncycastle.version>1.78.1</bouncycastle.version>
<galaxy.version>2.0.2</galaxy.version>
<guava-retrying.version>2.0.0</guava-retrying.version>
@@ -393,6 +394,12 @@
</dependency>
<dependency>
+ <groupId>com.fasterxml.uuid</groupId>
+ <artifactId>java-uuid-generator</artifactId>
+ <version>${uuid-generator.version}</version>
+ </dependency>
+
+ <dependency>
<groupId>org.bouncycastle</groupId>
<artifactId>bcprov-jdk18on</artifactId>
<version>${bouncycastle.version}</version>
@@ -539,6 +546,11 @@
<finalName>${project.artifactId}-${project.version}</finalName>
<pluginManagement>
<plugins>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>versions-maven-plugin</artifactId>
+ <version>2.16.0</version>
+ </plugin>
<!-- java compiler -->
<plugin>
<groupId>org.apache.maven.plugins</groupId>