summaryrefslogtreecommitdiff
path: root/groot-core/src
diff options
context:
space:
mode:
author窦凤虎 <[email protected]>2024-11-01 10:14:03 +0000
committer窦凤虎 <[email protected]>2024-11-01 10:14:03 +0000
commitf7cec560def3981d52f25fc038aab3d4308d4bd1 (patch)
tree1bebf6ee0210b7d5fa50b43e75a5f54a37639177 /groot-core/src
parentc0b9acfc3adc85abbd06207259b2515edc5c4eae (diff)
parent7868728ddbe3dc08263b1d21b5ffce5dcd9b8052 (diff)
Merge branch 'release/1.7.0' into 'master'v1.7.0master
[feature][bootstrap][common]node新增tags属性用于分流,需要与downstream相对应。rules中name标签修改为t... See merge request galaxy/platform/groot-stream!128
Diffstat (limited to 'groot-core/src')
-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
123 files changed, 4617 insertions, 1322 deletions
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