diff options
| author | 窦凤虎 <[email protected]> | 2024-11-01 10:14:03 +0000 |
|---|---|---|
| committer | 窦凤虎 <[email protected]> | 2024-11-01 10:14:03 +0000 |
| commit | f7cec560def3981d52f25fc038aab3d4308d4bd1 (patch) | |
| tree | 1bebf6ee0210b7d5fa50b43e75a5f54a37639177 /groot-core | |
| parent | c0b9acfc3adc85abbd06207259b2515edc5c4eae (diff) | |
| parent | 7868728ddbe3dc08263b1d21b5ffce5dcd9b8052 (diff) | |
[feature][bootstrap][common]node新增tags属性用于分流,需要与downstream相对应。rules中name标签修改为t...
See merge request galaxy/platform/groot-stream!128
Diffstat (limited to 'groot-core')
124 files changed, 4642 insertions, 1322 deletions
diff --git a/groot-core/pom.xml b/groot-core/pom.xml index e723fa5..184e148 100644 --- a/groot-core/pom.xml +++ b/groot-core/pom.xml @@ -12,6 +12,21 @@ <name>Groot : Core </name> <dependencies> + <dependency> + <groupId>com.typesafe</groupId> + <artifactId>config</artifactId> + </dependency> + + <dependency> + <groupId>com.fasterxml.uuid</groupId> + <artifactId>java-uuid-generator</artifactId> + </dependency> + + <dependency> + <groupId>com.uber</groupId> + <artifactId>h3</artifactId> + <version>4.1.1</version> + </dependency> <dependency> <groupId>org.mock-server</groupId> @@ -109,6 +124,16 @@ <scope>provided</scope> </dependency> + <dependency> + <groupId>io.github.jopenlibs</groupId> + <artifactId>vault-java-driver</artifactId> + <version>6.2.0</version> + </dependency> + <dependency> + <groupId>org.bouncycastle</groupId> + <artifactId>bcpkix-jdk18on</artifactId> + <version>1.78.1</version> + </dependency> </dependencies> <build> diff --git a/groot-core/src/main/java/com/geedgenetworks/core/connector/format/MapDeserialization.java b/groot-core/src/main/java/com/geedgenetworks/core/connector/format/MapDeserialization.java new file mode 100644 index 0000000..7887097 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/connector/format/MapDeserialization.java @@ -0,0 +1,8 @@ +package com.geedgenetworks.core.connector.format;
+
+import java.io.IOException;
+import java.util.Map;
+
+public interface MapDeserialization {
+ Map<String, Object> deserializeToMap(byte[] bytes) throws IOException;
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/factories/FactoryUtil.java b/groot-core/src/main/java/com/geedgenetworks/core/factories/FactoryUtil.java index 6b93dab..a120ca5 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/factories/FactoryUtil.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/factories/FactoryUtil.java @@ -159,6 +159,16 @@ public final class FactoryUtil { return discoverFactory(Thread.currentThread().getContextClassLoader(), factoryClass, connector); } + public static <T extends DecodingFormatFactory> T discoverDecodingFormatFactory( + Class<T> factoryClass, String type) { + return discoverFactory(Thread.currentThread().getContextClassLoader(), factoryClass, type); + } + + public static <T extends EncodingFormatFactory> T discoverEncodingFormatFactory( + Class<T> factoryClass, String type) { + return discoverFactory(Thread.currentThread().getContextClassLoader(), factoryClass, type); + } + private static <T> T readOption(ReadableConfig options, ConfigOption<T> option) { try { return options.get(option); diff --git a/groot-core/src/main/java/com/geedgenetworks/core/filter/AviatorFilter.java b/groot-core/src/main/java/com/geedgenetworks/core/filter/AviatorFilter.java index d8b8bc4..06693c9 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/filter/AviatorFilter.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/filter/AviatorFilter.java @@ -1,11 +1,14 @@ package com.geedgenetworks.core.filter; +import com.alibaba.fastjson.JSONObject; import com.geedgenetworks.common.Event; import com.geedgenetworks.core.pojo.FilterConfig; +import com.typesafe.config.Config; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -public class AviatorFilter implements Filter { +import java.util.Map; + +public class AviatorFilter implements Filter<FilterConfig> { @Override public DataStream<Event> filterFunction( @@ -29,4 +32,12 @@ public class AviatorFilter implements Filter { return "aviator"; } + @Override + public FilterConfig checkConfig(String name, Map<String, Object> configProperties, Config typeSafeConfig) { + + FilterConfig filterConfig = new JSONObject(configProperties).toJavaObject(FilterConfig.class); + filterConfig.setName(name); + return filterConfig; + } + } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/filter/Filter.java b/groot-core/src/main/java/com/geedgenetworks/core/filter/Filter.java index f8b50eb..41daf3d 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/filter/Filter.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/filter/Filter.java @@ -3,13 +3,20 @@ package com.geedgenetworks.core.filter; import com.geedgenetworks.common.Event; import com.geedgenetworks.core.pojo.FilterConfig; +import com.typesafe.config.Config; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -public interface Filter { +import java.io.Serializable; +import java.util.Map; + +public interface Filter<T extends FilterConfig> extends Serializable { DataStream<Event> filterFunction( - DataStream<Event> singleOutputStreamOperator, FilterConfig FilterConfig) + DataStream<Event> singleOutputStreamOperator, T FilterConfig) throws Exception; String type(); + + T checkConfig(String name, Map<String, Object> configProperties, Config typeSafeConfig); + } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/pojo/KmsKey.java b/groot-core/src/main/java/com/geedgenetworks/core/pojo/KmsKey.java new file mode 100644 index 0000000..2690254 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/pojo/KmsKey.java @@ -0,0 +1,19 @@ +package com.geedgenetworks.core.pojo; + + +import lombok.Data; + +@Data +public class KmsKey { + + private byte[] keyData; + private int keyVersion; + + public KmsKey() { + } + + public KmsKey(byte[] keyData, int keyVersion) { + this.keyData = keyData; + this.keyVersion = keyVersion; + } +}
\ No newline at end of file diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/Processor.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/Processor.java index 3852414..1c9ba6f 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/processor/Processor.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/Processor.java @@ -1,15 +1,22 @@ package com.geedgenetworks.core.processor; import com.geedgenetworks.common.Event; +import com.geedgenetworks.core.pojo.ProcessorConfig; +import com.typesafe.config.Config; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -public interface Processor<T> { +import java.io.Serializable; +import java.util.Map; + +public interface Processor<T extends ProcessorConfig> extends Serializable { DataStream<Event> processorFunction( DataStream<Event> singleOutputStreamOperator, T processorConfig, ExecutionConfig config) throws Exception; String type(); + + T checkConfig(String name, Map<String, Object> configProperties, Config typeSafeConfig); } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AbstractFirstAggregation.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AbstractFirstAggregation.java index 3632ba7..ce77ee8 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AbstractFirstAggregation.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AbstractFirstAggregation.java @@ -6,6 +6,7 @@ import com.alibaba.fastjson.JSON; import com.geedgenetworks.common.Accumulator; import com.geedgenetworks.common.Constants; import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.KeybyEntity; import com.geedgenetworks.common.exception.CommonErrorCode; import com.geedgenetworks.common.exception.GrootStreamRuntimeException; import com.geedgenetworks.common.udf.AggregateFunction; @@ -139,12 +140,10 @@ public class AbstractFirstAggregation extends ProcessFunction<Event, Accumulator return (long) (ThreadLocalRandom.current().nextDouble() * (double) windowSize); } - public Accumulator createAccumulator() { + public Accumulator createAccumulator(Map<String, Object> keysMap) { - Map<String, Object> map = new HashMap<>(); Accumulator accumulator = new Accumulator(); - - accumulator.setMetricsFields(map); + accumulator.setMetricsFields(keysMap); for (UdfEntity udfEntity : functions) { udfEntity.getAggregateFunction().initAccumulator(accumulator); } @@ -152,17 +151,21 @@ public class AbstractFirstAggregation extends ProcessFunction<Event, Accumulator } - public String getKey(Event value, List<String> keys) { + public KeybyEntity getKey(Event value, List<String> keys) { + KeybyEntity keybyEntity = new KeybyEntity(new HashMap<>()); StringBuilder stringBuilder = new StringBuilder(); for (String key : keys) { - - if (value.getExtractedFields().containsKey(key)) { - stringBuilder.append(value.getExtractedFields().get(key).toString()); - } else { + Object object = value.getExtractedFields().get(key); + if(object==null){ stringBuilder.append(","); + }else{ + keybyEntity.getKeys().put(key, object); + stringBuilder.append(object); } + } - return SecureUtil.md5(stringBuilder.toString()); + keybyEntity.setKeysToString(SecureUtil.md5(stringBuilder.toString())); + return keybyEntity; } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorImpl.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorImpl.java index c261fb6..4712d36 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorImpl.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/AggregateProcessorImpl.java @@ -1,9 +1,15 @@ package com.geedgenetworks.core.processor.aggregate; +import com.alibaba.fastjson.JSONObject; import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.config.AggregateConfigOptions; +import com.geedgenetworks.common.config.CheckConfigUtil; +import com.geedgenetworks.common.config.CheckResult; import com.geedgenetworks.common.exception.CommonErrorCode; +import com.geedgenetworks.common.exception.ConfigValidationException; import com.geedgenetworks.common.exception.GrootStreamRuntimeException; import com.geedgenetworks.core.pojo.AggregateConfig; +import com.typesafe.config.Config; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; @@ -13,6 +19,8 @@ import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindo import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows; import org.apache.flink.streaming.api.windowing.time.Time; +import java.util.Map; + import static com.geedgenetworks.common.Constants.*; public class AggregateProcessorImpl implements AggregateProcessor { @@ -97,4 +105,25 @@ public class AggregateProcessorImpl implements AggregateProcessor { return "aggregate"; } + @Override + public AggregateConfig checkConfig(String name, Map<String, Object> configProperties, Config typeSafeConfig) { + CheckResult result = CheckConfigUtil.checkAllExists(typeSafeConfig.getConfig(name), + AggregateConfigOptions.GROUP_BY_FIELDS.key(), + AggregateConfigOptions.WINDOW_TYPE.key(), + AggregateConfigOptions.FUNCTIONS.key(), + AggregateConfigOptions.WINDOW_SIZE.key()); + if (!result.isSuccess()) { + throw new ConfigValidationException(CommonErrorCode.CONFIG_VALIDATION_FAILED, String.format( + "Aggregate processor: %s, At least one of [%s] should be specified.", + name, String.join(",", + AggregateConfigOptions.OUTPUT_FIELDS.key(), + AggregateConfigOptions.REMOVE_FIELDS.key(), + AggregateConfigOptions.FUNCTIONS.key()))); + } + + AggregateConfig aggregateConfig = new JSONObject(configProperties).toJavaObject(AggregateConfig.class); + aggregateConfig.setName(name); + return aggregateConfig; + } + } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/FirstAggregationEventTime.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/FirstAggregationEventTime.java index 156c0ed..5adc6d1 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/FirstAggregationEventTime.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/FirstAggregationEventTime.java @@ -3,6 +3,7 @@ package com.geedgenetworks.core.processor.aggregate; import com.geedgenetworks.common.Accumulator; import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.KeybyEntity; import com.geedgenetworks.core.pojo.AggregateConfig; import lombok.extern.slf4j.Slf4j; import org.apache.flink.configuration.Configuration; @@ -33,7 +34,7 @@ public class FirstAggregationEventTime extends AbstractFirstAggregation { Long timestamp; internalMetrics.incrementInEvents(); try { - String key = getKey(value, groupByFields); + KeybyEntity keybyEntity = getKey(value, groupByFields); while ((timestamp = eventTimeTimersQueue.peek()) != null && timestamp <= ctx.timerService().currentWatermark()) { eventTimeTimersQueue.poll(); onTimer(timestamp, out); @@ -41,15 +42,15 @@ public class FirstAggregationEventTime extends AbstractFirstAggregation { long windowEnd = assignWindowEnd(ctx.timerService().currentWatermark()); if (!windows.containsKey(windowEnd)) { Map<String, Accumulator> map = new HashMap<>(); - map.put(key, createAccumulator()); + map.put(keybyEntity.getKeysToString(), createAccumulator(keybyEntity.getKeys())); windows.put(windowEnd, map); eventTimeTimersQueue.add(windowEnd); } else { - if (!windows.get(windowEnd).containsKey(key)) { - windows.get(windowEnd).put(key, createAccumulator()); + if (!windows.get(windowEnd).containsKey(keybyEntity.getKeysToString())) { + windows.get(windowEnd).put(keybyEntity.getKeysToString(), createAccumulator(keybyEntity.getKeys())); } } - add(value, windows.get(windowEnd).get(key)); + add(value, windows.get(windowEnd).get(keybyEntity.getKeysToString())); } catch (Exception e) { log.error("Error in pre-aggregate processElement", e); internalMetrics.incrementErrorEvents(); diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/FirstAggregationProcessingTime.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/FirstAggregationProcessingTime.java index e98daa5..01c346f 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/FirstAggregationProcessingTime.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/FirstAggregationProcessingTime.java @@ -3,6 +3,7 @@ package com.geedgenetworks.core.processor.aggregate; import com.geedgenetworks.common.Accumulator; import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.KeybyEntity; import com.geedgenetworks.core.pojo.AggregateConfig; import lombok.extern.slf4j.Slf4j; import org.apache.flink.configuration.Configuration; @@ -32,7 +33,7 @@ public class FirstAggregationProcessingTime extends AbstractFirstAggregation { Long timestamp; internalMetrics.incrementInEvents(); try { - String key = getKey(value, groupByFields); + KeybyEntity keybyEntity = getKey(value, groupByFields); while ((timestamp = processingTimeTimersQueue.peek()) != null && timestamp <= ctx.timerService().currentProcessingTime()) { processingTimeTimersQueue.poll(); onTimer(timestamp, out); @@ -40,15 +41,15 @@ public class FirstAggregationProcessingTime extends AbstractFirstAggregation { long windowEnd = assignWindowEnd(ctx.timerService().currentProcessingTime()); if (!windows.containsKey(windowEnd)) { Map<String, Accumulator> map = new HashMap<>(); - map.put(key, createAccumulator()); + map.put(keybyEntity.getKeysToString(), createAccumulator(keybyEntity.getKeys())); windows.put(windowEnd, map); processingTimeTimersQueue.add(windowEnd); } else { - if (!windows.get(windowEnd).containsKey(key)) { - windows.get(windowEnd).put(key, createAccumulator()); + if (!windows.get(windowEnd).containsKey(keybyEntity.getKeysToString())) { + windows.get(windowEnd).put(keybyEntity.getKeysToString(), createAccumulator(keybyEntity.getKeys())); } } - add(value, windows.get(windowEnd).get(key)); + add(value, windows.get(windowEnd).get(keybyEntity.getKeysToString())); } catch (Exception e) { log.error("Error in pre-aggregate processElement", e); internalMetrics.incrementErrorEvents(); diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/KeySelector.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/KeySelector.java index da09690..a6fb294 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/KeySelector.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/KeySelector.java @@ -22,12 +22,12 @@ public class KeySelector implements org.apache.flink.api.java.functions.KeySelec KeybyEntity keybyEntity = new KeybyEntity(new HashMap<>()); StringBuilder stringBuilder = new StringBuilder(); for (String key : keys) { - - if (value.getExtractedFields().containsKey(key)) { - keybyEntity.getKeys().put(key, value.getExtractedFields().get(key)); - stringBuilder.append(value.getExtractedFields().get(key).toString()); - } else { + Object object = value.getExtractedFields().get(key); + if(object==null){ stringBuilder.append(","); + }else{ + keybyEntity.getKeys().put(key, object); + stringBuilder.append(object); } } String hashedKey = SecureUtil.md5(stringBuilder.toString()); diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/PreKeySelector.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/PreKeySelector.java index 6e43184..4b21ba7 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/PreKeySelector.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/aggregate/PreKeySelector.java @@ -22,12 +22,12 @@ public class PreKeySelector implements org.apache.flink.api.java.functions.KeySe KeybyEntity keybyEntity = new KeybyEntity(new HashMap<>()); StringBuilder stringBuilder = new StringBuilder(); for (String key : keys) { - - if (value.getMetricsFields().containsKey(key)) { - keybyEntity.getKeys().put(key, value.getMetricsFields().get(key)); - stringBuilder.append(value.getMetricsFields().get(key).toString()); - } else { + Object object = value.getMetricsFields().get(key); + if(object==null){ stringBuilder.append(","); + }else{ + keybyEntity.getKeys().put(key, object); + stringBuilder.append(object); } } String hashedKey = SecureUtil.md5(stringBuilder.toString()); diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/ProjectionProcessorImpl.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/ProjectionProcessorImpl.java index d87e7e2..7b35566 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/ProjectionProcessorImpl.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/projection/ProjectionProcessorImpl.java @@ -1,13 +1,22 @@ package com.geedgenetworks.core.processor.projection; +import com.alibaba.fastjson.JSONObject; import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.config.CheckConfigUtil; +import com.geedgenetworks.common.config.CheckResult; +import com.geedgenetworks.common.config.ProjectionConfigOptions; +import com.geedgenetworks.common.exception.CommonErrorCode; +import com.geedgenetworks.common.exception.ConfigValidationException; import com.geedgenetworks.core.pojo.ProjectionConfig; +import com.typesafe.config.Config; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.util.OutputTag; +import java.util.Map; + public class ProjectionProcessorImpl implements ProjectionProcessor { @@ -28,4 +37,25 @@ public class ProjectionProcessorImpl implements ProjectionProcessor { public String type() { return "projection"; } + + @Override + public ProjectionConfig checkConfig(String name, Map<String, Object> configProperties, Config typeSafeConfig) { + CheckResult result = CheckConfigUtil.checkAtLeastOneExists(typeSafeConfig.getConfig(name), + ProjectionConfigOptions.OUTPUT_FIELDS.key(), + ProjectionConfigOptions.REMOVE_FIELDS.key(), + ProjectionConfigOptions.FUNCTIONS.key()); + if (!result.isSuccess()) { + throw new ConfigValidationException(CommonErrorCode.CONFIG_VALIDATION_FAILED, String.format( + "Processor: %s, At least one of [%s] should be specified.", + name, String.join(",", + ProjectionConfigOptions.OUTPUT_FIELDS.key(), + ProjectionConfigOptions.REMOVE_FIELDS.key(), + ProjectionConfigOptions.FUNCTIONS.key()))); + } + + ProjectionConfig projectionConfig = new JSONObject(configProperties).toJavaObject(ProjectionConfig.class); + projectionConfig.setName(name); + + return projectionConfig; + } } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessorImpl.java b/groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessorImpl.java index 6ddc616..84454cf 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessorImpl.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/processor/table/TableProcessorImpl.java @@ -1,7 +1,14 @@ package com.geedgenetworks.core.processor.table; +import com.alibaba.fastjson.JSONObject; import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.config.CheckConfigUtil; +import com.geedgenetworks.common.config.CheckResult; +import com.geedgenetworks.common.config.TableConfigOptions; +import com.geedgenetworks.common.exception.CommonErrorCode; +import com.geedgenetworks.common.exception.ConfigValidationException; import com.geedgenetworks.core.pojo.TableConfig; +import com.typesafe.config.Config; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; @@ -30,4 +37,24 @@ public class TableProcessorImpl implements TableProcessor { public String type() { return "table"; } + + @Override + public TableConfig checkConfig(String name, Map<String, Object> configProperties, Config typeSafeConfig) { + CheckResult result = CheckConfigUtil.checkAtLeastOneExists(typeSafeConfig.getConfig(name), + TableConfigOptions.OUTPUT_FIELDS.key(), + TableConfigOptions.REMOVE_FIELDS.key(), + TableConfigOptions.FUNCTIONS.key()); + if (!result.isSuccess()) { + throw new ConfigValidationException(CommonErrorCode.CONFIG_VALIDATION_FAILED, String.format( + "Table processor: %s, At least one of [%s] should be specified.", + name, String.join(",", + TableConfigOptions.OUTPUT_FIELDS.key(), + TableConfigOptions.REMOVE_FIELDS.key(), + TableConfigOptions.FUNCTIONS.key()))); + } + + TableConfig tableConfig = new JSONObject(configProperties).toJavaObject(TableConfig.class); + tableConfig.setName(name); + return tableConfig; + } } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/split/Split.java b/groot-core/src/main/java/com/geedgenetworks/core/split/Split.java index 37e7b44..4e4e387 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/split/Split.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/split/Split.java @@ -2,15 +2,21 @@ package com.geedgenetworks.core.split; import com.geedgenetworks.common.Event; import com.geedgenetworks.core.pojo.SplitConfig; +import com.typesafe.config.Config; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import java.io.Serializable; +import java.util.Map; import java.util.Set; -public interface Split { +public interface Split<T extends SplitConfig> extends Serializable { DataStream<Event> splitFunction( - DataStream<Event> dataStream, SplitConfig splitConfig) + DataStream<Event> dataStream, T splitConfig) throws Exception; String type(); + + T checkConfig(String name, Map<String, Object> configProperties, Config typeSafeConfig); + } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/split/SplitFunction.java b/groot-core/src/main/java/com/geedgenetworks/core/split/SplitFunction.java index f07b568..07d4f9f 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/split/SplitFunction.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/split/SplitFunction.java @@ -43,7 +43,7 @@ public class SplitFunction extends ProcessFunction<Event, Event> { instance.setFunctionMissing(null); Expression compiledExp = instance.compile(expression, true); rule.setCompiledExpression(compiledExp); - OutputTag<Event> outputTag = new OutputTag<>(rule.getName()){}; + OutputTag<Event> outputTag = new OutputTag<>(rule.getTag()){}; rule.setOutputTag(outputTag); } } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/split/SplitOperator.java b/groot-core/src/main/java/com/geedgenetworks/core/split/SplitOperator.java index f6d2c8c..48ef92d 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/split/SplitOperator.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/split/SplitOperator.java @@ -1,10 +1,15 @@ package com.geedgenetworks.core.split; +import com.alibaba.fastjson.JSONObject; import com.geedgenetworks.common.Event; +import com.geedgenetworks.core.pojo.FilterConfig; import com.geedgenetworks.core.pojo.SplitConfig; +import com.typesafe.config.Config; import org.apache.flink.streaming.api.datastream.DataStream; -public class SplitOperator implements Split { +import java.util.Map; + +public class SplitOperator implements Split<SplitConfig> { @Override public DataStream<Event> splitFunction( @@ -26,4 +31,11 @@ public class SplitOperator implements Split { return "split"; } + @Override + public SplitConfig checkConfig(String name, Map<String, Object> configProperties, Config config) { + SplitConfig splitConfig = new JSONObject(configProperties).toJavaObject(SplitConfig.class); + splitConfig.setName(name); + return splitConfig; + } + } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/AsnLookup.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/AsnLookup.java index bdb3698..ac282b3 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/AsnLookup.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/AsnLookup.java @@ -2,8 +2,7 @@ package com.geedgenetworks.core.udf; import com.alibaba.fastjson2.JSON; import com.geedgenetworks.common.Constants; -import com.geedgenetworks.common.config.CommonConfig; -import com.geedgenetworks.common.config.KnowledgeBaseConfig; +import com.geedgenetworks.common.config.*; import com.geedgenetworks.common.exception.CommonErrorCode; import com.geedgenetworks.common.exception.GrootStreamRuntimeException; import com.geedgenetworks.common.udf.ScalarFunction; @@ -24,10 +23,23 @@ public class AsnLookup implements ScalarFunction { private String lookupFieldName; private String outputFieldName; + + enum Option { + IP_TO_ASN; + + public static boolean isValid(String option) { + try { + Option.valueOf(option); + return true; + } catch (IllegalArgumentException e) { + return false; + } + } + } @Override public void open(RuntimeContext runtimeContext, UDFContext udfContext) { - checkUdfContext(udfContext); + checkConfig(udfContext); this.kbName = udfContext.getParameters().get("kb_name").toString(); this.option = udfContext.getParameters().get("option").toString(); Configuration configuration = (Configuration) runtimeContext @@ -44,8 +56,8 @@ public class AsnLookup implements ScalarFunction { } else { log.error("AsnLookup init KnowledgeBase error "); } - this.lookupFieldName = udfContext.getLookup_fields().get(0); - this.outputFieldName = udfContext.getOutput_fields().get(0); + this.lookupFieldName = udfContext.getLookupFields().get(0); + this.outputFieldName = udfContext.getOutputFields().get(0); AsnKnowledgeBaseHandler.increment(); log.warn("AsnKnowledgeBaseHandlerCount "+AsnKnowledgeBaseHandler.getCount()); } @@ -54,8 +66,9 @@ public class AsnLookup implements ScalarFunction { @Override public Event evaluate(Event event) { - if (event.getExtractedFields().containsKey(lookupFieldName)) { - String asn = AsnKnowledgeBaseHandler.lookUp(kbName, option, event.getExtractedFields().get(lookupFieldName).toString()); + Object valueObj = event.getExtractedFields().get(lookupFieldName); + if (valueObj!=null) { + String asn = AsnKnowledgeBaseHandler.lookUp(kbName, option, valueObj.toString()); if (!asn.isEmpty()) { event.getExtractedFields().put(outputFieldName, asn); } @@ -78,29 +91,38 @@ public class AsnLookup implements ScalarFunction { } } - private void checkUdfContext(UDFContext udfContext) { - - if (udfContext.getLookup_fields() == null || udfContext.getOutput_fields() == null || udfContext.getParameters() == null) { - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); + @Override + public void checkConfig(UDFContext udfContext) { + CheckResult result = CheckUDFContextUtil.checkAllExists(udfContext, + UDFContextConfigOptions.LOOKUP_FIELDS.key(), + UDFContextConfigOptions.OUTPUT_FIELDS.key(), + UDFContextConfigOptions.PARAMETERS.key()); + + if (!result.isSuccess()) { + throw new GrootStreamRuntimeException(CommonErrorCode.CONFIG_VALIDATION_FAILED, result.getMsg()); } - if (udfContext.getLookup_fields().size() != 1) { - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function lookup fields only support 1 value"); + result = CheckUDFContextUtil.checkCollectionSingleItemExists(udfContext, UDFContextConfigOptions.LOOKUP_FIELDS.key()); + if (!result.isSuccess()) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, result.getMsg()); } - if (udfContext.getOutput_fields().size() != 1) { - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value"); + + result = CheckUDFContextUtil.checkCollectionSingleItemExists(udfContext, UDFContextConfigOptions.OUTPUT_FIELDS.key()); + if (!result.isSuccess()) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, result.getMsg()); } - if (!udfContext.getParameters().containsKey("kb_name")) { - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters need containkey kb_name"); + result = CheckUDFContextUtil.checkParametersContainsKeys(udfContext, UDFContextConfigOptions.PARAMETERS_KB_NAME.key(), UDFContextConfigOptions.PARAMETERS_OPTION.key()); + if (!result.isSuccess()) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, result.getMsg()); } - if (!udfContext.getParameters().containsKey("option")) { - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters need containkey option"); - } else { - if (!udfContext.getParameters().get("option").toString().equals("IP_TO_ASN")) { - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters option value is not correct"); - } + + String optionValue = udfContext.getParameters().get(UDFContextConfigOptions.PARAMETERS_OPTION.key()).toString(); + if (!Option.isValid(optionValue)) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, String.format( "UDF: %s, [%s] Option value is not correct.", + udfContext.getFunction(), UDFContextConfigOptions.PARAMETERS.key())); } + } } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/CurrentUnixTimestamp.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/CurrentUnixTimestamp.java index 5770201..98b2d68 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/CurrentUnixTimestamp.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/CurrentUnixTimestamp.java @@ -18,10 +18,10 @@ public class CurrentUnixTimestamp implements ScalarFunction { public void open(RuntimeContext runtimeContext, UDFContext udfContext) { - if( udfContext.getOutput_fields()==null || udfContext.getParameters() == null){ + if( udfContext.getOutputFields()==null || udfContext.getParameters() == null){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } - if(udfContext.getOutput_fields().size() != 1){ + if(udfContext.getOutputFields().size() != 1){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value"); } if(!udfContext.getParameters().containsKey("precision")){ @@ -34,7 +34,7 @@ public class CurrentUnixTimestamp implements ScalarFunction { } } this.precision = udfContext.getParameters().get("precision").toString(); - this.outputFieldName = udfContext.getOutput_fields().get(0); + this.outputFieldName = udfContext.getOutputFields().get(0); } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/DecodeBase64.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/DecodeBase64.java index 3581d5c..bc8563a 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/DecodeBase64.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/DecodeBase64.java @@ -22,16 +22,16 @@ public class DecodeBase64 implements ScalarFunction { @Override public void open(RuntimeContext runtimeContext, UDFContext udfContext) { - if(udfContext.getParameters()==null || udfContext.getOutput_fields()==null){ + if(udfContext.getParameters()==null || udfContext.getOutputFields()==null){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } - if(udfContext.getOutput_fields().size() != 1){ + if(udfContext.getOutputFields().size() != 1){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value"); } if(!udfContext.getParameters().containsKey("value_field") ||!udfContext.getParameters().containsKey("charset_field") ){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters must containkey value_field and charset_field"); } - this.outputFieldName = udfContext.getOutput_fields().get(0); + this.outputFieldName = udfContext.getOutputFields().get(0); this.valueField =udfContext.getParameters().get("value_field").toString(); this.charsetField =udfContext.getParameters().get("charset_field").toString(); @@ -41,15 +41,11 @@ public class DecodeBase64 implements ScalarFunction { @Override public Event evaluate(Event event) { - if (event.getExtractedFields().containsKey(valueField)) { - + Object valueObj = event.getExtractedFields().get(valueField); + if (valueObj!=null) { String decodeResult = ""; - String message = - (String) - event.getExtractedFields() - .get(valueField); - Object charset = - event.getExtractedFields().getOrDefault(charsetField,""); + String message = (String)valueObj; + Object charset = event.getExtractedFields().getOrDefault(charsetField,"UTF-8"); try { if (StringUtil.isNotBlank(message)) { byte[] base64decodedBytes = Base64.getDecoder().decode(message); @@ -62,14 +58,9 @@ public class DecodeBase64 implements ScalarFunction { } catch (RuntimeException e) { log.error("Resolve Base64 exception, exception information:" + e.getMessage()); } catch (UnsupportedEncodingException e) { - log.error( - "The Character Encoding [" - + charset.toString() - + "] is not supported.exception information:" - + e.getMessage()); + log.error("The Character Encoding [" + charset.toString() + "] is not supported.exception information:" + e.getMessage()); } - event.getExtractedFields() - .put(outputFieldName, decodeResult); + event.getExtractedFields().put(outputFieldName, decodeResult); } return event; } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/Domain.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/Domain.java index 77b3246..74816f5 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/Domain.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/Domain.java @@ -1,111 +1,124 @@ package com.geedgenetworks.core.udf; - +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.config.CheckResult; +import com.geedgenetworks.common.config.CheckUDFContextUtil; +import com.geedgenetworks.common.config.UDFContextConfigOptions; import com.geedgenetworks.common.exception.CommonErrorCode; import com.geedgenetworks.common.exception.GrootStreamRuntimeException; import com.geedgenetworks.common.udf.ScalarFunction; -import com.geedgenetworks.common.Event; import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.shaded.com.google.common.net.InternetDomainName; import lombok.extern.slf4j.Slf4j; import org.apache.flink.api.common.functions.RuntimeContext; - import java.util.List; - import static com.geedgenetworks.utils.FormatUtils.getTopPrivateDomain; @Slf4j public class Domain implements ScalarFunction { - - - private String option; + private Option option; private List<String> lookupFields; private String outputFieldName; - @Override - public void open(RuntimeContext runtimeContext, UDFContext udfContext) { - if(udfContext.getLookup_fields()==null || udfContext.getOutput_fields()==null || udfContext.getParameters() == null){ - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); - } - if(udfContext.getLookup_fields().isEmpty()){ - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function lookup field is not empty"); - } - if(udfContext.getOutput_fields().size() != 1){ - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value"); - } - if(!udfContext.getParameters().containsKey("option")){ - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters need containkey option"); - } - else{ - if(!udfContext.getParameters().get("option").toString().equals("TOP_LEVEL_DOMAIN") && - !udfContext.getParameters().get("option").toString().equals("FIRST_SIGNIFICANT_SUBDOMAIN") && - !udfContext.getParameters().get("option").toString().equals("FQDN")){ - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters option value is not correct"); + enum Option { + TOP_LEVEL_DOMAIN, + FIRST_SIGNIFICANT_SUBDOMAIN, + FQDN; + + public static boolean isValid(String option) { + try { + Option.valueOf(option); + return true; + } catch (IllegalArgumentException e) { + return false; } } - this.option = udfContext.getParameters().get("option").toString(); - this.lookupFields = udfContext.getLookup_fields(); - this.outputFieldName = udfContext.getOutput_fields().get(0); + } + @Override + public void open(RuntimeContext runtimeContext, UDFContext udfContext) { + checkConfig(udfContext); + this.option = Option.valueOf(udfContext.getParameters().get("option").toString()); + this.lookupFields = udfContext.getLookupFields(); + this.outputFieldName = udfContext.getOutputFields().get(0); } @Override public Event evaluate(Event event) { String domain = ""; - switch (option) { - case "TOP_LEVEL_DOMAIN": - for (String lookupField : lookupFields){ - if(event.getExtractedFields().containsKey(lookupField)) { - domain = - getTopPrivateDomain( - (String) - event.getExtractedFields() - .get(lookupField)); - if (domain.contains(".")) { - domain = domain.substring(domain.indexOf(".") + 1); - } - if(!domain.isEmpty()){ - break; - } - } - } - break; - case "FIRST_SIGNIFICANT_SUBDOMAIN": - for (String lookupField : lookupFields){ - if(event.getExtractedFields().containsKey(lookupField)) { - domain = - getTopPrivateDomain( - (String) - event.getExtractedFields() - .get(lookupField)); - } - if(!domain.isEmpty()){ + for (String fieldName : lookupFields) { + Object fieldValue = event.getExtractedFields().get(fieldName); + if (fieldValue == null) { + continue; + } + + String value = fieldValue.toString(); + try { + + String topPrivateDomain = getTopPrivateDomain(value); + + switch (option) { + case TOP_LEVEL_DOMAIN: + domain = InternetDomainName.from(topPrivateDomain).publicSuffix().toString(); break; - } - } - break; - case "FQDN": - for (String lookupField : lookupFields) { - if (event.getExtractedFields().containsKey(lookupField)) { - domain = (String) event.getExtractedFields() - .get(lookupField); - } - if(!domain.isEmpty()){ + case FIRST_SIGNIFICANT_SUBDOMAIN: + domain = topPrivateDomain; + break; + case FQDN: // Use the original value + domain = value; break; - } + default: + throw new IllegalArgumentException("Unknown option: " + option); + } + + if (!domain.isEmpty()) { // Found a valid domain will break the loop + break; } - break; + + } catch (IllegalArgumentException e) { + log.error("Invalid domain: {}", value); + } } + event.getExtractedFields().put(outputFieldName, domain); return event; } @Override + public void checkConfig(UDFContext udfContext) { + CheckResult result = CheckUDFContextUtil.checkAllExists(udfContext, + UDFContextConfigOptions.LOOKUP_FIELDS.key(), + UDFContextConfigOptions.OUTPUT_FIELDS.key(), + UDFContextConfigOptions.PARAMETERS.key()); + + if (!result.isSuccess()) { + throw new GrootStreamRuntimeException(CommonErrorCode.CONFIG_VALIDATION_FAILED, result.getMsg()); + } + + result = CheckUDFContextUtil.checkCollectionSingleItemExists(udfContext, UDFContextConfigOptions.OUTPUT_FIELDS.key()); + if (!result.isSuccess()) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, result.getMsg()); + } + + if(!udfContext.getParameters().containsKey(UDFContextConfigOptions.PARAMETERS_OPTION.key())){ + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, String.format( + "UDF: %s, [%s] Option should be specified.", + udfContext.getFunction(), UDFContextConfigOptions.PARAMETERS.key())); + } + + String optionValue = udfContext.getParameters().get(UDFContextConfigOptions.PARAMETERS_OPTION.key()).toString(); + if (!Option.isValid(optionValue)) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, String.format( "UDF: %s, [%s] Option value is not correct.", + udfContext.getFunction(), UDFContextConfigOptions.PARAMETERS.key())); + } + + } + + @Override public String functionName() { return "DOMAIN"; } - - @Override public void close() { diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/Drop.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/Drop.java index 93cd0db..c7f13c2 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/Drop.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/Drop.java @@ -1,5 +1,10 @@ package com.geedgenetworks.core.udf; +import com.geedgenetworks.common.config.CheckResult; +import com.geedgenetworks.common.config.CheckUDFContextUtil; +import com.geedgenetworks.common.config.UDFContextConfigOptions; +import com.geedgenetworks.common.exception.CommonErrorCode; +import com.geedgenetworks.common.exception.GrootStreamRuntimeException; import com.geedgenetworks.common.udf.ScalarFunction; import com.geedgenetworks.common.Event; import com.geedgenetworks.common.udf.UDFContext; @@ -10,6 +15,7 @@ public class Drop implements ScalarFunction { @Override public void open(RuntimeContext runtimeContext, UDFContext udfContext) { + checkConfig(udfContext); } @Override @@ -24,6 +30,17 @@ public class Drop implements ScalarFunction { } @Override + public void checkConfig(UDFContext udfContext) { + CheckResult result = CheckUDFContextUtil.checkAllExists(udfContext, + UDFContextConfigOptions.FILTER.key()); + + if (!result.isSuccess()) { + throw new GrootStreamRuntimeException(CommonErrorCode.CONFIG_VALIDATION_FAILED, result.getMsg()); + } + + } + + @Override public void close() { } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/EncodeBase64.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/EncodeBase64.java index c22ff54..a950252 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/EncodeBase64.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/EncodeBase64.java @@ -8,44 +8,59 @@ import com.geedgenetworks.common.udf.UDFContext; import lombok.extern.slf4j.Slf4j; import org.apache.flink.api.common.functions.RuntimeContext; +import java.nio.charset.StandardCharsets; +import java.text.SimpleDateFormat; import java.util.Base64; +import java.util.List; @Slf4j public class EncodeBase64 implements ScalarFunction { - private String valueField; + private String input_type; private String outputFieldName; + private String lookupFieldName; @Override public void open(RuntimeContext runtimeContext, UDFContext udfContext) { - if(udfContext.getParameters()==null || udfContext.getOutput_fields()==null){ + if(udfContext.getParameters()==null || udfContext.getOutputFields()==null || udfContext.getLookupFields() == null){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } - if(udfContext.getOutput_fields().size() != 1){ + if(udfContext.getOutputFields().size() != 1){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value"); } - if(!udfContext.getParameters().containsKey("value_field") ){ + if(udfContext.getLookupFields().size() != 1){ + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function lookup fields only support 1 value"); + } + if(!udfContext.getParameters().containsKey("input_type") ){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters must containkey value_field "); } - this.outputFieldName = udfContext.getOutput_fields().get(0); - this.valueField =udfContext.getParameters().get("value_field").toString(); - - + this.outputFieldName = udfContext.getOutputFields().get(0); + this.lookupFieldName = udfContext.getLookupFields().get(0); + this.input_type =udfContext.getParameters().get("input_type").toString(); } @Override public Event evaluate(Event event) { String encodeResult = ""; - if (event.getExtractedFields().containsKey(valueField)) { + Object valueObj = event.getExtractedFields().get(lookupFieldName); + if (valueObj!=null) { try { - encodeResult = Base64.getEncoder().encodeToString((byte[]) event.getExtractedFields().getOrDefault(valueField,"".getBytes())); + switch (input_type) { + case "byte_array": + encodeResult = Base64.getEncoder().encodeToString((byte[]) valueObj); + break; + case "string": + encodeResult = Base64.getEncoder().encodeToString(valueObj.toString().getBytes(StandardCharsets.UTF_8)); + break; + default: + log.error("Encode Base64 exception, Unsupport input_type :" + input_type); + break; + } } catch (RuntimeException e) { log.error("Encode Base64 exception, exception information:" + e.getMessage()); } - - event.getExtractedFields() - .put(outputFieldName, encodeResult); + event.getExtractedFields().put(outputFieldName, encodeResult); } return event; } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/Encrypt.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/Encrypt.java new file mode 100644 index 0000000..c8e21b2 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/Encrypt.java @@ -0,0 +1,167 @@ +package com.geedgenetworks.core.udf; + +import cn.hutool.core.util.URLUtil; +import cn.hutool.json.JSONArray; +import cn.hutool.json.JSONObject; +import cn.hutool.json.JSONUtil; +import com.alibaba.fastjson2.JSON; +import com.geedgenetworks.common.Constants; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.config.CommonConfig; +import com.geedgenetworks.common.config.KmsConfig; +import com.geedgenetworks.common.config.SSLConfig; +import com.geedgenetworks.common.exception.CommonErrorCode; +import com.geedgenetworks.common.exception.GrootStreamRuntimeException; +import com.geedgenetworks.common.udf.ScalarFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.core.pojo.KmsKey; +import com.geedgenetworks.core.udf.encrypt.EncryptionAlgorithm; +import com.geedgenetworks.core.utils.*; +import com.geedgenetworks.shaded.org.apache.http.HttpHeaders; +import com.geedgenetworks.shaded.org.apache.http.HttpStatus; +import com.geedgenetworks.shaded.org.apache.http.message.BasicHeader; +import com.geedgenetworks.utils.StringUtil; +import lombok.extern.slf4j.Slf4j; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.configuration.Configuration; + +import java.io.IOException; +import java.net.URI; +import java.util.Arrays; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +@Slf4j +public class Encrypt implements ScalarFunction { + + private String lookupFieldName; + private String outputFieldName; + private String identifier; + private String defaultVal; + private String type; + private transient SingleValueMap.Data<LoadIntervalDataUtil<Set<String>>> sensitiveFieldsData; + private transient SingleValueMap.Data<LoadIntervalDataUtil<KmsKey>> kmsKeyData; + private transient EncryptionAlgorithm encryptionAlgorithm; + + @Override + public void open(RuntimeContext runtimeContext, UDFContext udfContext) { + checkUdfContext(udfContext); + if (udfContext.getParameters().containsKey("default_val")) { + this.defaultVal = udfContext.getParameters().get("default_val").toString(); + } + this.lookupFieldName = udfContext.getLookupFields().get(0); + this.outputFieldName = udfContext.getOutputFields().get(0); + this.identifier = udfContext.getParameters().get("identifier").toString(); + Configuration configuration = (Configuration) runtimeContext.getExecutionConfig().getGlobalJobParameters(); + CommonConfig commonConfig = JSON.parseObject(configuration.toMap().get(Constants.SYSPROP_GROOTSTREAM_CONFIG), CommonConfig.class); + KmsConfig kmsConfig = commonConfig.getKmsConfig().get(configuration.toMap().get(Constants.SYSPROP_KMS_TYPE_CONFIG)); + SSLConfig sslConfig = commonConfig.getSslConfig(); + Map<String, String> propertiesConfig = commonConfig.getPropertiesConfig(); + type = kmsConfig.getType(); + try { + encryptionAlgorithm = EncryptionAlgorithmUtils.createEncryptionAlgorithm(identifier); + if (encryptionAlgorithm == null) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Parameters identifier is illegal!"); + } + if (!type.equals(KmsUtils.KMS_TYPE_LOCAL)) { + kmsKeyData = SingleValueMap.acquireData("kmsKeyData", + () -> LoadIntervalDataUtil.newInstance(() -> KmsUtils.getVaultKey(kmsConfig, sslConfig, identifier), + LoadIntervalDataOptions.defaults("kmsKeyData", Integer.parseInt(propertiesConfig.getOrDefault(Constants.SYSPROP_ENCRYPT_KMS_KEY_SCHEDULER_INTERVAL_NAME, "5")) * 60000L)), + LoadIntervalDataUtil::stop); + KmsKey kmsKey = kmsKeyData.getData().data(); + if (kmsKey == null) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Initialization UDF Encrypt failed!"); + } + if (encryptionAlgorithm.getSecretKeyLength() != kmsKey.getKeyData().length) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Global parameter kms secret Key requires " + encryptionAlgorithm.getSecretKeyLength() + " bytes!"); + } + encryptionAlgorithm.setKmsKey(kmsKey); + } + sensitiveFieldsData = SingleValueMap.acquireData("sensitiveFields", + () -> LoadIntervalDataUtil.newInstance(() -> getSensitiveFields(propertiesConfig.get("projection.encrypt.schema.registry.uri")), + LoadIntervalDataOptions.defaults("sensitiveFields", Integer.parseInt(propertiesConfig.getOrDefault(Constants.SYSPROP_ENCRYPT_SENSITIVE_FIELDS_SCHEDULER_INTERVAL_NAME, "5")) * 60000L)), + LoadIntervalDataUtil::stop); + if (sensitiveFieldsData.getData().data() == null) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Initialization UDF Encrypt failed!"); + } + } catch (Exception e) { + throw new GrootStreamRuntimeException(CommonErrorCode.UNSUPPORTED_OPERATION, "Initialization UDF Encrypt failed!", e); + } + } + + @Override + public Event evaluate(Event event) { + try { + if (!type.equals(KmsUtils.KMS_TYPE_LOCAL)) { + KmsKey kmsKey = kmsKeyData.getData().data(); + if (kmsKey.getKeyVersion() != encryptionAlgorithm.getKmsKey().getKeyVersion() || !Arrays.equals(kmsKey.getKeyData(), encryptionAlgorithm.getKmsKey().getKeyData())) { + encryptionAlgorithm.setKmsKey(kmsKey); + } + } + if (sensitiveFieldsData.getData().data().contains(lookupFieldName) && event.getExtractedFields().containsKey(lookupFieldName)) { + String value = (String) event.getExtractedFields().get(lookupFieldName); + if (StringUtil.isNotBlank(value)) { + String encryptResult = encryptionAlgorithm.encrypt(value); + if (StringUtil.isEmpty(encryptResult)) { + event.getExtractedFields().put(outputFieldName, StringUtil.isNotBlank(defaultVal) ? defaultVal : value); + } else { + if (KmsUtils.KMS_TYPE_VAULT.equals(type)) { + encryptResult = "vault:v" + encryptionAlgorithm.getKmsKey().getKeyVersion() + ":" + encryptResult; + } + event.getExtractedFields().put(outputFieldName, encryptResult); + } + } + } + } catch (Exception e) { + throw new RuntimeException(e); + } + return event; + } + + @Override + public String functionName() { + return "ENCRYPT"; + } + + @Override + public void close() { + if (sensitiveFieldsData != null) { + sensitiveFieldsData.release(); + } + if (kmsKeyData != null) { + kmsKeyData.release(); + } + } + + private void checkUdfContext(UDFContext udfContext) { + if (udfContext.getParameters() == null) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); + } + if (udfContext.getLookupFields().size() != 1) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function lookup fields only support 1 value"); + } + if (udfContext.getOutputFields().size() != 1) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value"); + } + if (!udfContext.getParameters().containsKey("identifier")) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Parameters must contains identifier"); + } + } + + public Set<String> getSensitiveFields(String url) throws IOException { + Set<String> sensitiveFieldsSet; + String sensitiveFieldsStr = HttpClientPoolUtil.getInstance().httpGet(URI.create(URLUtil.normalize(url)), new BasicHeader(HttpHeaders.CONTENT_TYPE, "application/x-www-form-urlencoded")); + JSONObject sensitiveFieldsJson = JSONUtil.parseObj(sensitiveFieldsStr); + if (sensitiveFieldsJson.getInt("status", HttpStatus.SC_INTERNAL_SERVER_ERROR) == HttpStatus.SC_OK) { + JSONArray sensitiveFieldsJsonArr = sensitiveFieldsJson.getJSONArray("data"); + sensitiveFieldsSet = IntStream.range(0, sensitiveFieldsJsonArr.size()) + .mapToObj(sensitiveFieldsJsonArr::getStr) + .collect(Collectors.toSet()); + } else { + throw new GrootStreamRuntimeException(CommonErrorCode.UNSUPPORTED_OPERATION, "Get encrypt fields error! Error message: " + sensitiveFieldsStr); + } + return sensitiveFieldsSet; + } +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/Eval.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/Eval.java index 1b83d94..b04dc97 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/Eval.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/Eval.java @@ -21,14 +21,14 @@ public class Eval implements ScalarFunction { @Override public void open(RuntimeContext runtimeContext, UDFContext udfContext) { - if(udfContext.getOutput_fields()==null || udfContext.getParameters() == null){ + if(udfContext.getOutputFields()==null || udfContext.getParameters() == null){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } if(!udfContext.getParameters().containsKey("value_expression")){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters must containkey value_expression"); } String expr = (String) udfContext.getParameters().get("value_expression"); - List<String> outputField = udfContext.getOutput_fields(); + List<String> outputField = udfContext.getOutputFields(); output = outputField.get(0); calc = new EvalExecutor(expr); } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/Flatten.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/Flatten.java index 84c2c2a..d5d5761 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/Flatten.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/Flatten.java @@ -30,7 +30,7 @@ public class Flatten implements ScalarFunction { prefix = udfContext.getParameters().getOrDefault("prefix", "").toString(); delimiter = udfContext.getParameters().getOrDefault("delimiter", ".").toString(); flattenKeys = new HashSet<>(); - for (String key : udfContext.getLookup_fields()) { + for (String key : udfContext.getLookupFields()) { this.flattenKeys.add(prefix.isEmpty() ? key : prefix + delimiter + key); } depth = Integer.parseInt(udfContext.getParameters().getOrDefault("depth", "5").toString()); diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/FromUnixTimestamp.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/FromUnixTimestamp.java index e1ba384..d8803c3 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/FromUnixTimestamp.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/FromUnixTimestamp.java @@ -9,66 +9,73 @@ import lombok.extern.slf4j.Slf4j; import org.apache.flink.api.common.functions.RuntimeContext; import java.text.SimpleDateFormat; +import java.util.Arrays; import java.util.TimeZone; @Slf4j public class FromUnixTimestamp implements ScalarFunction { private String precision; private String outputFieldName; private String lookupFieldName; + + private String timeZone = "UTC"; + private SimpleDateFormat sdf; @Override public void open(RuntimeContext runtimeContext, UDFContext udfContext) { - if(udfContext.getLookup_fields()==null || udfContext.getOutput_fields()==null || udfContext.getParameters() == null){ + if(udfContext.getLookupFields()==null || udfContext.getOutputFields()==null || udfContext.getParameters() == null){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } - if(udfContext.getOutput_fields().size() != 1){ + if(udfContext.getOutputFields().size() != 1){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value"); } - if(udfContext.getLookup_fields().size() != 1){ + if(udfContext.getLookupFields().size() != 1){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function lookup fields only support 1 value"); } if(!udfContext.getParameters().containsKey("precision")){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters must containkey precision"); } else{ - if(!udfContext.getParameters().get("precision").toString().equals("seconds") && - !udfContext.getParameters().get("precision").toString().equals("milliseconds") && - !udfContext.getParameters().get("precision").toString().equals("microseconds") && - !udfContext.getParameters().get("precision").toString().equals("nanoseconds") ){ + if (!Arrays.asList("seconds", "milliseconds").contains(udfContext.getParameters().get("precision").toString().trim())) { throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters precision value is not correct"); } } + if(udfContext.getParameters().containsKey("timezone")){ + timeZone = udfContext.getParameters().get("timezone").toString(); + } + this.precision = udfContext.getParameters().get("precision").toString(); - this.outputFieldName = udfContext.getOutput_fields().get(0); - this.lookupFieldName = udfContext.getLookup_fields().get(0); + this.outputFieldName = udfContext.getOutputFields().get(0); + this.lookupFieldName = udfContext.getLookupFields().get(0); this.sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); switch (precision) { case "seconds": sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); break; case "milliseconds": - sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss:SSS"); - break; - case "microseconds": - sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss:SSS:000"); - break; - case "nanoseconds": - sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss:SSS:000:000"); + sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); break; default: break; } - sdf.setTimeZone(TimeZone.getTimeZone("UTC")); + sdf.setTimeZone(TimeZone.getTimeZone(timeZone)); } @Override public Event evaluate(Event event) { - if(event.getExtractedFields().containsKey(lookupFieldName)){ - String timestamp = sdf.format(Long.parseLong(event.getExtractedFields().get(lookupFieldName).toString())); - event.getExtractedFields().put(outputFieldName, timestamp); + Object objectValue = event.getExtractedFields().get(lookupFieldName); + if(objectValue!=null){ + String value = objectValue.toString(); + String dateTimeFormat =""; + try { + long timestamp = Long.parseLong(value); + dateTimeFormat = sdf.format(timestamp >= 10000000000L ? timestamp : timestamp * 1000); + } catch (NumberFormatException e) { + log.error("Invalid timestamp format for field {}: {}", lookupFieldName, value, e); + } + event.getExtractedFields().put(outputFieldName, dateTimeFormat); } return event; } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/GenerateStringArray.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/GenerateStringArray.java index ce4fc48..366f204 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/GenerateStringArray.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/GenerateStringArray.java @@ -18,14 +18,14 @@ public class GenerateStringArray implements ScalarFunction { @Override public void open(RuntimeContext runtimeContext, UDFContext udfContext) { - if(udfContext.getLookup_fields()==null || udfContext.getOutput_fields()==null ){ + if(udfContext.getLookupFields()==null || udfContext.getOutputFields()==null ){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } - if(udfContext.getOutput_fields().size() != 1){ + if(udfContext.getOutputFields().size() != 1){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value"); } - this.lookupFieldNames = udfContext.getLookup_fields(); - this.outputFieldName = udfContext.getOutput_fields().get(0); + this.lookupFieldNames = udfContext.getLookupFields(); + this.outputFieldName = udfContext.getOutputFields().get(0); } @Override diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/GeoIpLookup.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/GeoIpLookup.java index afe1bfb..e800e5d 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/GeoIpLookup.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/GeoIpLookup.java @@ -1,8 +1,7 @@ package com.geedgenetworks.core.udf; import com.geedgenetworks.common.Constants; -import com.geedgenetworks.common.config.CommonConfig; -import com.geedgenetworks.common.config.KnowledgeBaseConfig; +import com.geedgenetworks.common.config.*; import com.geedgenetworks.common.exception.CommonErrorCode; import com.geedgenetworks.common.exception.GrootStreamRuntimeException; import com.geedgenetworks.common.udf.ScalarFunction; @@ -28,13 +27,55 @@ public class GeoIpLookup implements ScalarFunction { private String outputFieldName; private Map<String, String> geoLocationFieldMapping; + enum Option { + IP_TO_COUNTRY, + IP_TO_PROVINCE, + IP_TO_CITY, + IP_TO_SUBDIVISION_ADDR, + IP_TO_DETAIL, + IP_TO_LATLNG, + IP_TO_PROVIDER, + IP_TO_JSON, + IP_TO_OBJECT + ; + + public static boolean isValid(String option) { + try { + Option.valueOf(option); + return true; + } catch (IllegalArgumentException e) { + return false; + } + } + } + + enum GeolocationFieldMapping { + COUNTRY, + PROVINCE, + CITY, + LONGITUDE, + LATITUDE, + ISP, + ORGANIZATION + ; + + public static boolean isValid(String option) { + try { + GeolocationFieldMapping.valueOf(option); + return true; + } catch (IllegalArgumentException e) { + return false; + } + } + } + @Override public void open(RuntimeContext runtimeContext, UDFContext udfContext) { - checkUdfContext(udfContext); - this.kbName = udfContext.getParameters().get("kb_name").toString(); - this.option = udfContext.getParameters().get("option").toString(); - if (option.equals("IP_TO_OBJECT")) { - this.geoLocationFieldMapping = (Map<String, String>) udfContext.getParameters().get("geolocation_field_mapping"); + checkConfig(udfContext); + this.kbName = udfContext.getParameters().get(UDFContextConfigOptions.PARAMETERS_KB_NAME.key()).toString(); + this.option = udfContext.getParameters().get(UDFContextConfigOptions.PARAMETERS_OPTION.key()).toString(); + if (option.equals(Option.IP_TO_OBJECT.name())) { + this.geoLocationFieldMapping = (Map<String, String>) udfContext.getParameters().get(UDFContextConfigOptions.PARAMETERS_GEOLOCATION_FIELD_MAPPING.key()); } Configuration configuration = (Configuration) runtimeContext .getExecutionConfig().getGlobalJobParameters(); @@ -50,9 +91,9 @@ public class GeoIpLookup implements ScalarFunction { } else { log.error("GeoIpLookup init KnowledgeBase error "); } - this.lookupFieldName = udfContext.getLookup_fields().get(0); - if(udfContext.getOutput_fields()!=null && !udfContext.getOutput_fields().isEmpty()){ - this.outputFieldName = udfContext.getOutput_fields().get(0); + this.lookupFieldName = udfContext.getLookupFields().get(0); + if(udfContext.getOutputFields()!=null && !udfContext.getOutputFields().isEmpty()){ + this.outputFieldName = udfContext.getOutputFields().get(0); } GeoIpKnowledgeBaseHandler.increment(); log.warn("GeoIpKnowledgeBaseHandler "+GeoIpKnowledgeBaseHandler.getCount()); @@ -61,10 +102,10 @@ public class GeoIpLookup implements ScalarFunction { @Override public Event evaluate(Event event) { - if (event.getExtractedFields().containsKey(lookupFieldName)) { - - if ("IP_TO_OBJECT".equals(option)) { - LocationResponse response = GeoIpKnowledgeBaseHandler.lookUpObject(kbName,event.getExtractedFields().get(lookupFieldName).toString()); + Object valueObj = event.getExtractedFields().get(lookupFieldName); + if (valueObj!=null) { + if (Option.IP_TO_OBJECT.name().equals(option)) { + LocationResponse response = GeoIpKnowledgeBaseHandler.lookUpObject(kbName,valueObj.toString()); for (Map.Entry<String, String> entry : geoLocationFieldMapping.entrySet()) { String result = ""; if (response!=null) { @@ -97,65 +138,56 @@ public class GeoIpLookup implements ScalarFunction { } } } else { - event.getExtractedFields().put(outputFieldName, GeoIpKnowledgeBaseHandler - .lookUp(kbName, option, event.getExtractedFields().get(lookupFieldName).toString())); + event.getExtractedFields().put(outputFieldName, GeoIpKnowledgeBaseHandler.lookUp(kbName, option, valueObj.toString())); } } return event; } - private void checkUdfContext(UDFContext udfContext) { + @Override + public void checkConfig(UDFContext udfContext) { + CheckResult result = CheckUDFContextUtil.checkAllExists(udfContext, + UDFContextConfigOptions.LOOKUP_FIELDS.key(), + UDFContextConfigOptions.PARAMETERS.key()); - if (udfContext.getLookup_fields() == null || udfContext.getParameters() == null) { - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); - } - if (udfContext.getLookup_fields().size() != 1) { - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function lookup fields only support 1 value"); + if (!result.isSuccess()) { + throw new GrootStreamRuntimeException(CommonErrorCode.CONFIG_VALIDATION_FAILED, result.getMsg()); } - if (!udfContext.getParameters().containsKey("kb_name")) { - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters need containkey kb_name"); - } - if (!udfContext.getParameters().containsKey("option")) { - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters need containkey option"); - } else { - if (!udfContext.getParameters().get("option").toString().equals("IP_TO_COUNTRY") && //IP_TO_COUNTRY - !udfContext.getParameters().get("option").toString().equals("IP_TO_PROVINCE") && //IP_TO_PROVINCE - !udfContext.getParameters().get("option").toString().equals("IP_TO_CITY") && //IP_TO_CITY - !udfContext.getParameters().get("option").toString().equals("IP_TO_SUBDIVISION_ADDR") && //IP_TO_SUBDIVISION_ADDR - !udfContext.getParameters().get("option").toString().equals("IP_TO_DETAIL") && //IP_TO_DETAIL - !udfContext.getParameters().get("option").toString().equals("IP_TO_LATLNG") && //IP_TO_LATLNG - !udfContext.getParameters().get("option").toString().equals("IP_TO_PROVIDER") && //IP_TO_PROVIDER - !udfContext.getParameters().get("option").toString().equals("IP_TO_JSON") && //IP_TO_JSON - !udfContext.getParameters().get("option").toString().equals("IP_TO_OBJECT")) { - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters option value is not correct"); - } - if (udfContext.getParameters().get("option").toString().equals("IP_TO_OBJECT")) { - if (!udfContext.getParameters().containsKey("geolocation_field_mapping")) { + result = CheckUDFContextUtil.checkCollectionSingleItemExists(udfContext, UDFContextConfigOptions.LOOKUP_FIELDS.key()); + if (!result.isSuccess()) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, result.getMsg()); + } - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters need containkey geolocation_field_mapping"); + result = CheckUDFContextUtil.checkParametersContainsKeys(udfContext, UDFContextConfigOptions.PARAMETERS_KB_NAME.key(), UDFContextConfigOptions.PARAMETERS_OPTION.key()); + if (!result.isSuccess()) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, result.getMsg()); + } - } else { - Map<String, String> geolocation_field_mapping = (Map<String, String>) udfContext.getParameters().get("geolocation_field_mapping"); + String optionValue = udfContext.getParameters().get(UDFContextConfigOptions.PARAMETERS_OPTION.key()).toString(); + if (!Option.isValid(optionValue)) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, String.format( "UDF: %s, [%s] Option value is not correct.", + udfContext.getFunction(), UDFContextConfigOptions.PARAMETERS.key())); + } - if (!geolocation_field_mapping.isEmpty()) { + if (optionValue.equals(Option.IP_TO_OBJECT.name())) { + result = CheckUDFContextUtil.checkParametersContainsKeys(udfContext, UDFContextConfigOptions.PARAMETERS_GEOLOCATION_FIELD_MAPPING.key()); + if (!result.isSuccess()) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, result.getMsg()); + } - for (Map.Entry<String, String> entry : geolocation_field_mapping.entrySet()) { + Map<String, String> fieldMap = (Map<String, String>) udfContext.getParameters().get(UDFContextConfigOptions.PARAMETERS_GEOLOCATION_FIELD_MAPPING.key()); - if (!entry.getKey().equals("COUNTRY") && !entry.getKey().equals("PROVINCE") && !entry.getKey().equals("CITY") && !entry.getKey().equals("LONGITUDE") && !entry.getKey().equals("LATITUDE") && !entry.getKey().equals("ISP") && !entry.getKey().equals("ORGANIZATION")) { - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters geolocation_field_mapping value is not correct"); - } - } - } else { - throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters geolocation_field_mapping value is not correct"); - } + for (Map.Entry<String, String> entry : fieldMap.entrySet()) { + if (!GeolocationFieldMapping.isValid(entry.getKey())) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters geolocation_field_mapping value is not correct"); } } + } } - @Override public String functionName() { return "GEOIP_LOOKUP"; diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/Hmac.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/Hmac.java new file mode 100644 index 0000000..098cdef --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/Hmac.java @@ -0,0 +1,104 @@ +package com.geedgenetworks.core.udf; + +import cn.hutool.crypto.digest.HMac; +import cn.hutool.crypto.digest.HmacAlgorithm; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.exception.CommonErrorCode; +import com.geedgenetworks.common.exception.GrootStreamRuntimeException; +import com.geedgenetworks.common.udf.ScalarFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.utils.StringUtil; +import lombok.extern.slf4j.Slf4j; +import org.apache.flink.api.common.functions.RuntimeContext; + +@Slf4j +public class Hmac implements ScalarFunction { + + private String lookupFieldName; + private String outputFieldName; + private String outputFormat; + private HMac hMac; + + @Override + public void open(RuntimeContext runtimeContext, UDFContext udfContext) { + checkUdfContext(udfContext); + String secretKey = udfContext.getParameters().get("secret_key").toString(); + String algorithm = "sha256"; + if (udfContext.getParameters().containsKey("algorithm")) { + algorithm = udfContext.getParameters().get("algorithm").toString(); + } + this.hMac = new HMac(getHmacAlgorithm(algorithm), secretKey.getBytes()); + this.lookupFieldName = udfContext.getLookupFields().get(0); + this.outputFieldName = udfContext.getOutputFields().get(0); + this.outputFormat = "base64"; + if (udfContext.getParameters().containsKey("output_format")) { + this.outputFormat = udfContext.getParameters().get("output_format").toString(); + } + } + + @Override + public Event evaluate(Event event) { + String encodeResult = ""; + String message = (String) event.getExtractedFields().get(lookupFieldName); + if (StringUtil.isNotBlank(message)) { + switch (outputFormat) { + case "hex": + encodeResult = hMac.digestHex(message); + break; + case "base64": + encodeResult = hMac.digestBase64(message, false); + break; + default: + encodeResult = hMac.digestBase64(message, false); + break; + } + } + event.getExtractedFields().put(outputFieldName, encodeResult); + return event; + } + + @Override + public String functionName() { + return "HMAC"; + } + + @Override + public void close() { + + } + + private void checkUdfContext(UDFContext udfContext) { + if (udfContext.getParameters() == null || udfContext.getOutputFields() == null) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); + } + if (udfContext.getLookupFields().size() != 1) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function lookup fields only support 1 value"); + } + if (udfContext.getOutputFields().size() != 1) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value"); + } + if (!udfContext.getParameters().containsKey("secret_key")) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters must contains secret_key"); + } + } + + private String getHmacAlgorithm(String algorithm) { + if (StringUtil.containsIgnoreCase(algorithm, "sha256")) { + return HmacAlgorithm.HmacSHA256.getValue(); + } else if (StringUtil.containsIgnoreCase(algorithm, "sha1")) { + return HmacAlgorithm.HmacSHA1.getValue(); + } else if (StringUtil.containsIgnoreCase(algorithm, "md5")) { + return HmacAlgorithm.HmacMD5.getValue(); + } else if (StringUtil.containsIgnoreCase(algorithm, "sha384")) { + return HmacAlgorithm.HmacSHA384.getValue(); + } else if (StringUtil.containsIgnoreCase(algorithm, "sha512")) { + return HmacAlgorithm.HmacSHA512.getValue(); + } else if (StringUtil.containsIgnoreCase(algorithm, "sm3")) { + return HmacAlgorithm.HmacSM3.getValue(); + } else if (StringUtil.containsIgnoreCase(algorithm, "sm4")) { + return HmacAlgorithm.SM4CMAC.getValue(); + } else { + return HmacAlgorithm.HmacSHA256.getValue(); + } + } +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/JsonExtract.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/JsonExtract.java index f78b952..57fe847 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/JsonExtract.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/JsonExtract.java @@ -17,20 +17,20 @@ public class JsonExtract implements ScalarFunction { @Override public void open(RuntimeContext runtimeContext, UDFContext udfContext) { - if(udfContext.getLookup_fields()==null || udfContext.getOutput_fields()==null || udfContext.getParameters() == null){ + if(udfContext.getLookupFields()==null || udfContext.getOutputFields()==null || udfContext.getParameters() == null){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } - if(udfContext.getLookup_fields().size() != 1){ + if(udfContext.getLookupFields().size() != 1){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function lookup fields only support 1 value"); } - if(udfContext.getOutput_fields().size() != 1){ + if(udfContext.getOutputFields().size() != 1){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value"); } if(!udfContext.getParameters().containsKey("value_expression")){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters must containkey value_expression"); } - this.lookupFieldName = udfContext.getLookup_fields().get(0); - this.outputFieldName = udfContext.getOutput_fields().get(0); + this.lookupFieldName = udfContext.getLookupFields().get(0); + this.outputFieldName = udfContext.getOutputFields().get(0); this.expression =udfContext.getParameters().get("value_expression").toString(); } @@ -39,14 +39,9 @@ public class JsonExtract implements ScalarFunction { @Override public Event evaluate(Event event) { - if (event.getExtractedFields().containsKey(lookupFieldName)) { - - String result = - (String) - JsonPathUtil.analysis( - event.getExtractedFields() - .get(lookupFieldName) - .toString(),expression); + Object valueObj = event.getExtractedFields().get(lookupFieldName); + if (valueObj!=null) { + String result = (String) JsonPathUtil.analysis(valueObj.toString(),expression); event.getExtractedFields().put(outputFieldName, result); } return event; diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/PathCombine.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/PathCombine.java index 874735d..0141a46 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/PathCombine.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/PathCombine.java @@ -55,7 +55,7 @@ public class PathCombine implements ScalarFunction { } } } - this.outputFieldName = udfContext.getOutput_fields().get(0); + this.outputFieldName = udfContext.getOutputFields().get(0); } @@ -65,9 +65,9 @@ public class PathCombine implements ScalarFunction { List<String> pathBuilder = new LinkedList<>(); for (Map.Entry<String, String> entry : pathParameters.entrySet()) { if (entry.getValue().isEmpty()) { - - if(event.getExtractedFields().containsKey(entry.getKey())) { - pathBuilder.add(event.getExtractedFields().get(entry.getKey()).toString()); + Object valueObj = event.getExtractedFields().get(entry.getKey()); + if (valueObj!=null) { + pathBuilder.add(valueObj.toString()); } else { return event; diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/SnowflakeId.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/SnowflakeId.java index 42a19e6..b206f3b 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/SnowflakeId.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/SnowflakeId.java @@ -17,15 +17,15 @@ public class SnowflakeId implements Serializable, ScalarFunction { @Override public void open(RuntimeContext runtimeContext, UDFContext udfContext) { - if(udfContext.getOutput_fields()==null || udfContext.getParameters() == null ){ + if(udfContext.getOutputFields()==null || udfContext.getParameters() == null ){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } - if(udfContext.getOutput_fields().size() != 1){ + if(udfContext.getOutputFields().size() != 1){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value"); } String data_center_id_num = udfContext.getParameters().getOrDefault("data_center_id_num","0").toString();//转为数字 snowflakeIdUtils = new SnowflakeIdUtils(Integer.parseInt(data_center_id_num),runtimeContext.getIndexOfThisSubtask()); - this.outputFieldName = udfContext.getOutput_fields().get(0); + this.outputFieldName = udfContext.getOutputFields().get(0); } @Override diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/StringJoiner.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/StringJoiner.java index df5c5b4..7e4ab68 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/StringJoiner.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/StringJoiner.java @@ -21,14 +21,14 @@ public class StringJoiner implements ScalarFunction { @Override public void open(RuntimeContext runtimeContext, UDFContext udfContext) { - if(udfContext.getLookup_fields()==null || udfContext.getOutput_fields()==null || udfContext.getParameters() == null){ + if(udfContext.getLookupFields()==null || udfContext.getOutputFields()==null || udfContext.getParameters() == null){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } - if(udfContext.getOutput_fields().size() != 1){ + if(udfContext.getOutputFields().size() != 1){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value"); } - this.lookupFieldNames = udfContext.getLookup_fields(); - this.outputFieldName = udfContext.getOutput_fields().get(0); + this.lookupFieldNames = udfContext.getLookupFields(); + this.outputFieldName = udfContext.getOutputFields().get(0); this.separator =udfContext.getParameters().getOrDefault("separator","").toString().trim(); this.prefix =udfContext.getParameters().getOrDefault("prefix","").toString().trim(); this.suffix =udfContext.getParameters().getOrDefault("suffix","").toString().trim(); diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/UnixTimestampConverter.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/UnixTimestampConverter.java index bdb41e0..62c4dfa 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/UnixTimestampConverter.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/UnixTimestampConverter.java @@ -23,13 +23,13 @@ public class UnixTimestampConverter implements ScalarFunction { public void open(RuntimeContext runtimeContext, UDFContext udfContext) { this.udfContext = udfContext; - if(udfContext.getLookup_fields()==null || udfContext.getOutput_fields()==null || udfContext.getParameters() == null){ + if(udfContext.getLookupFields()==null || udfContext.getOutputFields()==null || udfContext.getParameters() == null){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } - if(udfContext.getOutput_fields().size() != 1){ + if(udfContext.getOutputFields().size() != 1){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value"); } - if(udfContext.getLookup_fields().size() != 1){ + if(udfContext.getLookupFields().size() != 1){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value"); } if(!udfContext.getParameters().containsKey("precision")){ @@ -51,26 +51,22 @@ public class UnixTimestampConverter implements ScalarFunction { else{ this.interval = Long.parseLong(udfContext.getParameters().get("interval").toString()); } - this.lookupFieldName = udfContext.getLookup_fields().get(0); - this.outputFieldName = udfContext.getOutput_fields().get(0); + this.lookupFieldName = udfContext.getLookupFields().get(0); + this.outputFieldName = udfContext.getOutputFields().get(0); } @Override public Event evaluate(Event event) { - - if(event.getExtractedFields().containsKey(lookupFieldName)) { - Long timestamp = Long.parseLong(event.getExtractedFields().get(lookupFieldName).toString()); - Instant instant = null; - if (String.valueOf(timestamp).length() == 13) { - // 时间戳长度大于10,表示为毫秒级时间戳 + Object valueObj = event.getExtractedFields().get(lookupFieldName); + if (valueObj!=null) { + Long timestamp = Long.parseLong(valueObj.toString()); + Instant instant ; + if (String.valueOf(timestamp).length() > 10) { instant = Instant.ofEpochMilli(timestamp); - } else if (String.valueOf(timestamp).length() == 10) { - // 时间戳长度小于等于10,表示为秒级时间戳 - instant = Instant.ofEpochSecond(timestamp); } else { - return event; + instant = Instant.ofEpochSecond(timestamp); } switch (precision) { case "seconds": diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/AbstractKnowledgeScalarFunction.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/AbstractKnowledgeScalarFunction.java index 7c4aca2..e54b612 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/AbstractKnowledgeScalarFunction.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/AbstractKnowledgeScalarFunction.java @@ -50,14 +50,14 @@ public abstract class AbstractKnowledgeScalarFunction implements ScalarFunction registerKnowledges(); - this.lookupFieldName = udfContext.getLookup_fields().get(0); - if (udfContext.getOutput_fields() != null && udfContext.getOutput_fields().size() > 0) { - this.outputFieldName = udfContext.getOutput_fields().get(0); + this.lookupFieldName = udfContext.getLookupFields().get(0); + if (udfContext.getOutputFields() != null && udfContext.getOutputFields().size() > 0) { + this.outputFieldName = udfContext.getOutputFields().get(0); } } private String buildMetricPrefix(UDFContext udfContext) { - return functionName().toLowerCase() + "_" + udfContext.getLookup_fields().get(0); + return functionName().toLowerCase() + "_" + udfContext.getLookupFields().get(0); } protected abstract void registerKnowledges(); diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/ArrayElementsPrepend.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/ArrayElementsPrepend.java index a591884..de64073 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/ArrayElementsPrepend.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/ArrayElementsPrepend.java @@ -24,8 +24,8 @@ public class ArrayElementsPrepend implements ScalarFunction { @Override public void open(RuntimeContext runtimeContext, UDFContext udfContext) { this.prefix = udfContext.getParameters().get("prefix").toString(); - this.lookupFieldName = udfContext.getLookup_fields().get(0); - this.outputFieldName = udfContext.getOutput_fields().get(0); + this.lookupFieldName = udfContext.getLookupFields().get(0); + this.outputFieldName = udfContext.getOutputFields().get(0); } @Override diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/BaseStationLookup.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/BaseStationLookup.java new file mode 100644 index 0000000..191edd5 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/BaseStationLookup.java @@ -0,0 +1,81 @@ +package com.geedgenetworks.core.udf.cn; + +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.exception.CommonErrorCode; +import com.geedgenetworks.common.exception.GrootStreamRuntimeException; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.core.udf.knowlegdebase.KnowledgeBaseUpdateJob; +import com.geedgenetworks.core.udf.knowlegdebase.handler.BaseStationKnowledgeBaseHandler; +import org.apache.flink.api.common.functions.RuntimeContext; + +import java.util.List; +import java.util.Map; + +/** + * @author gujinkai + * @version 1.0 + * @date 2024/8/19 15:21 + */ +public class BaseStationLookup extends AbstractKnowledgeScalarFunction { + + private String cellIdFieldName; + + private String longitudeFieldName; + + private String latitudeFieldName; + + private BaseStationKnowledgeBaseHandler knowledgeBaseHandler; + + @Override + public void open(RuntimeContext runtimeContext, UDFContext udfContext) { + super.open(runtimeContext, udfContext); + if (udfContext.getLookupFields() == null || udfContext.getOutputFields() == null || udfContext.getParameters() == null) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); + } + + int lookupFieldsSize = 1; + if (udfContext.getLookupFields().size() != lookupFieldsSize) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "lookup_fields must contain field: cell_id"); + } + + int outputFieldsSize = 2; + if (udfContext.getOutputFields().size() != outputFieldsSize) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "output_fields must contain two fields: longitude and latitude"); + } + + List<String> lookupFields = udfContext.getLookupFields(); + cellIdFieldName = lookupFields.get(0); + List<String> outputFields = udfContext.getOutputFields(); + longitudeFieldName = outputFields.get(0); + latitudeFieldName = outputFields.get(1); + } + + @Override + public Event evaluate(Event event) { + Map<String, Object> extractedFields = event.getExtractedFields(); + if (extractedFields == null || extractedFields.get(cellIdFieldName) == null) { + return event; + } + BaseStationKnowledgeBaseHandler.BaseStationLocation lookup = knowledgeBaseHandler.lookup(extractedFields.get(cellIdFieldName).toString()); + if (lookup != null) { + extractedFields.put(longitudeFieldName, lookup.getLongitude()); + extractedFields.put(latitudeFieldName, lookup.getLatitude()); + } + return event; + } + + @Override + public String functionName() { + return "BASE_STATION_LOOKUP"; + } + + @Override + public void close() { + } + + @Override + protected void registerKnowledges() { + knowledgeBaseHandler = BaseStationKnowledgeBaseHandler.getInstance(); + KnowledgeBaseUpdateJob.registerKnowledgeBase(knowledgeBaseHandler, knowledgeBaseConfigs.get(0)); + } +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/FieldsMerge.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/FieldsMerge.java index 6cd0c6b..f4338fc 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/FieldsMerge.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/FieldsMerge.java @@ -22,8 +22,8 @@ public class FieldsMerge implements ScalarFunction { @Override public void open(RuntimeContext runtimeContext, UDFContext udfContext) { - this.lookupFieldNames = udfContext.getLookup_fields(); - this.outputFieldName = udfContext.getOutput_fields().get(0); + this.lookupFieldNames = udfContext.getLookupFields(); + this.outputFieldName = udfContext.getOutputFields().get(0); } @Override diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/H3CellLookup.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/H3CellLookup.java new file mode 100644 index 0000000..7389f4a --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/H3CellLookup.java @@ -0,0 +1,84 @@ +package com.geedgenetworks.core.udf.cn; + +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.exception.CommonErrorCode; +import com.geedgenetworks.common.exception.GrootStreamRuntimeException; +import com.geedgenetworks.common.udf.ScalarFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.uber.h3core.H3Core; +import org.apache.flink.api.common.functions.RuntimeContext; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +/** + * @author gujinkai + * @version 1.0 + * @date 2024/8/19 14:50 + */ +public class H3CellLookup implements ScalarFunction { + + private String longitudeFieldName; + + private String latitudeFieldName; + + private String outputFieldName; + + private int res; + + private H3Core h3; + + + @Override + public void open(RuntimeContext runtimeContext, UDFContext udfContext) { + if (udfContext.getLookupFields() == null || udfContext.getOutputFields() == null || udfContext.getParameters() == null) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); + } + + int lookupFieldsSize = 2; + if (udfContext.getLookupFields().size() != lookupFieldsSize) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "lookup_fields must contain two fields: longitude and latitude"); + } + + String resolution = "resolution"; + int maxResolution = 15; + if (!udfContext.getParameters().containsKey(resolution) || !(udfContext.getParameters().get(resolution) instanceof Integer) || (int) udfContext.getParameters().get(resolution) < 0 || (int) udfContext.getParameters().get(resolution) > maxResolution) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "parameters must contain key resolution and the value is between 0 and 15"); + } + + List<String> lookupFields = udfContext.getLookupFields(); + longitudeFieldName = lookupFields.get(0); + latitudeFieldName = lookupFields.get(1); + outputFieldName = udfContext.getOutputFields().get(0); + res = (int) udfContext.getParameters().get("resolution"); + try { + h3 = H3Core.newInstance(); + } catch (IOException io) { + throw new GrootStreamRuntimeException(CommonErrorCode.FILE_OPERATION_ERROR, "Failed to create H3Core instance"); + } + } + + @Override + public Event evaluate(Event event) { + Map<String, Object> extractedFields = event.getExtractedFields(); + if (extractedFields == null || extractedFields.get(longitudeFieldName) == null || extractedFields.get(latitudeFieldName) == null) { + return event; + } + if (!(extractedFields.get(longitudeFieldName) instanceof Double) || !(extractedFields.get(latitudeFieldName) instanceof Double)) { + return event; + } + String cellAddress = h3.latLngToCellAddress((double) extractedFields.get(latitudeFieldName), (double) extractedFields.get(longitudeFieldName), res); + extractedFields.put(outputFieldName, cellAddress); + return event; + } + + @Override + public String functionName() { + return "H3_CELL_LOOKUP"; + } + + @Override + public void close() { + } +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/IntelligenceIndicatorLookup.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/IntelligenceIndicatorLookup.java index 9f14bd2..857ae74 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/IntelligenceIndicatorLookup.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/cn/IntelligenceIndicatorLookup.java @@ -57,6 +57,17 @@ public class IntelligenceIndicatorLookup extends AbstractKnowledgeScalarFunction } } break; + case "SUBSCRIBER_TO_TAG": + List<String> subscriberTags = knowledgeBaseHandler.lookupBySubscriber(lookupValue); + if (subscriberTags != null && subscriberTags.size() > 0) { + hitCounter.inc(); + if (event.getExtractedFields().get(outputFieldName) != null && event.getExtractedFields().get(outputFieldName) instanceof List) { + ((List<String>) event.getExtractedFields().get(outputFieldName)).addAll(subscriberTags); + } else { + event.getExtractedFields().put(outputFieldName, subscriberTags); + } + } + break; default: logger.error("unknown option :" + option); break; diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/AES128GCM96.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/AES128GCM96.java new file mode 100644 index 0000000..90669b3 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/AES128GCM96.java @@ -0,0 +1,84 @@ +package com.geedgenetworks.core.udf.encrypt; + +import cn.hutool.core.util.RandomUtil; +import com.geedgenetworks.core.pojo.KmsKey; + +import javax.crypto.Cipher; +import javax.crypto.spec.GCMParameterSpec; +import javax.crypto.spec.SecretKeySpec; +import java.util.Base64; + +public class AES128GCM96 implements EncryptionAlgorithm { + private static final String IDENTIFIER = "aes-128-gcm96"; + private static final String ALGORITHM = "AES"; + private static final String TRANSFORMATION = "AES/GCM/NoPadding"; + private static final int GCM_TAG_LENGTH = 128; + private static final int GCM_96_NONCE_LENGTH = 12; + private static final int SECRET_KEY_LENGTH = 16; + private static final byte[] DEFAULT_SECRET_KEY = ".geedgenetworks.".getBytes(); + + private final Cipher cipher; + private KmsKey kmsKey; + + public AES128GCM96() throws Exception { + this.cipher = Cipher.getInstance(TRANSFORMATION); + this.kmsKey = new KmsKey(DEFAULT_SECRET_KEY, 1); + } + + @Override + public String getIdentifier() { + return IDENTIFIER; + } + + @Override + public int getSecretKeyLength() { + return SECRET_KEY_LENGTH; + } + + @Override + public KmsKey getKmsKey() { + return kmsKey; + } + + @Override + public void setKmsKey(KmsKey kmsKey) { + this.kmsKey = kmsKey; + } + + @Override + public String encrypt(String content) { + String encryptedString = ""; + try { + byte[] nonce = RandomUtil.randomBytes(GCM_96_NONCE_LENGTH); + GCMParameterSpec gcmSpec = new GCMParameterSpec(GCM_TAG_LENGTH, nonce); + cipher.init(Cipher.ENCRYPT_MODE, new SecretKeySpec(kmsKey.getKeyData(), ALGORITHM), gcmSpec); + byte[] encryptedBytes = cipher.doFinal(content.getBytes()); + byte[] combinedBytes = new byte[GCM_96_NONCE_LENGTH + encryptedBytes.length]; + System.arraycopy(nonce, 0, combinedBytes, 0, GCM_96_NONCE_LENGTH); + System.arraycopy(encryptedBytes, 0, combinedBytes, GCM_96_NONCE_LENGTH, encryptedBytes.length); + encryptedString = Base64.getEncoder().encodeToString(combinedBytes); + } catch (Exception e) { + throw new RuntimeException(e); + } + return encryptedString; + } + + @Override + public String decrypt(String content) { + String decryptedString = ""; + try { + byte[] nonce = RandomUtil.randomBytes(GCM_96_NONCE_LENGTH); + byte[] combined = Base64.getDecoder().decode(content); + byte[] encryptedBytes = new byte[combined.length - GCM_96_NONCE_LENGTH]; + System.arraycopy(combined, 0, nonce, 0, GCM_96_NONCE_LENGTH); + System.arraycopy(combined, GCM_96_NONCE_LENGTH, encryptedBytes, 0, encryptedBytes.length); + GCMParameterSpec gcmSpec = new GCMParameterSpec(GCM_TAG_LENGTH, nonce); + cipher.init(Cipher.DECRYPT_MODE, new SecretKeySpec(kmsKey.getKeyData(), ALGORITHM), gcmSpec); + byte[] decryptedBytes = cipher.doFinal(encryptedBytes); + decryptedString = new String(decryptedBytes); + } catch (Exception e) { + throw new RuntimeException(e); + } + return decryptedString; + } +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/AES256GCM96.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/AES256GCM96.java new file mode 100644 index 0000000..0306616 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/AES256GCM96.java @@ -0,0 +1,84 @@ +package com.geedgenetworks.core.udf.encrypt; + +import cn.hutool.core.util.RandomUtil; +import com.geedgenetworks.core.pojo.KmsKey; + +import javax.crypto.Cipher; +import javax.crypto.spec.GCMParameterSpec; +import javax.crypto.spec.SecretKeySpec; +import java.util.Base64; + +public class AES256GCM96 implements EncryptionAlgorithm { + private static final String IDENTIFIER = "aes-256-gcm96"; + private static final String ALGORITHM = "AES"; + private static final String TRANSFORMATION = "AES/GCM/NoPadding"; + private static final int GCM_TAG_LENGTH = 128; + private static final int GCM_96_NONCE_LENGTH = 12; + private static final int SECRET_KEY_LENGTH = 32; + private static final byte[] DEFAULT_SECRET_KEY = ".........geedgenetworks.........".getBytes(); + + private final Cipher cipher; + private KmsKey kmsKey; + + public AES256GCM96() throws Exception { + this.cipher = Cipher.getInstance(TRANSFORMATION); + this.kmsKey = new KmsKey(DEFAULT_SECRET_KEY, 1); + } + + @Override + public String getIdentifier() { + return IDENTIFIER; + } + + @Override + public int getSecretKeyLength() { + return SECRET_KEY_LENGTH; + } + + @Override + public KmsKey getKmsKey() { + return kmsKey; + } + + @Override + public void setKmsKey(KmsKey kmsKey) { + this.kmsKey = kmsKey; + } + + @Override + public String encrypt(String content) { + String encryptedString = ""; + try { + byte[] nonce = RandomUtil.randomBytes(GCM_96_NONCE_LENGTH); + GCMParameterSpec gcmSpec = new GCMParameterSpec(GCM_TAG_LENGTH, nonce); + cipher.init(Cipher.ENCRYPT_MODE, new SecretKeySpec(kmsKey.getKeyData(), ALGORITHM), gcmSpec); + byte[] encryptedBytes = cipher.doFinal(content.getBytes()); + byte[] combinedBytes = new byte[GCM_96_NONCE_LENGTH + encryptedBytes.length]; + System.arraycopy(nonce, 0, combinedBytes, 0, GCM_96_NONCE_LENGTH); + System.arraycopy(encryptedBytes, 0, combinedBytes, GCM_96_NONCE_LENGTH, encryptedBytes.length); + encryptedString = Base64.getEncoder().encodeToString(combinedBytes); + } catch (Exception e) { + throw new RuntimeException(e); + } + return encryptedString; + } + + @Override + public String decrypt(String content) { + String decryptedString = ""; + try { + byte[] nonce = RandomUtil.randomBytes(GCM_96_NONCE_LENGTH); + byte[] combined = Base64.getDecoder().decode(content); + byte[] encryptedBytes = new byte[combined.length - GCM_96_NONCE_LENGTH]; + System.arraycopy(combined, 0, nonce, 0, GCM_96_NONCE_LENGTH); + System.arraycopy(combined, GCM_96_NONCE_LENGTH, encryptedBytes, 0, encryptedBytes.length); + GCMParameterSpec gcmSpec = new GCMParameterSpec(GCM_TAG_LENGTH, nonce); + cipher.init(Cipher.DECRYPT_MODE, new SecretKeySpec(kmsKey.getKeyData(), ALGORITHM), gcmSpec); + byte[] decryptedBytes = cipher.doFinal(encryptedBytes); + decryptedString = new String(decryptedBytes); + } catch (Exception e) { + throw new RuntimeException(e); + } + return decryptedString; + } +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/EncryptionAlgorithm.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/EncryptionAlgorithm.java new file mode 100644 index 0000000..3fc4e74 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/EncryptionAlgorithm.java @@ -0,0 +1,17 @@ +package com.geedgenetworks.core.udf.encrypt; + +import com.geedgenetworks.core.pojo.KmsKey; + +public interface EncryptionAlgorithm { + String getIdentifier(); + + int getSecretKeyLength(); + + KmsKey getKmsKey(); + + void setKmsKey(KmsKey kmsKey); + + String encrypt(String content); + + String decrypt(String content); +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/SM4GCM96.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/SM4GCM96.java new file mode 100644 index 0000000..f4ad0a2 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/encrypt/SM4GCM96.java @@ -0,0 +1,84 @@ +package com.geedgenetworks.core.udf.encrypt; + +import cn.hutool.core.util.RandomUtil; +import com.geedgenetworks.core.pojo.KmsKey; + +import javax.crypto.Cipher; +import javax.crypto.spec.GCMParameterSpec; +import javax.crypto.spec.SecretKeySpec; +import java.util.Base64; + +public class SM4GCM96 implements EncryptionAlgorithm { + private static final String IDENTIFIER = "sm4-gcm96"; + private static final String ALGORITHM = "SM4"; + private static final String TRANSFORMATION = "SM4/GCM/NoPadding"; + private static final int GCM_TAG_LENGTH = 128; + private static final int GCM_96_NONCE_LENGTH = 12; + private static final int SECRET_KEY_LENGTH = 16; + private static final byte[] DEFAULT_SECRET_KEY = ".geedgenetworks.".getBytes(); + + private final Cipher cipher; + private KmsKey kmsKey; + + public SM4GCM96() throws Exception { + this.cipher = Cipher.getInstance(TRANSFORMATION); + this.kmsKey = new KmsKey(DEFAULT_SECRET_KEY, 1); + } + + @Override + public String getIdentifier() { + return IDENTIFIER; + } + + @Override + public int getSecretKeyLength() { + return SECRET_KEY_LENGTH; + } + + @Override + public KmsKey getKmsKey() { + return kmsKey; + } + + @Override + public void setKmsKey(KmsKey kmsKey) { + this.kmsKey = kmsKey; + } + + @Override + public String encrypt(String content) { + String encryptedString = ""; + try { + byte[] nonce = RandomUtil.randomBytes(GCM_96_NONCE_LENGTH); + GCMParameterSpec gcmSpec = new GCMParameterSpec(GCM_TAG_LENGTH, nonce); + cipher.init(Cipher.ENCRYPT_MODE, new SecretKeySpec(kmsKey.getKeyData(), ALGORITHM), gcmSpec); + byte[] encryptedBytes = cipher.doFinal(content.getBytes()); + byte[] combinedBytes = new byte[GCM_96_NONCE_LENGTH + encryptedBytes.length]; + System.arraycopy(nonce, 0, combinedBytes, 0, GCM_96_NONCE_LENGTH); + System.arraycopy(encryptedBytes, 0, combinedBytes, GCM_96_NONCE_LENGTH, encryptedBytes.length); + encryptedString = Base64.getEncoder().encodeToString(combinedBytes); + } catch (Exception e) { + throw new RuntimeException(e); + } + return encryptedString; + } + + @Override + public String decrypt(String content) { + String decryptedString = ""; + try { + byte[] nonce = RandomUtil.randomBytes(GCM_96_NONCE_LENGTH); + byte[] combined = Base64.getDecoder().decode(content); + byte[] encryptedBytes = new byte[combined.length - GCM_96_NONCE_LENGTH]; + System.arraycopy(combined, 0, nonce, 0, GCM_96_NONCE_LENGTH); + System.arraycopy(combined, GCM_96_NONCE_LENGTH, encryptedBytes, 0, encryptedBytes.length); + GCMParameterSpec gcmSpec = new GCMParameterSpec(GCM_TAG_LENGTH, nonce); + cipher.init(Cipher.DECRYPT_MODE, new SecretKeySpec(kmsKey.getKeyData(), ALGORITHM), gcmSpec); + byte[] decryptedBytes = cipher.doFinal(encryptedBytes); + decryptedString = new String(decryptedBytes); + } catch (Exception e) { + throw new RuntimeException(e); + } + return decryptedString; + } +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/AbstractSingleKnowledgeBaseHandler.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/AbstractSingleKnowledgeBaseHandler.java index 6ac292c..759ab38 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/AbstractSingleKnowledgeBaseHandler.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/AbstractSingleKnowledgeBaseHandler.java @@ -31,6 +31,13 @@ public abstract class AbstractSingleKnowledgeBaseHandler extends AbstractKnowled if ("http".equals(knowledgeBaseConfig.getFsType())) { this.knowledgeMetedataCache = getMetadata(knowledgeBaseConfig.getFsType(), knowledgeBaseConfig.getFsPath(), knowledgeBaseConfig.getFiles().get(0)); } + if ("local".equals(knowledgeBaseConfig.getFsType())) { + knowledgeMetedataCache = new KnowLedgeBaseFileMeta(); + knowledgeMetedataCache.setIsValid(1); + knowledgeMetedataCache.setFormat(knowledgeBaseConfig.getFiles().get(0).substring(knowledgeBaseConfig.getFiles().get(0).lastIndexOf(".") + 1)); + knowledgeMetedataCache.setName(knowledgeBaseConfig.getFiles().get(0)); + knowledgeMetedataCache.setPath(knowledgeBaseConfig.getFsPath() + knowledgeBaseConfig.getFiles().get(0)); + } return buildKnowledgeBase(); } @@ -81,11 +88,8 @@ public abstract class AbstractSingleKnowledgeBaseHandler extends AbstractKnowled case "aes": result = AESUtil.decrypt(data, AES_KEY); break; - case "csv": - result = data; - break; default: - logger.error("unknown format: " + knowledgeMetedataCache.getFormat()); + result = data; } } catch (Exception e) { logger.error("decrypt error", e); @@ -102,6 +106,10 @@ public abstract class AbstractSingleKnowledgeBaseHandler extends AbstractKnowled knowledgeMetedataCache = knowledgeMetedata; return true; } + if (knowledgeMetedata.getIsValid() != knowledgeMetedataCache.getIsValid()) { + knowledgeMetedataCache = knowledgeMetedata; + return true; + } if (knowledgeMetedataCache.getSha256().equals(knowledgeMetedata.getSha256())) { return false; } else { diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/AsnKnowledgeBaseHandler.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/AsnKnowledgeBaseHandler.java index a1927db..a0b9ce5 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/AsnKnowledgeBaseHandler.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/AsnKnowledgeBaseHandler.java @@ -117,6 +117,7 @@ public class AsnKnowledgeBaseHandler extends AbstractKnowledgeBaseHandler { } } catch (Exception e) { + log.error("Current class path {}", this.getClass().getProtectionDomain().getCodeSource().getLocation().getPath()); log.error("File {} operation failed. {} ", knowledgeBaseConfig.getFiles().get(i), e.getMessage()); return false; } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/BaseStationKnowledgeBaseHandler.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/BaseStationKnowledgeBaseHandler.java new file mode 100644 index 0000000..0eeae53 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/BaseStationKnowledgeBaseHandler.java @@ -0,0 +1,90 @@ +package com.geedgenetworks.core.udf.knowlegdebase.handler; + +import com.geedgenetworks.core.utils.cn.csv.HighCsvReader; +import lombok.Data; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.InputStreamReader; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * @author gujinkai + * @version 1.0 + * @date 2024/8/26 16:27 + */ +public class BaseStationKnowledgeBaseHandler extends AbstractSingleKnowledgeBaseHandler { + + private static final Logger logger = LoggerFactory.getLogger(BaseStationKnowledgeBaseHandler.class); + + private HashMap<String, BaseStationLocation> baseStationLocationHashMap = new HashMap<>(); + + private BaseStationKnowledgeBaseHandler() { + } + + private static final class InstanceHolder { + private static final BaseStationKnowledgeBaseHandler instance = new BaseStationKnowledgeBaseHandler(); + } + + public static BaseStationKnowledgeBaseHandler getInstance() { + return BaseStationKnowledgeBaseHandler.InstanceHolder.instance; + } + + @Override + public Boolean buildKnowledgeBase() { + try { + List<String> needColumns = new ArrayList<>(); + needColumns.add("cell_id"); + needColumns.add("longitude"); + needColumns.add("latitude"); + byte[] content = downloadFile(); + HighCsvReader highCsvReader = new HighCsvReader(new InputStreamReader(new ByteArrayInputStream(content)), needColumns); + HashMap<String, BaseStationLocation> newBaseStationLocationHashMap = new HashMap<>((int) (highCsvReader.getLineNumber() / 0.75F + 1.0F)); + HighCsvReader.CsvIterator iterator = highCsvReader.getIterator(); + while (iterator.hasNext()) { + Map<String, String> line = iterator.next(); + try { + String cellId = line.get("cell_id"); + BaseStationLocation baseStationLocation = new BaseStationLocation(); + baseStationLocation.setLongitude(parseDouble(line.get("longitude"))); + baseStationLocation.setLatitude(parseDouble(line.get("latitude"))); + newBaseStationLocationHashMap.put(cellId, baseStationLocation); + } catch (Exception lineException) { + logger.error(this.getClass().getSimpleName() + " line: " + line.toString() + " parse error:" + lineException, lineException); + } + } + baseStationLocationHashMap = newBaseStationLocationHashMap; + } catch (Exception e) { + logger.error(this.getClass().getSimpleName() + " update error", e); + return false; + } + return true; + } + + public BaseStationLocation lookup(String ecgi) { + return baseStationLocationHashMap.get(ecgi); + } + + public void close() { + baseStationLocationHashMap.clear(); + baseStationLocationHashMap = null; + } + + private double parseDouble(String str) { + try { + return Double.parseDouble(str); + } catch (Exception e) { + return 0; + } + } + + @Data + public static final class BaseStationLocation { + private double longitude; + private double latitude; + } +}
\ No newline at end of file diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/IntelligenceIndicatorKnowledgeBaseHandler.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/IntelligenceIndicatorKnowledgeBaseHandler.java index 94fdae1..093dd1d 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/IntelligenceIndicatorKnowledgeBaseHandler.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/knowlegdebase/handler/IntelligenceIndicatorKnowledgeBaseHandler.java @@ -31,6 +31,8 @@ public class IntelligenceIndicatorKnowledgeBaseHandler extends AbstractSingleKno // *开头,模糊匹配 private Trie<String> domainSuffix = new Trie<>(); + private HashMap<String, List<String>> subscriberTagMap = new HashMap<>(); + private IntelligenceIndicatorKnowledgeBaseHandler() { } @@ -51,12 +53,14 @@ public class IntelligenceIndicatorKnowledgeBaseHandler extends AbstractSingleKno needColumns.add("ip1"); needColumns.add("ip2"); needColumns.add("domain"); + needColumns.add("subscriber"); needColumns.add("tags"); byte[] content = downloadFile(); HighCsvReader highCsvReader = new HighCsvReader(new InputStreamReader(new ByteArrayInputStream(content)), needColumns); TreeRangeMap<IPAddress, List<String>> newIpTagMap = TreeRangeMap.create(); HashMap<String, List<String>> newDomainMap = new HashMap<>((int) (highCsvReader.getLineNumber() / 0.75F + 1.0F)); Trie<String> newDomainSuffix = new Trie<>(); + HashMap<String, List<String>> newSubscriberTagMap = new HashMap<>(); HighCsvReader.CsvIterator iterator = highCsvReader.getIterator(); while (iterator.hasNext()) { Map<String, String> line = iterator.next(); @@ -66,6 +70,7 @@ public class IntelligenceIndicatorKnowledgeBaseHandler extends AbstractSingleKno String ip1 = line.get("ip1"); String ip2 = line.get("ip2"); String domain = line.get("domain"); + String subscriberId = line.get("subscriber"); List<String> tags = Arrays.asList(line.get("tags").split(",")); if ("IP".equals(type)) { @@ -121,6 +126,8 @@ public class IntelligenceIndicatorKnowledgeBaseHandler extends AbstractSingleKno } else { logger.warn("intelligence indicator find unknown domain: " + domain); } + } else if ("Subscriber".equals(type)) { + newSubscriberTagMap.computeIfAbsent(subscriberId, k -> new ArrayList<>()).addAll(tags); } } catch (Exception lineException) { logger.error(this.getClass().getSimpleName() + " line: " + line.toString() + " parse error:" + lineException, lineException); @@ -129,6 +136,7 @@ public class IntelligenceIndicatorKnowledgeBaseHandler extends AbstractSingleKno ipTagMap = newIpTagMap; domainTagMap = newDomainMap; domainSuffix = newDomainSuffix; + subscriberTagMap = newSubscriberTagMap; } catch (Exception e) { logger.error(this.getClass().getSimpleName() + " update error", e); return false; @@ -155,6 +163,10 @@ public class IntelligenceIndicatorKnowledgeBaseHandler extends AbstractSingleKno return result; } + public List<String> lookupBySubscriber(String subscriber) { + return subscriberTagMap.computeIfAbsent(subscriber, k -> new ArrayList<>()); + } + @Override public void close() { ipTagMap.clear(); diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectList.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectList.java index 3921ee2..8219fbd 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectList.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectList.java @@ -36,12 +36,12 @@ public class CollectList implements AggregateFunction { @Override public void open(UDFContext udfContext) { - if (udfContext.getLookup_fields() == null) { + if (udfContext.getLookupFields() == null) { throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } - this.lookupField = udfContext.getLookup_fields().get(0); - if (udfContext.getOutput_fields() != null && !udfContext.getOutput_fields().isEmpty()) { - this.outputField = udfContext.getOutput_fields().get(0); + this.lookupField = udfContext.getLookupFields().get(0); + if (udfContext.getOutputFields() != null && !udfContext.getOutputFields().isEmpty()) { + this.outputField = udfContext.getOutputFields().get(0); } else { outputField = lookupField; } @@ -56,8 +56,9 @@ public class CollectList implements AggregateFunction { @Override public Accumulator add(Event event, Accumulator acc) { - if (event.getExtractedFields().containsKey(lookupField)) { - Object object = event.getExtractedFields().get(lookupField); + Object valueObj = event.getExtractedFields().get(lookupField); + if (valueObj != null) { + Object object = valueObj; List<Object> aggregate = (List<Object>) acc.getMetricsFields().get(outputField); aggregate.add(object); acc.getMetricsFields().put(outputField, aggregate); @@ -77,12 +78,14 @@ public class CollectList implements AggregateFunction { @Override public Accumulator merge(Accumulator firstAcc, Accumulator secondAcc) { - if (firstAcc.getMetricsFields().containsKey(outputField) && secondAcc.getMetricsFields().containsKey(outputField)) { - List<Object> firstValue = (List<Object>) firstAcc.getMetricsFields().get(outputField); - List<Object> secondValue = (List<Object>) secondAcc.getMetricsFields().get(outputField); + Object firstValueObj = firstAcc.getMetricsFields().get(outputField); + Object secondValueObj = secondAcc.getMetricsFields().get(outputField); + if (firstValueObj != null && secondValueObj != null) { + List<Object> firstValue = (List<Object>) firstValueObj; + List<Object> secondValue = (List<Object>) secondValueObj; firstValue.addAll(secondValue); - } else if (!firstAcc.getMetricsFields().containsKey(outputField) && secondAcc.getMetricsFields().containsKey(outputField)) { - List<Object> secondValue = (List<Object>) secondAcc.getMetricsFields().get(outputField); + } else if (firstValueObj == null && secondValueObj != null) { + List<Object> secondValue = (List<Object>) secondValueObj; firstAcc.getMetricsFields().put(outputField, secondValue); } return firstAcc; diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectSet.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectSet.java index 9ec9b09..c23f0ca 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectSet.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/CollectSet.java @@ -22,12 +22,12 @@ public class CollectSet implements AggregateFunction { @Override public void open(UDFContext udfContext) { - if (udfContext.getLookup_fields() == null) { + if (udfContext.getLookupFields() == null) { throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } - this.lookupField = udfContext.getLookup_fields().get(0); - if (udfContext.getOutput_fields() != null && !udfContext.getOutput_fields().isEmpty()) { - this.outputField = udfContext.getOutput_fields().get(0); + this.lookupField = udfContext.getLookupFields().get(0); + if (udfContext.getOutputFields() != null && !udfContext.getOutputFields().isEmpty()) { + this.outputField = udfContext.getOutputFields().get(0); } else { outputField = lookupField; } @@ -41,8 +41,9 @@ public class CollectSet implements AggregateFunction { @Override public Accumulator add(Event event, Accumulator acc) { - if (event.getExtractedFields().containsKey(lookupField)) { - Object object = event.getExtractedFields().get(lookupField); + Object valueObj = event.getExtractedFields().get(lookupField); + if (valueObj != null) { + Object object = valueObj; Set<Object> aggregate = (Set<Object>) acc.getMetricsFields().get(outputField); aggregate.add(object); acc.getMetricsFields().put(outputField, aggregate); @@ -62,12 +63,14 @@ public class CollectSet implements AggregateFunction { @Override public Accumulator merge(Accumulator firstAcc, Accumulator secondAcc) { - if (firstAcc.getMetricsFields().containsKey(outputField) && secondAcc.getMetricsFields().containsKey(outputField)) { - Set<Object> firstValue = (Set<Object>) firstAcc.getMetricsFields().get(outputField); - Set<Object> secondValue = (Set<Object>) secondAcc.getMetricsFields().get(outputField); + Object firstValueObj = firstAcc.getMetricsFields().get(outputField); + Object secondValueObj = secondAcc.getMetricsFields().get(outputField); + if (firstValueObj != null && secondValueObj != null) { + Set<Object> firstValue = (Set<Object>) firstValueObj; + Set<Object> secondValue = (Set<Object>) secondValueObj; firstValue.addAll(secondValue); - } else if (!firstAcc.getMetricsFields().containsKey(outputField) && secondAcc.getMetricsFields().containsKey(outputField)) { - Set<Object> secondValue = (Set<Object>) secondAcc.getMetricsFields().get(outputField); + } else if (firstValueObj == null && secondValueObj !=null) { + Set<Object> secondValue = (Set<Object>)secondValueObj; firstAcc.getMetricsFields().put(outputField, secondValue); } return firstAcc; diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/FirstValue.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/FirstValue.java index a1a35be..f68448f 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/FirstValue.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/FirstValue.java @@ -34,12 +34,12 @@ public class FirstValue implements AggregateFunction { @Override public void open(UDFContext udfContext) { - if (udfContext.getLookup_fields() == null) { + if (udfContext.getLookupFields() == null) { throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } - this.lookupField = udfContext.getLookup_fields().get(0); - if (udfContext.getOutput_fields() != null && !udfContext.getOutput_fields().isEmpty()) { - this.outputField = udfContext.getOutput_fields().get(0); + this.lookupField = udfContext.getLookupFields().get(0); + if (udfContext.getOutputFields() != null && !udfContext.getOutputFields().isEmpty()) { + this.outputField = udfContext.getOutputFields().get(0); } else { outputField = lookupField; } @@ -52,7 +52,8 @@ public class FirstValue implements AggregateFunction { @Override public Accumulator add(Event event, Accumulator acc) { - if (!acc.getMetricsFields().containsKey(outputField) && event.getExtractedFields().containsKey(lookupField)) { + Object valueObj = event.getExtractedFields().get(lookupField); + if (!acc.getMetricsFields().containsKey(outputField) && valueObj != null) { acc.getMetricsFields().put(outputField, event.getExtractedFields().get(lookupField)); } return acc; @@ -70,8 +71,10 @@ public class FirstValue implements AggregateFunction { @Override public Accumulator merge(Accumulator firstAcc, Accumulator secondAcc) { - if (!firstAcc.getMetricsFields().containsKey(outputField) && secondAcc.getMetricsFields().containsKey(outputField)) { - firstAcc.getMetricsFields().put(outputField, secondAcc.getMetricsFields().get(outputField)); + Object firstValueObj = firstAcc.getMetricsFields().get(outputField); + Object secondValueObj = secondAcc.getMetricsFields().get(outputField); + if (firstValueObj == null && secondValueObj != null ) { + firstAcc.getMetricsFields().put(outputField, secondValueObj); } return firstAcc; } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramBaseAggregate.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramBaseAggregate.java index a099fde..d8656e0 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramBaseAggregate.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramBaseAggregate.java @@ -1,121 +1,121 @@ -package com.geedgenetworks.core.udf.udaf.HdrHistogram;
-
-import com.geedgenetworks.common.Accumulator;
-import com.geedgenetworks.common.Event;
-import com.geedgenetworks.common.udf.AggregateFunction;
-import com.geedgenetworks.common.udf.UDFContext;
-import com.geedgenetworks.sketch.util.StringUtils;
-import org.HdrHistogram.ArrayHistogram;
-import org.HdrHistogram.DirectMapHistogram;
-import org.HdrHistogram.Histogramer;
-import org.apache.commons.collections.CollectionUtils;
-
-import java.nio.charset.StandardCharsets;
-import java.util.Map;
-
-public abstract class HdrHistogramBaseAggregate implements AggregateFunction {
- protected String inputField;
- protected String outputField;
- protected boolean inputSketch;
- protected long lowestDiscernibleValue;
- protected long highestTrackableValue;
- protected int numberOfSignificantValueDigits;
- protected boolean autoResize;
-
- @Override
- public void open(UDFContext c) {
- inputField = c.getLookup_fields().get(0);
- if (CollectionUtils.isNotEmpty(c.getOutput_fields())) {
- outputField = c.getOutput_fields().get(0);
- } else {
- outputField = inputField;
- }
- Map<String, Object> params = c.getParameters();
- lowestDiscernibleValue = Long.parseLong(params.getOrDefault("lowestDiscernibleValue", "1").toString());
- highestTrackableValue = Long.parseLong(params.getOrDefault("highestTrackableValue", "2").toString());
- numberOfSignificantValueDigits = Integer.parseInt(params.getOrDefault("numberOfSignificantValueDigits", "1").toString());
- autoResize = Boolean.valueOf(params.getOrDefault("autoResize", "true").toString());
- inputSketch = "sketch".equalsIgnoreCase(params.getOrDefault("input_type", "sketch").toString());
- }
-
- @Override
- public Accumulator initAccumulator(Accumulator acc) {
- return acc;
- }
-
- @Override
- public Accumulator add(Event event, Accumulator acc) {
- Object value = event.getExtractedFields().get(inputField);
- if (value == null) {
- return acc;
- }
-
- if (inputSketch) {
- updateHdrMerge(acc, value);
- } else {
- updateHdr(acc, value);
- }
-
- return acc;
- }
-
- @Override
- public Accumulator merge(Accumulator acc, Accumulator other) {
- Object agg = acc.getMetricsFields().get(outputField);
- Object aggOther = other.getMetricsFields().get(outputField);
- Object rst;
-
- if(agg == null){
- rst = aggOther;
- } else if (aggOther == null) {
- rst = agg;
- }else{
- rst = ((Histogramer)agg).merge(((Histogramer) aggOther));
- }
-
- if(rst != null){
- acc.getMetricsFields().put(outputField, rst);
- }
- return acc;
- }
-
- protected void updateHdr(Accumulator acc, Object value) {
- Map<String, Object> aggs = acc.getMetricsFields();
- ArrayHistogram his = (ArrayHistogram) aggs.get(outputField);
- if (his == null) {
- his = new ArrayHistogram(lowestDiscernibleValue, highestTrackableValue, numberOfSignificantValueDigits);
- his.setAutoResize(autoResize);
- aggs.put(outputField, his);
- }
-
- his.recordValue(((Number) value).longValue());
- }
-
-
- protected void updateHdrMerge(Accumulator acc, Object value) {
- Map<String, Object> aggs = acc.getMetricsFields();
- ArrayHistogram his = (ArrayHistogram) aggs.get(outputField);
- if (his == null) {
- his = new ArrayHistogram(lowestDiscernibleValue, highestTrackableValue, numberOfSignificantValueDigits);
- his.setAutoResize(autoResize);
- aggs.put(outputField, his);
- }
-
- Histogramer h;
- if (value instanceof String) {
- byte[] bytes = StringUtils.decodeBase64(((String) value).getBytes(StandardCharsets.UTF_8));
- h = DirectMapHistogram.wrapBytes(bytes);
- } else if (value instanceof byte[]) {
- h = DirectMapHistogram.wrapBytes((byte[]) value);
- } else if (value instanceof Histogramer) {
- h = (Histogramer) value;
- } else {
- throw new IllegalArgumentException("Unsupported type " + value.getClass());
- }
-
- his.merge(h);
- }
-
- @Override
- public void close() {}
-}
+package com.geedgenetworks.core.udf.udaf.HdrHistogram; + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.AggregateFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.sketch.util.StringUtils; +import org.HdrHistogram.ArrayHistogram; +import org.HdrHistogram.DirectMapHistogram; +import org.HdrHistogram.Histogramer; +import org.apache.commons.collections.CollectionUtils; + +import java.nio.charset.StandardCharsets; +import java.util.Map; + +public abstract class HdrHistogramBaseAggregate implements AggregateFunction { + protected String inputField; + protected String outputField; + protected boolean inputSketch; + protected long lowestDiscernibleValue; + protected long highestTrackableValue; + protected int numberOfSignificantValueDigits; + protected boolean autoResize; + + @Override + public void open(UDFContext c) { + inputField = c.getLookupFields().get(0); + if (CollectionUtils.isNotEmpty(c.getOutputFields())) { + outputField = c.getOutputFields().get(0); + } else { + outputField = inputField; + } + Map<String, Object> params = c.getParameters(); + lowestDiscernibleValue = Long.parseLong(params.getOrDefault("lowestDiscernibleValue", "1").toString()); + highestTrackableValue = Long.parseLong(params.getOrDefault("highestTrackableValue", "2").toString()); + numberOfSignificantValueDigits = Integer.parseInt(params.getOrDefault("numberOfSignificantValueDigits", "1").toString()); + autoResize = Boolean.valueOf(params.getOrDefault("autoResize", "true").toString()); + inputSketch = "sketch".equalsIgnoreCase(params.getOrDefault("input_type", "sketch").toString()); + } + + @Override + public Accumulator initAccumulator(Accumulator acc) { + return acc; + } + + @Override + public Accumulator add(Event event, Accumulator acc) { + Object value = event.getExtractedFields().get(inputField); + if (value == null) { + return acc; + } + + if (inputSketch) { + updateHdrMerge(acc, value); + } else { + updateHdr(acc, value); + } + + return acc; + } + + @Override + public Accumulator merge(Accumulator acc, Accumulator other) { + Object agg = acc.getMetricsFields().get(outputField); + Object aggOther = other.getMetricsFields().get(outputField); + Object rst; + + if(agg == null){ + rst = aggOther; + } else if (aggOther == null) { + rst = agg; + }else{ + rst = ((Histogramer)agg).merge(((Histogramer) aggOther)); + } + + if(rst != null){ + acc.getMetricsFields().put(outputField, rst); + } + return acc; + } + + protected void updateHdr(Accumulator acc, Object value) { + Map<String, Object> aggs = acc.getMetricsFields(); + ArrayHistogram his = (ArrayHistogram) aggs.get(outputField); + if (his == null) { + his = new ArrayHistogram(lowestDiscernibleValue, highestTrackableValue, numberOfSignificantValueDigits); + his.setAutoResize(autoResize); + aggs.put(outputField, his); + } + + his.recordValue(((Number) value).longValue()); + } + + + protected void updateHdrMerge(Accumulator acc, Object value) { + Map<String, Object> aggs = acc.getMetricsFields(); + ArrayHistogram his = (ArrayHistogram) aggs.get(outputField); + if (his == null) { + his = new ArrayHistogram(lowestDiscernibleValue, highestTrackableValue, numberOfSignificantValueDigits); + his.setAutoResize(autoResize); + aggs.put(outputField, his); + } + + Histogramer h; + if (value instanceof String) { + byte[] bytes = StringUtils.decodeBase64(((String) value).getBytes(StandardCharsets.UTF_8)); + h = DirectMapHistogram.wrapBytes(bytes); + } else if (value instanceof byte[]) { + h = DirectMapHistogram.wrapBytes((byte[]) value); + } else if (value instanceof Histogramer) { + h = (Histogramer) value; + } else { + throw new IllegalArgumentException("Unsupported type " + value.getClass()); + } + + his.merge(h); + } + + @Override + public void close() {} +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LastValue.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LastValue.java index 44b374e..f319e8d 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LastValue.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LastValue.java @@ -34,12 +34,12 @@ public class LastValue implements AggregateFunction { @Override public void open(UDFContext udfContext) { - if (udfContext.getLookup_fields() == null) { + if (udfContext.getLookupFields() == null) { throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } - this.lookupField = udfContext.getLookup_fields().get(0); - if (udfContext.getOutput_fields() != null && !udfContext.getOutput_fields().isEmpty()) { - this.outputField = udfContext.getOutput_fields().get(0); + this.lookupField = udfContext.getLookupFields().get(0); + if (udfContext.getOutputFields() != null && !udfContext.getOutputFields().isEmpty()) { + this.outputField = udfContext.getOutputFields().get(0); } else { outputField = lookupField; } @@ -70,8 +70,10 @@ public class LastValue implements AggregateFunction { @Override public Accumulator merge(Accumulator firstAcc, Accumulator secondAcc) { - if (secondAcc.getMetricsFields().containsKey(outputField)) { - firstAcc.getMetricsFields().put(outputField, secondAcc.getMetricsFields().get(outputField)); + + Object secondValueObj = secondAcc.getMetricsFields().get(outputField); + if (secondValueObj != null) { + firstAcc.getMetricsFields().put(outputField, secondValueObj); } return firstAcc; } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LongCount.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LongCount.java index 05de38c..418eb9c 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LongCount.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/LongCount.java @@ -14,10 +14,10 @@ public class LongCount implements AggregateFunction { @Override public void open(UDFContext udfContext) { - if (udfContext.getOutput_fields() == null) { + if (udfContext.getOutputFields() == null) { throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } - outputField = udfContext.getOutput_fields().get(0); + outputField = udfContext.getOutputFields().get(0); } @Override @@ -44,14 +44,16 @@ public class LongCount implements AggregateFunction { @Override public Accumulator merge(Accumulator firstAcc, Accumulator secondAcc) { - if (firstAcc.getMetricsFields().containsKey(outputField) && secondAcc.getMetricsFields().containsKey(outputField)) { - long firstValue = (long) firstAcc.getMetricsFields().get(outputField); - long secondValue = (long) secondAcc.getMetricsFields().get(outputField); + Object firstValueObj = firstAcc.getMetricsFields().get(outputField); + Object secondValueObj = secondAcc.getMetricsFields().get(outputField); + if (firstValueObj!=null && secondValueObj!=null) { + long firstValue = (long) firstValueObj; + long secondValue = (long) secondValueObj; firstValue = firstValue + secondValue; firstAcc.getMetricsFields().put(outputField, firstValue); - } else if (!firstAcc.getMetricsFields().containsKey(outputField) && secondAcc.getMetricsFields().containsKey(outputField)) { - Number secondValue = (Number) secondAcc.getMetricsFields().get(outputField); + } else if (firstValueObj==null && secondValueObj!=null) { + Number secondValue = (Number) secondValueObj; firstAcc.getMetricsFields().put(outputField, secondValue); } return firstAcc; diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Max.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Max.java new file mode 100644 index 0000000..226df0a --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Max.java @@ -0,0 +1,121 @@ +package com.geedgenetworks.core.udf.udaf; + + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.exception.CommonErrorCode; +import com.geedgenetworks.common.exception.GrootStreamRuntimeException; +import com.geedgenetworks.common.udf.AggregateFunction; +import com.geedgenetworks.common.udf.UDFContext; + +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; +import java.util.Map; + + +public class Max implements AggregateFunction { + + private String lookupField; + private String outputField; + + @Override + public void open(UDFContext udfContext) { + if (udfContext.getLookupFields() == null) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); + } + this.lookupField = udfContext.getLookupFields().get(0); + if (udfContext.getOutputFields() != null && !udfContext.getOutputFields().isEmpty()) { + this.outputField = udfContext.getOutputFields().get(0); + } else { + outputField = lookupField; + } + } + + @Override + public Accumulator initAccumulator(Accumulator acc) { + return acc; + } + + @Override + public Accumulator add(Event event, Accumulator acc) { + Map<String, Object> eventFields = event.getExtractedFields(); + Map<String, Object> metricsFields = acc.getMetricsFields(); + + if (metricsFields.get(outputField) == null && eventFields.get(lookupField) != null) { + metricsFields.put(outputField, eventFields.get(lookupField)); + } else if (metricsFields.get(outputField) != null && eventFields.get(lookupField) != null) { + Object currentValue = metricsFields.get(outputField); + Object newValue = eventFields.get(lookupField); + + // 直接检测是否为时间类型对象 + if (currentValue instanceof LocalDateTime && newValue instanceof LocalDateTime) { + LocalDateTime time1 = (LocalDateTime) currentValue; + LocalDateTime time2 = (LocalDateTime) newValue; + if (time1.isBefore(time2)) { + metricsFields.put(outputField, newValue); + } + } else if (currentValue instanceof String && newValue instanceof String) { + // 处理字符串比较 + String value1 = currentValue.toString(); + String value2 = newValue.toString(); + if (value1.compareTo(value2) < 0) { + metricsFields.put(outputField, newValue); + } + + } else { + // 数字进行比较 + double value1 = Double.parseDouble(currentValue.toString()); + double value2 = Double.parseDouble(newValue.toString()); + if (value1 < value2) { + metricsFields.put(outputField, newValue); + } + } + } + return acc; + } + + @Override + public String functionName() { + return "MAX"; + } + + @Override + public Accumulator getResult(Accumulator acc) { + return acc; + } + + @Override + public Accumulator merge(Accumulator firstAcc, Accumulator secondAcc) { + Map<String, Object> firstMetrics = firstAcc.getMetricsFields(); + Map<String, Object> secondMetrics = secondAcc.getMetricsFields(); + Object firstValue = firstMetrics.get(outputField); + Object secondValue = secondMetrics.get(outputField); + if (firstValue == null && secondValue != null) { + firstMetrics.put(outputField, secondValue); + } else if (firstValue != null && secondValue != null) { + // 直接检测是否为时间类型对象 + if (firstValue instanceof LocalDateTime && secondValue instanceof LocalDateTime) { + LocalDateTime time1 = (LocalDateTime) firstValue; + LocalDateTime time2 = (LocalDateTime) secondValue; + if (time1.isBefore(time2)) { + firstMetrics.put(outputField, secondValue); + } + } else if (firstValue instanceof String && secondValue instanceof String) { + + String value1 = firstValue.toString(); + String value2 = secondValue.toString(); + if (value1.compareTo(value2) < 0) { + firstMetrics.put(outputField, secondValue); + } + } else { + // 假设为数字,进行比较 + double value1 = Double.parseDouble(firstValue.toString()); + double value2 = Double.parseDouble(secondValue.toString()); + if (value1 < value2) { + firstMetrics.put(outputField, secondValue); + } + } + } + return firstAcc; + } +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Mean.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Mean.java index 9c4e070..88f693f 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Mean.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Mean.java @@ -20,12 +20,12 @@ public class Mean implements AggregateFunction { @Override public void open(UDFContext udfContext) { - if (udfContext.getLookup_fields() == null) { + if (udfContext.getLookupFields() == null) { throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } - lookupField = udfContext.getLookup_fields().get(0); - if (udfContext.getOutput_fields() != null && !udfContext.getOutput_fields().isEmpty()) { - outputField = udfContext.getOutput_fields().get(0); + lookupField = udfContext.getLookupFields().get(0); + if (udfContext.getOutputFields() != null && !udfContext.getOutputFields().isEmpty()) { + outputField = udfContext.getOutputFields().get(0); } else { outputField = lookupField; } diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Min.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Min.java new file mode 100644 index 0000000..6fd7046 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/Min.java @@ -0,0 +1,120 @@ +package com.geedgenetworks.core.udf.udaf; + + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.exception.CommonErrorCode; +import com.geedgenetworks.common.exception.GrootStreamRuntimeException; +import com.geedgenetworks.common.udf.AggregateFunction; +import com.geedgenetworks.common.udf.UDFContext; + +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; +import java.util.Map; + + +public class Min implements AggregateFunction { + + private String lookupField; + private String outputField; + + @Override + public void open(UDFContext udfContext) { + if (udfContext.getLookupFields() == null) { + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); + } + this.lookupField = udfContext.getLookupFields().get(0); + if (udfContext.getOutputFields() != null && !udfContext.getOutputFields().isEmpty()) { + this.outputField = udfContext.getOutputFields().get(0); + } else { + outputField = lookupField; + } + } + + @Override + public Accumulator initAccumulator(Accumulator acc) { + return acc; + } + + @Override + public Accumulator add(Event event, Accumulator acc) { + Map<String, Object> eventFields = event.getExtractedFields(); + Map<String, Object> metricsFields = acc.getMetricsFields(); + + if (metricsFields.get(outputField) == null && eventFields.get(lookupField) != null) { + metricsFields.put(outputField, eventFields.get(lookupField)); + } else if (metricsFields.get(outputField) != null && eventFields.get(lookupField) != null) { + Object currentValue = metricsFields.get(outputField); + Object newValue = eventFields.get(lookupField); + + // 直接检测是否为时间类型对象 + if (currentValue instanceof LocalDateTime && newValue instanceof LocalDateTime) { + LocalDateTime time1 = (LocalDateTime) currentValue; + LocalDateTime time2 = (LocalDateTime) newValue; + if (time1.isAfter(time2)) { + metricsFields.put(outputField, newValue); + } + } else if (currentValue instanceof String && newValue instanceof String) { + // 处理字符串比较 + String value1 = currentValue.toString(); + String value2 = newValue.toString(); + if (value1.compareTo(value2) > 0) { + metricsFields.put(outputField, newValue); + } + + } else { + // 假设为数字,进行比较 + double value1 = Double.parseDouble(currentValue.toString()); + double value2 = Double.parseDouble(newValue.toString()); + if (value1 > value2) { + metricsFields.put(outputField, newValue); + } + } + } + return acc; + } + + @Override + public String functionName() { + return "MIN"; + } + + @Override + public Accumulator getResult(Accumulator acc) { + return acc; + } + + @Override + public Accumulator merge(Accumulator firstAcc, Accumulator secondAcc) { + Map<String, Object> firstMetrics = firstAcc.getMetricsFields(); + Map<String, Object> secondMetrics = secondAcc.getMetricsFields(); + Object firstValue = firstMetrics.get(outputField); + Object secondValue = secondMetrics.get(outputField); + if (firstValue == null && secondValue != null) { + firstMetrics.put(outputField, secondValue); + } else if (firstValue != null && secondValue != null) { + // 直接检测是否为时间类型对象 + if (firstValue instanceof LocalDateTime && secondValue instanceof LocalDateTime) { + LocalDateTime time1 = (LocalDateTime) firstValue; + LocalDateTime time2 = (LocalDateTime) secondValue; + if (time1.isAfter(time2)) { + firstMetrics.put(outputField, secondValue); + } + } else if (firstValue instanceof String && secondValue instanceof String) { + String value1 = firstValue.toString(); + String value2 = secondValue.toString(); + if (value1.compareTo(value2) > 0) { + firstMetrics.put(outputField, secondValue); + } + } else { + // 假设为数字,进行比较 + double value1 = Double.parseDouble(firstValue.toString()); + double value2 = Double.parseDouble(secondValue.toString()); + if (value1 > value2) { + firstAcc.getMetricsFields().put(outputField, secondValue); + } + } + } + return firstAcc; + } +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/NumberSum.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/NumberSum.java index e972133..ab8f744 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/NumberSum.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/NumberSum.java @@ -15,12 +15,12 @@ public class NumberSum implements AggregateFunction { @Override public void open(UDFContext udfContext) { - if (udfContext.getLookup_fields() == null) { + if (udfContext.getLookupFields() == null) { throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } - lookupField = udfContext.getLookup_fields().get(0); - if (udfContext.getOutput_fields() != null && !udfContext.getOutput_fields().isEmpty()) { - outputField = udfContext.getOutput_fields().get(0); + lookupField = udfContext.getLookupFields().get(0); + if (udfContext.getOutputFields() != null && !udfContext.getOutputFields().isEmpty()) { + outputField = udfContext.getOutputFields().get(0); } else { outputField = lookupField; } @@ -33,9 +33,9 @@ public class NumberSum implements AggregateFunction { @Override public Accumulator add(Event event, Accumulator acc) { - - if (event.getExtractedFields().containsKey(lookupField)) { - Number val = (Number) event.getExtractedFields().get(lookupField); + Object valueObj = event.getExtractedFields().get(lookupField); + if (valueObj!=null) { + Number val = (Number) valueObj; Number aggregate = (Number) acc.getMetricsFields().getOrDefault(outputField, 0L); if (aggregate instanceof Long && (val instanceof Integer || val instanceof Long)) { aggregate = aggregate.longValue() + val.longValue(); @@ -66,10 +66,14 @@ public class NumberSum implements AggregateFunction { @Override public Accumulator merge(Accumulator firstAcc, Accumulator secondAcc) { - if (firstAcc.getMetricsFields().containsKey(outputField) && secondAcc.getMetricsFields().containsKey(outputField)) { - Number firstValue = (Number) firstAcc.getMetricsFields().get(outputField); - Number secondValue = (Number) secondAcc.getMetricsFields().get(outputField); + + Object firstValueObj = firstAcc.getMetricsFields().get(outputField); + Object secondValueObj = secondAcc.getMetricsFields().get(outputField); + if (firstValueObj != null && secondValueObj != null) { + + Number firstValue = (Number) firstValueObj; + Number secondValue = (Number) secondValueObj; if (firstValue instanceof Long && (secondValue instanceof Integer || secondValue instanceof Long)) { firstValue = firstValue.longValue() + secondValue.longValue(); } else if (firstValue instanceof Float || secondValue instanceof Float) { @@ -78,8 +82,8 @@ public class NumberSum implements AggregateFunction { firstValue = firstValue.doubleValue() + secondValue.doubleValue(); } firstAcc.getMetricsFields().put(outputField, firstValue); - } else if (!firstAcc.getMetricsFields().containsKey(outputField) && secondAcc.getMetricsFields().containsKey(outputField)) { - Number secondValue = (Number) secondAcc.getMetricsFields().get(outputField); + } else if (firstValueObj == null && secondValueObj != null) { + Number secondValue = (Number) secondValueObj; firstAcc.getMetricsFields().put(outputField, secondValue); } return firstAcc; diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/HlldBaseAggregate.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/HlldBaseAggregate.java index 0802c22..c113c4a 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/HlldBaseAggregate.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udaf/hlld/HlldBaseAggregate.java @@ -1,126 +1,126 @@ -package com.geedgenetworks.core.udf.udaf.hlld;
-
-import com.geedgenetworks.common.Accumulator;
-import com.geedgenetworks.common.Event;
-import com.geedgenetworks.common.udf.AggregateFunction;
-import com.geedgenetworks.common.udf.UDFContext;
-import com.geedgenetworks.sketch.hlld.Hll;
-import com.geedgenetworks.sketch.hlld.HllUnion;
-import com.geedgenetworks.sketch.hlld.HllUtils;
-import org.apache.commons.collections.CollectionUtils;
-
-import java.util.Map;
-
-public abstract class HlldBaseAggregate implements AggregateFunction {
- protected String inputField;
- protected String outputField;
- protected boolean inputSketch;
- protected int precision = 12;
-
- @Override
- public void open(UDFContext c) {
- inputField = c.getLookup_fields().get(0);
- if (CollectionUtils.isNotEmpty(c.getOutput_fields())) {
- outputField = c.getOutput_fields().get(0);
- } else {
- outputField = inputField;
- }
- Map<String, Object> params = c.getParameters();
- precision = Integer.parseInt(params.getOrDefault("precision", "12").toString());
- inputSketch = "sketch".equalsIgnoreCase(params.getOrDefault("input_type", "sketch").toString());
- }
-
- @Override
- public Accumulator initAccumulator(Accumulator acc) {
- return acc;
- }
-
- @Override
- public Accumulator add(Event event, Accumulator acc) {
- Object value = event.getExtractedFields().get(inputField);
- if (value == null) {
- return acc;
- }
-
- if (inputSketch) {
- updateHllUnion(acc, value);
- } else {
- updateHll(acc, value);
- }
-
- return acc;
- }
-
- @Override
- public Accumulator merge(Accumulator acc, Accumulator other) {
- Object agg = acc.getMetricsFields().get(outputField);
- Object aggOther = other.getMetricsFields().get(outputField);
- Object rst;
-
- if(agg == null){
- rst = aggOther;
- } else if (aggOther == null) {
- rst = agg;
- }else{
- if(inputSketch){
- ((HllUnion)agg).update(((HllUnion) aggOther).getResult());
- rst = agg;
- }else{
- final HllUnion union = new HllUnion(precision);
- union.update((Hll) agg);
- union.update((Hll) aggOther);
- rst = union.getResult();
- }
- }
-
- if(rst != null){
- acc.getMetricsFields().put(outputField, rst);
- }
- return acc;
- }
-
- protected Hll getResultHll(Accumulator acc){
- Object agg = acc.getMetricsFields().get(outputField);
- if (agg == null) {
- return null;
- }
-
- return inputSketch ? ((HllUnion) agg).getResult() : (Hll) agg;
- }
-
- protected void updateHll(Accumulator acc, Object value) {
- Map<String, Object> aggs = acc.getMetricsFields();
- Hll hll = (Hll) aggs.get(outputField);
- if (hll == null) {
- hll = new Hll(precision);
- aggs.put(outputField, hll);
- }
-
- if (value instanceof Integer || value instanceof Long) {
- hll.add(((Number) value).longValue());
- } else if (value instanceof Float || value instanceof Double) {
- hll.add(((Number) value).doubleValue());
- } else if (value instanceof String) {
- hll.add((String) value);
- } else if (value instanceof byte[]) {
- hll.add((byte[]) value);
- } else {
- throw new IllegalArgumentException("Unsupported type " + value.getClass());
- }
- }
-
- protected void updateHllUnion(Accumulator acc, Object value) {
- Map<String, Object> aggs = acc.getMetricsFields();
- HllUnion hllUnion = (HllUnion) aggs.get(outputField);
- if (hllUnion == null) {
- hllUnion = new HllUnion(precision);
- aggs.put(outputField, hllUnion);
- }
-
- Hll hll = HllUtils.deserializeHll(value);
- hllUnion.update(hll);
- }
-
- @Override
- public void close() {}
-}
+package com.geedgenetworks.core.udf.udaf.hlld; + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.AggregateFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.sketch.hlld.Hll; +import com.geedgenetworks.sketch.hlld.HllUnion; +import com.geedgenetworks.sketch.hlld.HllUtils; +import org.apache.commons.collections.CollectionUtils; + +import java.util.Map; + +public abstract class HlldBaseAggregate implements AggregateFunction { + protected String inputField; + protected String outputField; + protected boolean inputSketch; + protected int precision = 12; + + @Override + public void open(UDFContext c) { + inputField = c.getLookupFields().get(0); + if (CollectionUtils.isNotEmpty(c.getOutputFields())) { + outputField = c.getOutputFields().get(0); + } else { + outputField = inputField; + } + Map<String, Object> params = c.getParameters(); + precision = Integer.parseInt(params.getOrDefault("precision", "12").toString()); + inputSketch = "sketch".equalsIgnoreCase(params.getOrDefault("input_type", "sketch").toString()); + } + + @Override + public Accumulator initAccumulator(Accumulator acc) { + return acc; + } + + @Override + public Accumulator add(Event event, Accumulator acc) { + Object value = event.getExtractedFields().get(inputField); + if (value == null) { + return acc; + } + + if (inputSketch) { + updateHllUnion(acc, value); + } else { + updateHll(acc, value); + } + + return acc; + } + + @Override + public Accumulator merge(Accumulator acc, Accumulator other) { + Object agg = acc.getMetricsFields().get(outputField); + Object aggOther = other.getMetricsFields().get(outputField); + Object rst; + + if(agg == null){ + rst = aggOther; + } else if (aggOther == null) { + rst = agg; + }else{ + if(inputSketch){ + ((HllUnion)agg).update(((HllUnion) aggOther).getResult()); + rst = agg; + }else{ + final HllUnion union = new HllUnion(precision); + union.update((Hll) agg); + union.update((Hll) aggOther); + rst = union.getResult(); + } + } + + if(rst != null){ + acc.getMetricsFields().put(outputField, rst); + } + return acc; + } + + protected Hll getResultHll(Accumulator acc){ + Object agg = acc.getMetricsFields().get(outputField); + if (agg == null) { + return null; + } + + return inputSketch ? ((HllUnion) agg).getResult() : (Hll) agg; + } + + protected void updateHll(Accumulator acc, Object value) { + Map<String, Object> aggs = acc.getMetricsFields(); + Hll hll = (Hll) aggs.get(outputField); + if (hll == null) { + hll = new Hll(precision); + aggs.put(outputField, hll); + } + + if (value instanceof Integer || value instanceof Long) { + hll.add(((Number) value).longValue()); + } else if (value instanceof Float || value instanceof Double) { + hll.add(((Number) value).doubleValue()); + } else if (value instanceof String) { + hll.add((String) value); + } else if (value instanceof byte[]) { + hll.add((byte[]) value); + } else { + throw new IllegalArgumentException("Unsupported type " + value.getClass()); + } + } + + protected void updateHllUnion(Accumulator acc, Object value) { + Map<String, Object> aggs = acc.getMetricsFields(); + HllUnion hllUnion = (HllUnion) aggs.get(outputField); + if (hllUnion == null) { + hllUnion = new HllUnion(precision); + aggs.put(outputField, hllUnion); + } + + Hll hll = HllUtils.deserializeHll(value); + hllUnion.update(hll); + } + + @Override + public void close() {} +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/JsonUnroll.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/JsonUnroll.java index 2e8eb7e..de89e2c 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/JsonUnroll.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/JsonUnroll.java @@ -34,12 +34,12 @@ public class JsonUnroll implements TableFunction { @Override public void open(RuntimeContext runtimeContext, UDFContext udfContext) { - if(udfContext.getLookup_fields()==null ){ + if(udfContext.getLookupFields()==null ){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } - this.lookupFieldName = udfContext.getLookup_fields().get(0); - if(udfContext.getOutput_fields()!=null && !udfContext.getOutput_fields().isEmpty()) { - this.outputFieldName = udfContext.getOutput_fields().get(0); + this.lookupFieldName = udfContext.getLookupFields().get(0); + if(udfContext.getOutputFields()!=null && !udfContext.getOutputFields().isEmpty()) { + this.outputFieldName = udfContext.getOutputFields().get(0); } else { outputFieldName = lookupFieldName; @@ -57,13 +57,14 @@ public class JsonUnroll implements TableFunction { @Override public List<Event> evaluate(Event event) { try { - if(event.getExtractedFields().containsKey(lookupFieldName) ){ + Object valueObj = event.getExtractedFields().get(lookupFieldName); + if (valueObj!=null) { try { if(path.isEmpty()){ - JSONArray jsonArray = JSONArray.parseArray(event.getExtractedFields().get(lookupFieldName).toString()); + JSONArray jsonArray = JSONArray.parseArray(valueObj.toString()); return parseList(jsonArray,event); }else { - JSONObject jsonObject = JSONObject.parseObject(event.getExtractedFields().get(lookupFieldName).toString()); + JSONObject jsonObject = JSONObject.parseObject(valueObj.toString()); Object obj = JsonPathUtil.get(jsonObject,path); if(obj instanceof List || obj instanceof Array) { List list = (List) obj; diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/PathUnroll.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/PathUnroll.java new file mode 100644 index 0000000..e5732e0 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/PathUnroll.java @@ -0,0 +1,118 @@ +package com.geedgenetworks.core.udf.udtf; + +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.TableFunction; +import com.geedgenetworks.common.udf.UDFContext; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.util.Preconditions; + +import java.util.*; + +public class PathUnroll implements TableFunction { + private String pathField; + private String fileField; + private char separator; + + private String outputPathField; + private String outputFileField; + private List<Event> events; + + @Override + public void open(RuntimeContext runtimeContext, UDFContext c) { + Preconditions.checkArgument(c.getLookupFields().size() >= 1, "input fields requested one path param at least"); + Preconditions.checkArgument(CollectionUtils.isEmpty(c.getOutputFields()) || c.getOutputFields().size() == c.getOutputFields().size(), "output fields requested same count param with input fields"); + pathField = c.getLookupFields().get(0); + fileField = c.getLookupFields().size() == 1? null: c.getLookupFields().get(1); + + outputPathField = CollectionUtils.isEmpty(c.getOutputFields())? pathField : c.getOutputFields().get(0); + outputFileField = CollectionUtils.isEmpty(c.getOutputFields()) || c.getLookupFields().size() == 1 ? fileField : c.getOutputFields().get(1); + Map<String, Object> params = c.getParameters() == null? Collections.EMPTY_MAP:c.getParameters(); + String sep = params.getOrDefault("separator", "/").toString(); + Preconditions.checkArgument(sep.length() == 1, "separator mush has one char"); + separator = sep.charAt(0); + events = new ArrayList<>(); + } + + @Override + public List<Event> evaluate(Event event) { + Map<String, Object> map = event.getExtractedFields(); + String p = (String) map.get(pathField); + // 去除path结尾的分隔符 + final String path = StringUtils.isBlank(p)? null: (separator != p.charAt(p.length() - 1) ? p: p.substring(0, p.length() - 1)); + final String fileName = fileField == null? null: (String) map.get(fileField); + + if (StringUtils.isBlank(path)) { + return Collections.emptyList(); + } + + if(events.size() > 100){ + events = new ArrayList<>(); + }else if(events.size() > 0){ + events.clear(); + } + Event e; + Map<String, Object> fields; + + // 拆分path + int index = path.indexOf(separator); + String subPath; + while (index > 0) { + subPath = path.substring(0, index); + e = new Event(); + fields = new HashMap<>(map); + fields.put(outputPathField, subPath); + if(outputFileField != null){ + fields.put(outputFileField, null); + } + e.setExtractedFields(fields); + events.add(e); + index = path.indexOf(separator, index + 1); + } + boolean hasFile = StringUtils.isNotBlank(fileName); + boolean pathContainsFile = hasFile && path.endsWith(fileName); + + if(!hasFile){ + e = new Event(); + fields = new HashMap<>(map); + fields.put(outputPathField, path); + if(outputFileField != null){ + fields.put(outputFileField, null); + } + e.setExtractedFields(fields); + events.add(e); + }else{ + e = new Event(); + fields = new HashMap<>(map); + fields.put(outputPathField, path); + if(outputFileField != null){ + fields.put(outputFileField, pathContainsFile? fileName:null); + } + e.setExtractedFields(fields); + events.add(e); + + // 输出path + file + if(!pathContainsFile){ + e = new Event(); + fields = new HashMap<>(map); + fields.put(outputPathField, path + separator + fileName); + if(outputFileField != null){ + fields.put(outputFileField, fileName); + } + e.setExtractedFields(fields); + events.add(e); + } + } + + return events; + } + + @Override + public void close() {} + + @Override + public String functionName() { + return "PATH_UNROLL"; + } +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/Unroll.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/Unroll.java index 5becb8e..ff4a9ae 100644 --- a/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/Unroll.java +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/udtf/Unroll.java @@ -21,12 +21,12 @@ public class Unroll implements TableFunction { @Override public void open(RuntimeContext runtimeContext, UDFContext udfContext) { - if(udfContext.getLookup_fields()==null ){ + if(udfContext.getLookupFields()==null ){ throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); } - this.lookupFieldName = udfContext.getLookup_fields().get(0); - if(udfContext.getOutput_fields()!=null && !udfContext.getOutput_fields().isEmpty()) { - this.outputFieldName = udfContext.getOutput_fields().get(0); + this.lookupFieldName = udfContext.getLookupFields().get(0); + if(udfContext.getOutputFields()!=null && !udfContext.getOutputFields().isEmpty()) { + this.outputFieldName = udfContext.getOutputFields().get(0); } else { outputFieldName = lookupFieldName; @@ -42,20 +42,20 @@ public class Unroll implements TableFunction { @Override public List<Event> evaluate(Event event) { try { - if(event.getExtractedFields().containsKey(lookupFieldName)) { - + Object valueObj = event.getExtractedFields().get(lookupFieldName); + if (valueObj!=null) { if(regex.isEmpty()){ - if (event.getExtractedFields().get(lookupFieldName) instanceof List ) { - return parseList(event.getExtractedFields().get(lookupFieldName), event); - } else if(event.getExtractedFields().get(lookupFieldName) instanceof Object[]){ - return parseArray(event.getExtractedFields().get(lookupFieldName), event); + if (valueObj instanceof List ) { + return parseList(valueObj, event); + } else if(valueObj instanceof Object[]){ + return parseArray(valueObj, event); }else { log.error("Invalid unroll ! Object is not instance of list or array. expression=" + regex); } } else { - if (event.getExtractedFields().get(lookupFieldName) instanceof String) { - String[] array =((String) event.getExtractedFields().get(lookupFieldName)).split(regex); + if (valueObj instanceof String) { + String[] array =((String) valueObj).split(regex); return parseArray(array, event); }else { log.error("Invalid unroll ! Object is not instance of String. expression=" + regex); diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUID.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUID.java new file mode 100644 index 0000000..4e9a031 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUID.java @@ -0,0 +1,48 @@ +package com.geedgenetworks.core.udf.uuid; + +import com.fasterxml.uuid.Generators; +import com.fasterxml.uuid.impl.RandomBasedGenerator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.exception.CommonErrorCode; +import com.geedgenetworks.common.exception.GrootStreamRuntimeException; +import com.geedgenetworks.common.udf.ScalarFunction; +import com.geedgenetworks.common.udf.UDFContext; +import lombok.extern.slf4j.Slf4j; +import org.apache.flink.api.common.functions.RuntimeContext; + +@Slf4j +public class UUID implements ScalarFunction { + private String outputFieldName; + private RandomBasedGenerator randomBasedGenerator; + @Override + public void open(RuntimeContext runtimeContext, UDFContext udfContext) { + + if(udfContext.getOutputFields()==null ){ + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); + } + if(udfContext.getOutputFields().size() != 1){ + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value"); + } + this.outputFieldName = udfContext.getOutputFields().get(0); + this.randomBasedGenerator = Generators.randomBasedGenerator(); + } + + @Override + public Event evaluate(Event event) { + + event.getExtractedFields() + .put(outputFieldName, randomBasedGenerator.generate().toString()); + return event; + } + + @Override + public String functionName() { + return "UUID"; + } + + @Override + public void close() { + } + + +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDNameSpace.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDNameSpace.java new file mode 100644 index 0000000..a8941e2 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDNameSpace.java @@ -0,0 +1,43 @@ +package com.geedgenetworks.core.udf.uuid; + +import com.geedgenetworks.common.exception.GrootStreamRuntimeException; + +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; + +import static com.geedgenetworks.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; + +public enum UUIDNameSpace { + + NAMESPACE_IP(UUID.fromString("6ba7b890-9dad-11d1-80b4-00c04fd430c8")), + NAMESPACE_DOMAIN(UUID.fromString("6ba7b891-9dad-11d1-80b4-00c04fd430c8")), + NAMESPACE_APP(UUID.fromString("6ba7b892-9dad-11d1-80b4-00c04fd430c8")), + NAMESPACE_SUBSCRIBER(UUID.fromString("6ba7b893-9dad-11d1-80b4-00c04fd430c8")); + + private final UUID uuid; + + // Static map to hold the mapping from name to UUID + private static final Map<String, UUID> NAME_TO_UUID_MAP = new HashMap<>(); + + // Static block to populate the map + static { + for (UUIDNameSpace namespace : UUIDNameSpace.values()) { + NAME_TO_UUID_MAP.put(namespace.name(), namespace.uuid); + } + } + + UUIDNameSpace(UUID uuid) { + this.uuid = uuid; + } + + public static UUID getUUID(String name) { + UUID uuid = NAME_TO_UUID_MAP.get(name); + if (uuid == null) { + throw new GrootStreamRuntimeException(ILLEGAL_ARGUMENT,"No enum constant " + UUIDNameSpace.class.getCanonicalName() + "." + name); + } + return uuid; + } + + +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDv5.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDv5.java new file mode 100644 index 0000000..3a433b8 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDv5.java @@ -0,0 +1,68 @@ +package com.geedgenetworks.core.udf.uuid; + +import com.fasterxml.uuid.Generators; +import com.fasterxml.uuid.impl.NameBasedGenerator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.exception.CommonErrorCode; +import com.geedgenetworks.common.exception.GrootStreamRuntimeException; +import com.geedgenetworks.common.udf.ScalarFunction; +import com.geedgenetworks.common.udf.UDFContext; +import lombok.extern.slf4j.Slf4j; +import org.apache.flink.api.common.functions.RuntimeContext; + +import java.util.List; + +@Slf4j +public class UUIDv5 implements ScalarFunction { + private List<String> lookupFieldNames; + private String outputFieldName; + private NameBasedGenerator nameBasedGenerator; + private static final String NAMESPACE_KEY = "namespace"; + @Override + public void open(RuntimeContext runtimeContext, UDFContext udfContext) { + + if(udfContext.getOutputFields() == null || udfContext.getParameters() == null || udfContext.getLookupFields() == null){ + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); + } + if(udfContext.getOutputFields().size() != 1){ + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value"); + } + if(!udfContext.getParameters().containsKey(NAMESPACE_KEY) ){ + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Parameters must contain key: " + NAMESPACE_KEY); + } + + this.outputFieldName = udfContext.getOutputFields().get(0); + this.lookupFieldNames = udfContext.getLookupFields(); + String namespace = udfContext.getParameters().get(NAMESPACE_KEY).toString(); + this.nameBasedGenerator = Generators.nameBasedGenerator(UUIDNameSpace.getUUID(namespace)); + } + + @Override + public Event evaluate(Event event) { + + String concatenatedFields = String.join("_", + lookupFieldNames.stream() + .map(field -> event.getExtractedFields().getOrDefault(field, "")) + .toArray(String[]::new) + ); + + // Generate the UUID based on concatenated fields + String generatedUUID = nameBasedGenerator.generate(concatenatedFields).toString(); + + // Set the generated UUID in the output field + event.getExtractedFields().put(outputFieldName, generatedUUID); + return event; + + } + + @Override + public String functionName() { + return "UUIDv5"; + } + + @Override + public void close() { + + } + +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDv7.java b/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDv7.java new file mode 100644 index 0000000..60c388f --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/udf/uuid/UUIDv7.java @@ -0,0 +1,47 @@ +package com.geedgenetworks.core.udf.uuid; + +import com.fasterxml.uuid.Generators; +import com.fasterxml.uuid.impl.TimeBasedEpochGenerator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.exception.CommonErrorCode; +import com.geedgenetworks.common.exception.GrootStreamRuntimeException; +import com.geedgenetworks.common.udf.ScalarFunction; +import com.geedgenetworks.common.udf.UDFContext; +import lombok.extern.slf4j.Slf4j; +import org.apache.flink.api.common.functions.RuntimeContext; + +@Slf4j +public class UUIDv7 implements ScalarFunction { + private String outputFieldName; + private TimeBasedEpochGenerator timeBasedEpochRandomGenerator; + + @Override + public void open(RuntimeContext runtimeContext, UDFContext udfContext) { + + if(udfContext.getOutputFields()==null ){ + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "Missing required parameters"); + } + if(udfContext.getOutputFields().size() != 1){ + throw new GrootStreamRuntimeException(CommonErrorCode.ILLEGAL_ARGUMENT, "The function output fields only support 1 value"); + } + this.outputFieldName = udfContext.getOutputFields().get(0); + this.timeBasedEpochRandomGenerator = Generators.timeBasedEpochGenerator(); + + } + + @Override + public Event evaluate(Event event) { + event.getExtractedFields() + .put(outputFieldName, timeBasedEpochRandomGenerator.generate().toString()); + return event; + } + + @Override + public String functionName() { + return "UUIDv7"; + } + + @Override + public void close() { + } +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/utils/EncryptionAlgorithmUtils.java b/groot-core/src/main/java/com/geedgenetworks/core/utils/EncryptionAlgorithmUtils.java new file mode 100644 index 0000000..0327e49 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/utils/EncryptionAlgorithmUtils.java @@ -0,0 +1,30 @@ +package com.geedgenetworks.core.utils; + +import com.geedgenetworks.core.udf.encrypt.EncryptionAlgorithm; +import com.geedgenetworks.core.udf.encrypt.AES128GCM96; +import com.geedgenetworks.core.udf.encrypt.AES256GCM96; +import com.geedgenetworks.core.udf.encrypt.SM4GCM96; +import lombok.extern.slf4j.Slf4j; + +/** + * Crypto shade utilities + */ +@Slf4j +public final class EncryptionAlgorithmUtils { + public static final String ALGORITHM_AES_128_GCM96_NAME = "aes-128-gcm96"; + public static final String ALGORITHM_AES_256_GCM96_NAME = "aes-256-gcm96"; + public static final String ALGORITHM_SM4_GCM96_NAME = "sm4-gcm96"; + + public static EncryptionAlgorithm createEncryptionAlgorithm(String identifier) throws Exception { + switch (identifier) { + case ALGORITHM_AES_128_GCM96_NAME: + return new AES128GCM96(); + case ALGORITHM_AES_256_GCM96_NAME: + return new AES256GCM96(); + case ALGORITHM_SM4_GCM96_NAME: + return new SM4GCM96(); + default: + return null; + } + } +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/utils/KmsUtils.java b/groot-core/src/main/java/com/geedgenetworks/core/utils/KmsUtils.java new file mode 100644 index 0000000..9519dd5 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/utils/KmsUtils.java @@ -0,0 +1,71 @@ +package com.geedgenetworks.core.utils; + +import cn.hutool.core.util.StrUtil; +import com.geedgenetworks.common.config.KmsConfig; +import com.geedgenetworks.common.config.SSLConfig; +import com.geedgenetworks.core.pojo.KmsKey; +import io.github.jopenlibs.vault.SslConfig; +import io.github.jopenlibs.vault.Vault; +import io.github.jopenlibs.vault.VaultConfig; +import io.github.jopenlibs.vault.VaultException; +import io.github.jopenlibs.vault.json.JsonObject; +import io.github.jopenlibs.vault.response.AuthResponse; +import io.github.jopenlibs.vault.response.LogicalResponse; +import lombok.extern.slf4j.Slf4j; + +import java.io.File; +import java.util.Base64; + +@Slf4j +public class KmsUtils { + public static final String KMS_TYPE_LOCAL = "local"; + public static final String KMS_TYPE_VAULT = "vault"; + + public static KmsKey getVaultKey(KmsConfig kmsConfig, SSLConfig sslConfig, String identifier) throws Exception { + Vault vault = getVaultClient(kmsConfig, sslConfig); + String exportKeyPath; + if (EncryptionAlgorithmUtils.ALGORITHM_SM4_GCM96_NAME.equals(identifier)) { + exportKeyPath = kmsConfig.getPluginKeyPath() + "/export/encryption-key/" + identifier; + } else { + exportKeyPath = kmsConfig.getDefaultKeyPath() + "/export/encryption-key/" + identifier; + } + LogicalResponse exportResponse = vault.logical().read(exportKeyPath); + if (exportResponse.getRestResponse().getStatus() == 200) { + JsonObject keys = exportResponse.getDataObject().get("keys").asObject(); + return new KmsKey(Base64.getDecoder().decode(StrUtil.trim(keys.get(keys.size() + "").asString(), '"')), keys.size()); + } else { + throw new RuntimeException("Get vault key error! code: " + exportResponse.getRestResponse().getStatus() + " body: " + new String(exportResponse.getRestResponse().getBody())); + } + } + + public static Vault getVaultClient(KmsConfig kmsConfig, SSLConfig sslConfig) throws VaultException { + String username = kmsConfig.getUsername(); + String password = kmsConfig.getPassword(); + String url = kmsConfig.getUrl(); + boolean skipVerification = true; + String caCertificatePath = null; + String certificatePath = null; + String privateKeyPath = null; + if (sslConfig != null) { + skipVerification = sslConfig.getSkipVerification(); + caCertificatePath = sslConfig.getCaCertificatePath(); + certificatePath = sslConfig.getCertificatePath(); + privateKeyPath = sslConfig.getPrivateKeyPath(); + } + SslConfig vaultSslConfig = new SslConfig().verify(!skipVerification).build(); + if (!skipVerification) { + vaultSslConfig.pemFile(new File(caCertificatePath)) + .clientPemFile(new File(certificatePath)) + .clientKeyPemFile(new File(privateKeyPath)) + .build(); + } + VaultConfig config = new VaultConfig() + .address(url) + .engineVersion(1) + .sslConfig(vaultSslConfig) + .build(); + AuthResponse authResponse = Vault.create(config).auth().loginByUserPass(username, password); + config.token(authResponse.getAuthClientToken()); + return Vault.create(config); + } +} diff --git a/groot-core/src/main/java/com/geedgenetworks/core/utils/LoadIntervalDataOptions.java b/groot-core/src/main/java/com/geedgenetworks/core/utils/LoadIntervalDataOptions.java new file mode 100644 index 0000000..a81794d --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/utils/LoadIntervalDataOptions.java @@ -0,0 +1,80 @@ +package com.geedgenetworks.core.utils;
+
+import java.io.Serializable;
+
+public class LoadIntervalDataOptions implements Serializable {
+ final String name;
+
+ final long intervalMs;
+ final boolean failOnException;
+ final boolean updateDataOnStart;
+
+ /**
+ * @param name 名称, 用于日志打印以及线程名称标识
+ * @param intervalMs 每隔多长时间更新数据
+ * @param failOnException 更新数据时发生异常是否失败(默认false), 为true时如果发现异常data()方法下次返回数据时会抛出异常
+ * @param updateDataOnStart start时是否先更新数据(默认true), 为false时start候intervalMs时间后才会第一个更新数据
+ */
+ private LoadIntervalDataOptions(String name, long intervalMs, boolean failOnException, boolean updateDataOnStart) {
+ this.name = name;
+ this.intervalMs = intervalMs;
+ this.failOnException = failOnException;
+ this.updateDataOnStart = updateDataOnStart;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public long getIntervalMs() {
+ return intervalMs;
+ }
+
+ public boolean isFailOnException() {
+ return failOnException;
+ }
+
+ public boolean isUpdateDataOnStart() {
+ return updateDataOnStart;
+ }
+
+ public static Builder builder() {
+ return new Builder();
+ }
+
+ public static LoadIntervalDataOptions defaults(String name, long intervalMs) {
+ return builder().withName(name).withIntervalMs(intervalMs).build();
+ }
+
+ public static final class Builder {
+ private String name = "";
+ private long intervalMs = 1000 * 60 * 10;
+ private boolean failOnException = false;
+ private boolean updateDataOnStart = true;
+
+ public Builder withName(String name) {
+ this.name = name;
+ return this;
+ }
+
+ public Builder withIntervalMs(long intervalMs) {
+ this.intervalMs = intervalMs;
+ return this;
+ }
+
+ public Builder withFailOnException(boolean failOnException) {
+ this.failOnException = failOnException;
+ return this;
+ }
+
+ public Builder withUpdateDataOnStart(boolean updateDataOnStart) {
+ this.updateDataOnStart = updateDataOnStart;
+ return this;
+ }
+
+ public LoadIntervalDataOptions build() {
+ return new LoadIntervalDataOptions(name, intervalMs, failOnException, updateDataOnStart);
+ }
+ }
+
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/utils/LoadIntervalDataUtil.java b/groot-core/src/main/java/com/geedgenetworks/core/utils/LoadIntervalDataUtil.java new file mode 100644 index 0000000..566d217 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/utils/LoadIntervalDataUtil.java @@ -0,0 +1,86 @@ +package com.geedgenetworks.core.utils;
+
+import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.flink.util.function.SupplierWithException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class LoadIntervalDataUtil<T> {
+ static final Logger LOG = LoggerFactory.getLogger(LoadIntervalDataUtil.class);
+
+ private final SupplierWithException<T, Exception> dataSupplier;
+ private final LoadIntervalDataOptions options;
+
+ private final AtomicBoolean started = new AtomicBoolean(false);
+ private final AtomicBoolean stopped = new AtomicBoolean(false);
+ private ScheduledExecutorService scheduler;
+ private volatile Exception exception;
+ private volatile T data;
+
+ private LoadIntervalDataUtil(SupplierWithException<T, Exception> dataSupplier, LoadIntervalDataOptions options) {
+ this.dataSupplier = dataSupplier;
+ this.options = options;
+ }
+
+ public static <T> LoadIntervalDataUtil<T> newInstance(SupplierWithException<T, Exception> dataSupplier, LoadIntervalDataOptions options) {
+ LoadIntervalDataUtil<T> loadIntervalDataUtil = new LoadIntervalDataUtil(dataSupplier, options);
+ loadIntervalDataUtil.start();
+ return loadIntervalDataUtil;
+ }
+
+ public T data() throws Exception {
+ if (!options.failOnException || exception == null) {
+ return data;
+ } else {
+ throw exception;
+ }
+ }
+
+ private void updateData() {
+ try {
+ LOG.info("{} updateData start....", options.name);
+ data = dataSupplier.get();
+ LOG.info("{} updateData end....", options.name);
+ } catch (Throwable t) {
+ if (options.failOnException) {
+ exception = new RuntimeException(t);
+ }
+ LOG.info("{} updateData error", options.name, t);
+ }
+ }
+
+ private void start() {
+ if (started.compareAndSet(false, true)) {
+ if (options.updateDataOnStart) {
+ updateData();
+ }
+ this.scheduler = newDaemonSingleThreadScheduledExecutor(String.format("LoadIntervalDataUtil[%s]", options.name));
+ this.scheduler.scheduleWithFixedDelay(() -> updateData(), options.intervalMs, options.intervalMs, TimeUnit.MILLISECONDS);
+ LOG.info("{} start....", options.name);
+ }
+ }
+
+ public void stop() {
+ if (stopped.compareAndSet(false, true)) {
+ if (scheduler != null) {
+ this.scheduler.shutdown();
+ }
+ LOG.info("{} stop....", options.name);
+ }
+ }
+
+ private static ScheduledExecutorService newDaemonSingleThreadScheduledExecutor(String threadName) {
+ ThreadFactory threadFactory = new ThreadFactoryBuilder().setDaemon(true).setNameFormat(threadName).build();
+ ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1, threadFactory);
+ // By default, a cancelled task is not automatically removed from the work queue until its delay
+ // elapses. We have to enable it manually.
+ executor.setRemoveOnCancelPolicy(true);
+ return executor;
+ }
+}
diff --git a/groot-core/src/main/java/com/geedgenetworks/core/utils/SingleValueMap.java b/groot-core/src/main/java/com/geedgenetworks/core/utils/SingleValueMap.java new file mode 100644 index 0000000..f6f73c3 --- /dev/null +++ b/groot-core/src/main/java/com/geedgenetworks/core/utils/SingleValueMap.java @@ -0,0 +1,125 @@ +package com.geedgenetworks.core.utils;
+
+import org.apache.flink.util.function.SupplierWithException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.function.Consumer;
+
+/**
+ * 主要用于实现全局对象
+ * 主要用于flink算子中,方便复用全局对象
+ * flink算子中使用方法:
+ * // open方法中根据传入的获取data函数获取data
+ * data = SingleValueMap.acquireData("key", () -> getDataFunc());
+ * // process方法中使用data
+ * data.getData()
+ * // close方法中释放data
+ * if(data != null)
+ * data.release();
+ */
+public class SingleValueMap {
+ static final Logger LOG = LoggerFactory.getLogger(SingleValueMap.class);
+ private static Map<Object, Data<?>> cache = new LinkedHashMap<>();
+
+ public static synchronized <T> Data<T> acquireData(Object key, SupplierWithException<T, Exception> dataSupplier) throws Exception {
+ return acquireData(key, dataSupplier, x -> {});
+ }
+
+ public static synchronized <T> Data<T> acquireData(Object key, SupplierWithException<T, Exception> dataSupplier, Consumer<T> releaseFunc) throws Exception {
+ assert releaseFunc != null;
+ Data<?> existingData = cache.get(key);
+ Data<T> data;
+ if (existingData == null) {
+ Data<T> newData = new Data<>(key, dataSupplier.get(), releaseFunc);
+ cache.put(key, newData);
+ data = newData;
+ } else {
+ data = (Data<T>) existingData;
+ }
+ data.useCnt += 1;
+
+ LOG.info("acquireData: {}", data);
+
+ return data;
+ }
+
+ private static synchronized <T> void releaseData(Data<T> data) {
+ Data<?> cachedData = cache.get(data.key);
+ if (cachedData == null) {
+ LOG.error("can not get data: {}", data);
+ return;
+ }
+
+ assert data == cachedData;
+ LOG.info("releaseData: {}", data);
+
+ data.useCnt -= 1;
+ if (!data.inUse()) {
+ data.destroy();
+ cache.remove(data.key);
+
+ LOG.info("removeData: {}", data);
+ }
+ }
+
+ public static synchronized void clear() {
+ Iterator<Map.Entry<Object, Data<?>>> iter = cache.entrySet().iterator();
+ while (iter.hasNext()) {
+ Data<?> data = iter.next().getValue();
+ data.destroy();
+ iter.remove();
+ }
+ }
+
+ public final static class Data<T> {
+ final Object key;
+ final T data;
+ final Consumer<T> destroyFunc;
+ volatile int useCnt = 0;
+
+ Data(Object key, T data, Consumer<T> destroyFunc) {
+ this.key = key;
+ this.data = data;
+ this.destroyFunc = destroyFunc;
+ }
+
+ boolean inUse() {
+ return useCnt > 0;
+ }
+
+ void destroy() {
+ if (destroyFunc != null) {
+ try {
+ destroyFunc.accept(data);
+ } catch (Exception e) {
+ LOG.error("error when destroy data: {}", data);
+ }
+ }
+ }
+
+ public void release() {
+ releaseData(this);
+ }
+
+ public Object getKey() {
+ return key;
+ }
+
+ public T getData() {
+ return data;
+ }
+
+ @Override
+ public String toString() {
+ return "Data{" +
+ "key=" + key +
+ ", data=" + data +
+ ", useCnt=" + useCnt +
+ '}';
+ }
+ }
+}
diff --git a/groot-core/src/test/java/com/geedgenetworks/core/types/TypesTest.java b/groot-core/src/test/java/com/geedgenetworks/core/types/TypesTest.java index ca8d4e5..518a3f4 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/types/TypesTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/types/TypesTest.java @@ -1,8 +1,11 @@ package com.geedgenetworks.core.types; +import com.alibaba.fastjson2.JSON; import org.junit.jupiter.api.Test; import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -29,6 +32,42 @@ public class TypesTest { } @Test + void test() { + Map<String, Object> map = new LinkedHashMap<>(); + map.put("a", 1); + map.put("b", "aa"); + map.put("c", List.of(1, 2, 3)); + map.put("int_array", new int[]{1, 2, 3}); + map.put("str_array", new String[]{"1", "2", "3"}); + map.put("obj_array", new Object[]{"1", "2", "3"}); + String jsonString = JSON.toJSONString(map); + System.out.println(jsonString); + } + + @Test + void test2() { + Object obj = new int[]{1, 2, 3}; + System.out.println(obj instanceof byte[]); + System.out.println(obj instanceof int[]); + System.out.println(obj instanceof String[]); + System.out.println(obj instanceof Object[]); + System.out.println(); + + obj = new String[]{"1", "2", "3"}; + System.out.println(obj instanceof byte[]); + System.out.println(obj instanceof int[]); + System.out.println(obj instanceof String[]); + System.out.println(obj instanceof Object[]); + System.out.println(); + + obj = new Object[]{"1", "2", "3"}; + System.out.println(obj instanceof byte[]); + System.out.println(obj instanceof int[]); + System.out.println(obj instanceof String[]); + System.out.println(obj instanceof Object[]); + } + + @Test void testParserBaseType() { assertEquals(new IntegerType(), Types.parseDataType("INT")); assertEquals(new LongType(), Types.parseDataType("biGint")); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/AnonymityLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/AnonymityLookupTest.java index a52deb1..ae74fea 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/AnonymityLookupTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/AnonymityLookupTest.java @@ -37,8 +37,8 @@ class AnonymityLookupTest { parameters.put("kb_name", kbName); parameters.put("option", "IP_TO_NODE_TYPE"); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("server_ip")); - udfContext.setOutput_fields(Collections.singletonList("server_node_type")); + udfContext.setLookupFields(Collections.singletonList("server_ip")); + udfContext.setOutputFields(Collections.singletonList("server_node_type")); anonymityLookup.open(runtimeContext, udfContext); Event event = new Event(); @@ -56,8 +56,8 @@ class AnonymityLookupTest { parameters.put("kb_name", kbName); parameters.put("option", "DOMAIN_TO_NODE_TYPE"); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("domain")); - udfContext.setOutput_fields(Collections.singletonList("domain_node_type")); + udfContext.setLookupFields(Collections.singletonList("domain")); + udfContext.setOutputFields(Collections.singletonList("domain_node_type")); anonymityLookup.open(runtimeContext, udfContext); Event event = new Event(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/AppCategoryLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/AppCategoryLookupTest.java index f74ce39..713be3f 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/AppCategoryLookupTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/AppCategoryLookupTest.java @@ -30,7 +30,7 @@ class AppCategoryLookupTest { fieldMapping.put("COMPANY_CATEGORY", "app_company_category"); parameters.put("field_mapping", fieldMapping); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("app")); + udfContext.setLookupFields(Collections.singletonList("app")); RuntimeContext runtimeContext = mockRuntimeContext(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/BaseStationLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/BaseStationLookupTest.java new file mode 100644 index 0000000..43b0bd5 --- /dev/null +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/BaseStationLookupTest.java @@ -0,0 +1,69 @@ +package com.geedgenetworks.core.udf.cn; + +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.UDFContext; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import static com.geedgenetworks.core.udf.cn.LookupTestUtils.*; +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * @author gujinkai + * @version 1.0 + * @date 2024/9/2 15:20 + */ +class BaseStationLookupTest { + + private static BaseStationLookup baseStationLookup; + + @BeforeAll + static void setUp() { + /** + * Create a UDFContext object and set the parameters + * e.g.: + * - function: BASE_STATION_LOOKUP + * lookup_fields: [ cell_id ] + * output_fields: [ subscriber_longitude,subscriber_latitude ] + * parameters: + * kb_name: base_station_location + */ + UDFContext udfContext = new UDFContext(); + Map<String, Object> parameters = new HashMap<>(); + parameters.put("kb_name", kbName); + udfContext.setParameters(parameters); + udfContext.setLookupFields(Arrays.asList("cell_id")); + udfContext.setOutputFields(Arrays.asList("subscriber_longitude", "subscriber_latitude")); + + RuntimeContext runtimeContext = mockRuntimeContext(); + + String content = "cell_id,longitude,latitude\n460-11-630947-1,93.9290001,42.66884"; + mockKnowledgeBaseHandler(content); + + baseStationLookup = new BaseStationLookup(); + baseStationLookup.open(runtimeContext, udfContext); + } + + @Test + void evaluate() { + Event event = new Event(); + Map<String, Object> fields = new HashMap<>(); + fields.put("cell_id", "460-11-630947-1"); + fields.put("cell_type", 1L); + event.setExtractedFields(fields); + Event evaluate = baseStationLookup.evaluate(event); + assertEquals(93.9290001, evaluate.getExtractedFields().get("subscriber_longitude")); + assertEquals(42.66884, evaluate.getExtractedFields().get("subscriber_latitude")); + } + + @AfterEach + void afterAll() { + clearState(); + } +} diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/DnsServerInfoLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/DnsServerInfoLookupTest.java index 7f526d5..d1cca09 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/DnsServerInfoLookupTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/DnsServerInfoLookupTest.java @@ -25,8 +25,8 @@ class DnsServerInfoLookupTest { Map<String, Object> parameters = new HashMap<>(); parameters.put("kb_name", kbName); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("server_ip")); - udfContext.setOutput_fields(Collections.singletonList("server_dns_server")); + udfContext.setLookupFields(Collections.singletonList("server_ip")); + udfContext.setOutputFields(Collections.singletonList("server_dns_server")); RuntimeContext runtimeContext = mockRuntimeContext(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/FqdnCategoryLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/FqdnCategoryLookupTest.java index db15642..0e64982 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/FqdnCategoryLookupTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/FqdnCategoryLookupTest.java @@ -29,7 +29,7 @@ class FqdnCategoryLookupTest { fieldMapping.put("REPUTATION_LEVEL", "domain_reputation_level"); parameters.put("field_mapping", fieldMapping); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("domain")); + udfContext.setLookupFields(Collections.singletonList("domain")); RuntimeContext runtimeContext = mockRuntimeContext(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/FqdnWhoisLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/FqdnWhoisLookupTest.java index 42a98dc..93ee663 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/FqdnWhoisLookupTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/FqdnWhoisLookupTest.java @@ -24,8 +24,8 @@ class FqdnWhoisLookupTest { Map<String, Object> parameters = new HashMap<>(); parameters.put("kb_name", kbName); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("domain")); - udfContext.setOutput_fields(Collections.singletonList("domain_whois_org")); + udfContext.setLookupFields(Collections.singletonList("domain")); + udfContext.setOutputFields(Collections.singletonList("domain_whois_org")); RuntimeContext runtimeContext = mockRuntimeContext(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/H3CellLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/H3CellLookupTest.java new file mode 100644 index 0000000..a7b98ab --- /dev/null +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/H3CellLookupTest.java @@ -0,0 +1,65 @@ +package com.geedgenetworks.core.udf.cn; + +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.UDFContext; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * @author gujinkai + * @version 1.0 + * @date 2024/9/6 16:15 + */ +public class H3CellLookupTest { + + + private static H3CellLookup h3CellLookup; + + @BeforeAll + static void setUp() { + /** + * Create a UDFContext object and set the parameters + * e.g.: + * - function: BASE_STATION_LOOKUP + * lookup_fields: [ subscriber_longitude,subscriber_latitude ] + * output_fields: [ first_location ] + * parameters: + * resolution: 9 + */ + UDFContext udfContext = new UDFContext(); + udfContext.setLookupFields(Arrays.asList("subscriber_longitude", "subscriber_latitude")); + udfContext.setOutputFields(Arrays.asList("first_location")); + Map<String, Object> parameters = new HashMap<>(); + parameters.put("resolution", 9); + udfContext.setParameters(parameters); + + RuntimeContext runtimeContext = Mockito.mock(RuntimeContext.class); + + /*String content = "cell_id,longitude,latitude\n460-11-630947-1,93.9290001,42.66884"; + mockKnowledgeBaseHandler(content);*/ + + h3CellLookup = new H3CellLookup(); + h3CellLookup.open(runtimeContext, udfContext); + } + + @Test + void testValueExpression() throws IOException { + Event event = new Event(); + Map<String, Object> fields = new HashMap<>(); + fields.put("subscriber_longitude", 116.390249); + fields.put("subscriber_latitude", 39.905392); + event.setExtractedFields(fields); + Event evaluate = h3CellLookup.evaluate(event); + assertEquals("8931aa42853ffff", evaluate.getExtractedFields().get("first_location")); + + } +} diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IcpLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IcpLookupTest.java index 3158124..c2032e0 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IcpLookupTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IcpLookupTest.java @@ -24,8 +24,8 @@ class IcpLookupTest { Map<String, Object> parameters = new HashMap<>(); parameters.put("kb_name", kbName); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("domain")); - udfContext.setOutput_fields(Collections.singletonList("domain_icp_company_name")); + udfContext.setLookupFields(Collections.singletonList("domain")); + udfContext.setOutputFields(Collections.singletonList("domain_icp_company_name")); RuntimeContext runtimeContext = mockRuntimeContext(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IdcRenterLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IdcRenterLookupTest.java index b15096b..7409a2f 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IdcRenterLookupTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IdcRenterLookupTest.java @@ -24,8 +24,8 @@ class IdcRenterLookupTest { Map<String, Object> parameters = new HashMap<>(); parameters.put("kb_name", kbName); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("server_ip")); - udfContext.setOutput_fields(Collections.singletonList("server_idc_renter")); + udfContext.setLookupFields(Collections.singletonList("server_ip")); + udfContext.setOutputFields(Collections.singletonList("server_idc_renter")); RuntimeContext runtimeContext = mockRuntimeContext(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IntelligenceIndicatorLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IntelligenceIndicatorLookupTest.java index 34fef6b..7d643f4 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IntelligenceIndicatorLookupTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IntelligenceIndicatorLookupTest.java @@ -27,7 +27,7 @@ public class IntelligenceIndicatorLookupTest { void setUp() { runtimeContext = mockRuntimeContext(); - String content = "type,ip_addr_format,ip1,ip2,domain,tags\nIP,CIDR,116.178.65.0,25,ali.com,\"阿里1,云服务1\"\nDomain,CIDR,116.178.65.0,25,$ali.com,\"阿里2,云服务2\"\nDomain,CIDR,116.178.65.0,25,*baidu.com,\"阿里3,云服务3\"\nIP,Single,116.178.65.64,116.178.65.64,ali.com,\"test\""; + String content = "type,ip_addr_format,ip1,ip2,domain,subscriber,tags\nIP,CIDR,116.178.65.0,25,ali.com,,\"阿里1,云服务1\"\nDomain,CIDR,116.178.65.0,25,$ali.com,,\"阿里2,云服务2\"\nDomain,CIDR,116.178.65.0,25,*baidu.com,,\"阿里3,云服务3\"\nIP,Single,116.178.65.64,116.178.65.64,ali.com,,\"test\"\nSubscriber,Single,116.178.65.64,116.178.65.64,ali.com,1234567,\"test_subscriber\""; mockKnowledgeBaseHandler(content); intelligenceIndicatorLookup = new IntelligenceIndicatorLookup(); @@ -40,8 +40,8 @@ public class IntelligenceIndicatorLookupTest { parameters.put("kb_name", kbName); parameters.put("option", "IP_TO_TAG"); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("server_ip")); - udfContext.setOutput_fields(Collections.singletonList("server_ip_tags")); + udfContext.setLookupFields(Collections.singletonList("server_ip")); + udfContext.setOutputFields(Collections.singletonList("server_ip_tags")); intelligenceIndicatorLookup.open(runtimeContext, udfContext); Event event = new Event(); @@ -59,8 +59,8 @@ public class IntelligenceIndicatorLookupTest { parameters.put("kb_name", kbName); parameters.put("option", "IP_TO_TAG"); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("server_ip")); - udfContext.setOutput_fields(Collections.singletonList("server_ip_tags")); + udfContext.setLookupFields(Collections.singletonList("server_ip")); + udfContext.setOutputFields(Collections.singletonList("server_ip_tags")); intelligenceIndicatorLookup.open(runtimeContext, udfContext); Event event = new Event(); @@ -82,8 +82,8 @@ public class IntelligenceIndicatorLookupTest { parameters.put("kb_name", kbName); parameters.put("option", "DOMAIN_TO_TAG"); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("domain")); - udfContext.setOutput_fields(Collections.singletonList("domain_tags")); + udfContext.setLookupFields(Collections.singletonList("domain")); + udfContext.setOutputFields(Collections.singletonList("domain_tags")); intelligenceIndicatorLookup.open(runtimeContext, udfContext); Event event = new Event(); @@ -101,8 +101,8 @@ public class IntelligenceIndicatorLookupTest { parameters.put("kb_name", kbName); parameters.put("option", "DOMAIN_TO_TAG"); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("domain")); - udfContext.setOutput_fields(Collections.singletonList("domain_tags")); + udfContext.setLookupFields(Collections.singletonList("domain")); + udfContext.setOutputFields(Collections.singletonList("domain_tags")); intelligenceIndicatorLookup.open(runtimeContext, udfContext); Event event = new Event(); @@ -124,8 +124,8 @@ public class IntelligenceIndicatorLookupTest { parameters.put("kb_name", kbName); parameters.put("option", "DOMAIN_TO_TAG"); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("domain")); - udfContext.setOutput_fields(Collections.singletonList("domain_tags")); + udfContext.setLookupFields(Collections.singletonList("domain")); + udfContext.setOutputFields(Collections.singletonList("domain_tags")); intelligenceIndicatorLookup.open(runtimeContext, udfContext); Event event = new Event(); @@ -143,8 +143,8 @@ public class IntelligenceIndicatorLookupTest { parameters.put("kb_name", kbName); parameters.put("option", "IP_TO_TAG"); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("server_ip")); - udfContext.setOutput_fields(Collections.singletonList("server_ip_tags")); + udfContext.setLookupFields(Collections.singletonList("server_ip")); + udfContext.setOutputFields(Collections.singletonList("server_ip_tags")); intelligenceIndicatorLookup.open(runtimeContext, udfContext); Event event = new Event(); @@ -155,6 +155,25 @@ public class IntelligenceIndicatorLookupTest { assertEquals(Arrays.asList("阿里1", "云服务1", "test"), evaluate1.getExtractedFields().get("server_ip_tags")); } + @Test + void evaluate7() { + UDFContext udfContext = new UDFContext(); + Map<String, Object> parameters = new HashMap<>(); + parameters.put("kb_name", kbName); + parameters.put("option", "SUBSCRIBER_TO_TAG"); + udfContext.setParameters(parameters); + udfContext.setLookupFields(Collections.singletonList("subscriber_id")); + udfContext.setOutputFields(Collections.singletonList("subscriber_tags")); + intelligenceIndicatorLookup.open(runtimeContext, udfContext); + + Event event = new Event(); + Map<String, Object> fields = new HashMap<>(); + fields.put("subscriber_id", "1234567"); + event.setExtractedFields(fields); + Event evaluate1 = intelligenceIndicatorLookup.evaluate(event); + assertEquals(Arrays.asList("test_subscriber"), evaluate1.getExtractedFields().get("subscriber_tags")); + } + @AfterEach void afterAll() { clearState(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IocLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IocLookupTest.java index f9d3b25..8c01bc7 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IocLookupTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IocLookupTest.java @@ -37,8 +37,8 @@ class IocLookupTest { parameters.put("kb_name", kbName); parameters.put("option", "IP_TO_MALWARE"); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("server_ip")); - udfContext.setOutput_fields(Collections.singletonList("server_malware")); + udfContext.setLookupFields(Collections.singletonList("server_ip")); + udfContext.setOutputFields(Collections.singletonList("server_malware")); iocLookup.open(runtimeContext, udfContext); Event event = new Event(); @@ -56,8 +56,8 @@ class IocLookupTest { parameters.put("kb_name", kbName); parameters.put("option", "DOMAIN_TO_MALWARE"); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("domain")); - udfContext.setOutput_fields(Collections.singletonList("domain_malware")); + udfContext.setLookupFields(Collections.singletonList("domain")); + udfContext.setOutputFields(Collections.singletonList("domain_malware")); iocLookup.open(runtimeContext, udfContext); Event event = new Event(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IpZoneLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IpZoneLookupTest.java index abe5ba0..e3024b5 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IpZoneLookupTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/IpZoneLookupTest.java @@ -24,8 +24,8 @@ class IpZoneLookupTest { Map<String, Object> parameters = new HashMap<>(); parameters.put("kb_name", kbName); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("client_ip")); - udfContext.setOutput_fields(Collections.singletonList("client_zone")); + udfContext.setLookupFields(Collections.singletonList("client_ip")); + udfContext.setOutputFields(Collections.singletonList("client_zone")); RuntimeContext runtimeContext = mockRuntimeContext(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/LinkDirectionLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/LinkDirectionLookupTest.java index c0a06fe..4f2f551 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/LinkDirectionLookupTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/LinkDirectionLookupTest.java @@ -24,8 +24,8 @@ class LinkDirectionLookupTest { Map<String, Object> parameters = new HashMap<>(); parameters.put("kb_name", kbName); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("in_link_id")); - udfContext.setOutput_fields(Collections.singletonList("in_link_direction")); + udfContext.setLookupFields(Collections.singletonList("in_link_id")); + udfContext.setOutputFields(Collections.singletonList("in_link_direction")); RuntimeContext runtimeContext = mockRuntimeContext(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/LookupTestUtils.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/LookupTestUtils.java index 05df41d..bf95c57 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/LookupTestUtils.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/LookupTestUtils.java @@ -47,6 +47,20 @@ public class LookupTestUtils { private static MockedStatic<AbstractMultipleKnowledgeBaseHandler> abstractMultipleKnowledgeBaseHandlerMockedStatic = mockStatic(AbstractMultipleKnowledgeBaseHandler.class); + /** + * mock runtime context + * the configuration is set in the global job parameters + * the configuration contains the knowledge base configuration + * the knowledge base configuration contains the knowledge base name and file path and file type and file list + * e.g.: + * - name: cn_ip_location + * fs_type: http + * fs_path: http://192.168.44.55:9999/v1/knowledge_base + * files: + * - 1 + * + * @return runtime context + */ static RuntimeContext mockRuntimeContext() { RuntimeContext runtimeContext = Mockito.mock(RuntimeContext.class); ExecutionConfig executionConfig = Mockito.mock(ExecutionConfig.class); @@ -77,6 +91,12 @@ public class LookupTestUtils { } } + /** + * mock knowledge base handler + * the knowledge base handler is used to get the metadata and read the file content + * + * @param downloadContent download content + */ static void mockKnowledgeBaseHandler(String downloadContent) { checkStaticMock(); KnowLedgeBaseFileMeta knowLedgeBaseFileMeta = new KnowLedgeBaseFileMeta(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/UserDefineTagLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/UserDefineTagLookupTest.java index 9688199..a586aeb 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/UserDefineTagLookupTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/UserDefineTagLookupTest.java @@ -37,8 +37,8 @@ class UserDefineTagLookupTest { parameters.put("kb_name", kbName); parameters.put("option", "IP_TO_TAG"); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("client_ip")); - udfContext.setOutput_fields(Collections.singletonList("client_ip_tags")); + udfContext.setLookupFields(Collections.singletonList("client_ip")); + udfContext.setOutputFields(Collections.singletonList("client_ip_tags")); userDefineTagLookup.open(runtimeContext, udfContext); Event event = new Event(); @@ -59,8 +59,8 @@ class UserDefineTagLookupTest { parameters.put("kb_name", kbName); parameters.put("option", "DOMAIN_TO_TAG"); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("domain")); - udfContext.setOutput_fields(Collections.singletonList("domain_tags")); + udfContext.setLookupFields(Collections.singletonList("domain")); + udfContext.setOutputFields(Collections.singletonList("domain_tags")); userDefineTagLookup.open(runtimeContext, udfContext); Event event = new Event(); @@ -81,8 +81,8 @@ class UserDefineTagLookupTest { parameters.put("kb_name", kbName); parameters.put("option", "DOMAIN_TO_TAG"); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("domain")); - udfContext.setOutput_fields(Collections.singletonList("domain_tags")); + udfContext.setLookupFields(Collections.singletonList("domain")); + udfContext.setOutputFields(Collections.singletonList("domain_tags")); userDefineTagLookup.open(runtimeContext, udfContext); Event event = new Event(); @@ -103,8 +103,8 @@ class UserDefineTagLookupTest { parameters.put("kb_name", kbName); parameters.put("option", "APP_TO_TAG"); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("app")); - udfContext.setOutput_fields(Collections.singletonList("app_tags")); + udfContext.setLookupFields(Collections.singletonList("app")); + udfContext.setOutputFields(Collections.singletonList("app_tags")); userDefineTagLookup.open(runtimeContext, udfContext); Event event = new Event(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/VpnLookupTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/VpnLookupTest.java index 3dd0992..50374f7 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/VpnLookupTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/cn/VpnLookupTest.java @@ -30,8 +30,8 @@ class VpnLookupTest { parameters.put("kb_name", kbName); parameters.put("option", "IP_TO_VPN"); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("server_ip")); - udfContext.setOutput_fields(Collections.singletonList("server_vpn_service_name")); + udfContext.setLookupFields(Collections.singletonList("server_ip")); + udfContext.setOutputFields(Collections.singletonList("server_vpn_service_name")); RuntimeContext runtimeContext = mockRuntimeContext(); @@ -55,8 +55,8 @@ class VpnLookupTest { parameters.put("kb_name", kbName); parameters.put("option", "DOMAIN_TO_VPN"); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("domain")); - udfContext.setOutput_fields(Collections.singletonList("domain_vpn_service_name")); + udfContext.setLookupFields(Collections.singletonList("domain")); + udfContext.setOutputFields(Collections.singletonList("domain_vpn_service_name")); RuntimeContext runtimeContext = mockRuntimeContext(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/AsnLookupFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/AsnLookupFunctionTest.java index 1e5c0a3..f10fe2b 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/AsnLookupFunctionTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/AsnLookupFunctionTest.java @@ -26,8 +26,8 @@ public class AsnLookupFunctionTest { udfContext = new UDFContext(); parameters = new HashMap<>(); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("ip")); - udfContext.setOutput_fields(Collections.singletonList("asn")); + udfContext.setLookupFields(Collections.singletonList("ip")); + udfContext.setOutputFields(Collections.singletonList("asn")); } @@ -36,7 +36,7 @@ public class AsnLookupFunctionTest { @Test public void testInit(){ AsnLookup asnLookup = new AsnLookup(); - udfContext.setLookup_fields(new ArrayList<>()); + udfContext.setLookupFields(new ArrayList<>()); udfContext.setParameters(new HashMap<>()); udfContext.setParameters(null); Assertions.assertThrows(GrootStreamRuntimeException.class, () -> { @@ -48,10 +48,10 @@ public class AsnLookupFunctionTest { asnLookup.open(null, udfContext); }); - udfContext.setLookup_fields(new ArrayList<>()); - udfContext.getLookup_fields().add("v1"); - udfContext.setOutput_fields(new ArrayList<>()); - udfContext.getOutput_fields().add("v2"); + udfContext.setLookupFields(new ArrayList<>()); + udfContext.getLookupFields().add("v1"); + udfContext.setOutputFields(new ArrayList<>()); + udfContext.getOutputFields().add("v2"); udfContext.setParameters(new HashMap<>()); udfContext.getParameters().put("option","other"); Assertions.assertThrows(GrootStreamRuntimeException.class, () -> { diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/GeoIpLookupFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/GeoIpLookupFunctionTest.java index bc67f1a..83f8ab4 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/GeoIpLookupFunctionTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/GeoIpLookupFunctionTest.java @@ -25,14 +25,14 @@ public class GeoIpLookupFunctionTest { @BeforeAll public static void setUp() { udfContext = new UDFContext(); - udfContext.setLookup_fields(Collections.singletonList("ip")); - udfContext.setOutput_fields(Collections.singletonList("iplocation")); + udfContext.setLookupFields(Collections.singletonList("ip")); + udfContext.setOutputFields(Collections.singletonList("iplocation")); } @Test public void testInit(){ GeoIpLookup geoIpLookup = new GeoIpLookup(); - udfContext.setLookup_fields(new ArrayList<>()); + udfContext.setLookupFields(new ArrayList<>()); udfContext.setParameters(new HashMap<>()); udfContext.setParameters(null); Assertions.assertThrows(GrootStreamRuntimeException.class, () -> { @@ -44,10 +44,10 @@ public class GeoIpLookupFunctionTest { geoIpLookup.open(null, udfContext); }); - udfContext.setLookup_fields(new ArrayList<>()); - udfContext.getLookup_fields().add("v1"); - udfContext.setOutput_fields(new ArrayList<>()); - udfContext.getOutput_fields().add("v2"); + udfContext.setLookupFields(new ArrayList<>()); + udfContext.getLookupFields().add("v1"); + udfContext.setOutputFields(new ArrayList<>()); + udfContext.getOutputFields().add("v2"); udfContext.setParameters(new HashMap<>()); udfContext.getParameters().put("option","other"); Assertions.assertThrows(GrootStreamRuntimeException.class, () -> { diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectListTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectListTest.java index 2bf13a5..16d5cce 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectListTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectListTest.java @@ -1,18 +1,3 @@ -/** - * Copyright 2017 Hortonworks. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - - * http://www.apache.org/licenses/LICENSE-2.0 - - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - **/ package com.geedgenetworks.core.udf.test.aggregate; @@ -45,8 +30,8 @@ public class CollectListTest { private void testMerge(List<String> arr,List<String> arr2) { UDFContext udfContext = new UDFContext(); - udfContext.setLookup_fields(List.of("field")); - udfContext.setOutput_fields(Collections.singletonList("field_list")); + udfContext.setLookupFields(List.of("field")); + udfContext.setOutputFields(Collections.singletonList("field_list")); CollectList collectList = new CollectList(); Map<String, Object> metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); @@ -83,8 +68,8 @@ public class CollectListTest { private void testGetResult(List<String> arr) throws ParseException { UDFContext udfContext = new UDFContext(); - udfContext.setLookup_fields(List.of("field")); - udfContext.setOutput_fields(Collections.singletonList("field_list")); + udfContext.setLookupFields(List.of("field")); + udfContext.setOutputFields(Collections.singletonList("field_list")); CollectList collectList = new CollectList(); Map<String, Object> metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectSetTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectSetTest.java index 8e992f6..8909794 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectSetTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/CollectSetTest.java @@ -1,18 +1,3 @@ -/** - * Copyright 2017 Hortonworks. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - - * http://www.apache.org/licenses/LICENSE-2.0 - - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - **/ package com.geedgenetworks.core.udf.test.aggregate; @@ -43,8 +28,8 @@ public class CollectSetTest { private void testMerge(List<String> arr,List<String> arr2) { UDFContext udfContext = new UDFContext(); - udfContext.setLookup_fields(List.of("field")); - udfContext.setOutput_fields(Collections.singletonList("field_list")); + udfContext.setLookupFields(List.of("field")); + udfContext.setOutputFields(Collections.singletonList("field_list")); CollectSet collectSet = new CollectSet(); Map<String, Object> metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); @@ -80,8 +65,8 @@ public class CollectSetTest { private static void testGetResult(List<String> arr) throws ParseException { UDFContext udfContext = new UDFContext(); - udfContext.setLookup_fields(List.of("field")); - udfContext.setOutput_fields(Collections.singletonList("field_list")); + udfContext.setLookupFields(List.of("field")); + udfContext.setOutputFields(Collections.singletonList("field_list")); CollectSet collectSet = new CollectSet(); Map<String, Object> metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/FirstValueTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/FirstValueTest.java index 43a9732..0acf1d5 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/FirstValueTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/FirstValueTest.java @@ -1,18 +1,3 @@ -/** - * Copyright 2017 Hortonworks. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - - * http://www.apache.org/licenses/LICENSE-2.0 - - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - **/ package com.geedgenetworks.core.udf.test.aggregate; @@ -42,8 +27,8 @@ public class FirstValueTest { private void testMerge(List<String> arr,List<String> arr2) { UDFContext udfContext = new UDFContext(); - udfContext.setLookup_fields(List.of("field")); - udfContext.setOutput_fields(Collections.singletonList("field_first")); + udfContext.setLookupFields(List.of("field")); + udfContext.setOutputFields(Collections.singletonList("field_first")); FirstValue firstValue = new FirstValue(); Map<String, Object> metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); @@ -78,8 +63,8 @@ public class FirstValueTest { private static void testGetResult(List<String> arr) throws ParseException { UDFContext udfContext = new UDFContext(); - udfContext.setLookup_fields(List.of("field")); - udfContext.setOutput_fields(Collections.singletonList("field_first")); + udfContext.setLookupFields(List.of("field")); + udfContext.setOutputFields(Collections.singletonList("field_first")); FirstValue firstValue = new FirstValue(); Map<String, Object> metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LastValueTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LastValueTest.java index e952908..b2c9ceb 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LastValueTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LastValueTest.java @@ -1,18 +1,3 @@ -/** - * Copyright 2017 Hortonworks. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - - * http://www.apache.org/licenses/LICENSE-2.0 - - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - **/ package com.geedgenetworks.core.udf.test.aggregate; @@ -45,8 +30,8 @@ public class LastValueTest { private void testMerge(List<String> arr,List<String> arr2) { UDFContext udfContext = new UDFContext(); - udfContext.setLookup_fields(List.of("field")); - udfContext.setOutput_fields(Collections.singletonList("field_last")); + udfContext.setLookupFields(List.of("field")); + udfContext.setOutputFields(Collections.singletonList("field_last")); LastValue lastValue = new LastValue(); Map<String, Object> metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); @@ -81,8 +66,8 @@ public class LastValueTest { private static void testGetResult(List<String> arr) throws ParseException { UDFContext udfContext = new UDFContext(); - udfContext.setLookup_fields(List.of("field")); - udfContext.setOutput_fields(Collections.singletonList("field_last")); + udfContext.setLookupFields(List.of("field")); + udfContext.setOutputFields(Collections.singletonList("field_last")); LastValue lastValue = new LastValue(); Map<String, Object> metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LongCountTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LongCountTest.java index c1dfb9e..54d9dba 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LongCountTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/LongCountTest.java @@ -1,18 +1,3 @@ -/** - * Copyright 2017 Hortonworks. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - - * http://www.apache.org/licenses/LICENSE-2.0 - - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - **/ package com.geedgenetworks.core.udf.test.aggregate; @@ -46,8 +31,8 @@ public class LongCountTest { private void testMerge(Number[] arr,Number[] arr2) { UDFContext udfContext = new UDFContext(); - udfContext.setLookup_fields(List.of("field")); - udfContext.setOutput_fields(Collections.singletonList("count")); + udfContext.setLookupFields(List.of("field")); + udfContext.setOutputFields(Collections.singletonList("count")); LongCount longCount = new LongCount(); Map<String, Object> metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); @@ -81,7 +66,7 @@ public class LongCountTest { private static void testGetResult(Number[] arr) throws ParseException { UDFContext udfContext = new UDFContext(); - udfContext.setOutput_fields(Collections.singletonList("count")); + udfContext.setOutputFields(Collections.singletonList("count")); LongCount longCount = new LongCount(); Map<String, Object> metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MaxTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MaxTest.java new file mode 100644 index 0000000..311d51f --- /dev/null +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MaxTest.java @@ -0,0 +1,144 @@ +package com.geedgenetworks.core.udf.test.aggregate; + + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.core.udf.udaf.Max; +import com.geedgenetworks.core.udf.udaf.Min; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.time.LocalDateTime; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +public class MaxTest { + + private Max maxFunction; + private Accumulator acc; + private Event event; + + @BeforeEach + void setUp() { + maxFunction = new Max(); + acc = new Accumulator(); + event = new Event(); + + // 初始化上下文 + UDFContext udfContext = new UDFContext(); + udfContext.setLookupFields(List.of("value")); + udfContext.setOutputFields(List.of("maxValue")); + maxFunction.open(udfContext); + + // 初始化累加器的 metricsFields + acc.setMetricsFields(new HashMap<>()); + } + + @Test + void testAddMultipleNumericValues() { + // 设置事件中的多个数值 + Map<String, Object> extractedFields = new HashMap<>(); + extractedFields.put("value", 100); + event.setExtractedFields(extractedFields); + Accumulator result = maxFunction.add(event, acc); + + // 添加第二个数值 + extractedFields.put("value", 200); + result = maxFunction.add(event, result); + + // 添加第三个数值 + extractedFields.put("value", 150); + result = maxFunction.add(event, result); + + // 验证最大值应该是200 + assertEquals(200, result.getMetricsFields().get("maxValue")); + } + + @Test + void testAddMultipleStringValues() { + // 设置事件中的多个字符串值 + Map<String, Object> extractedFields = new HashMap<>(); + extractedFields.put("value", "abc"); + event.setExtractedFields(extractedFields); + Accumulator result = maxFunction.add(event, acc); + + // 添加第二个字符串值 + extractedFields.put("value", "def"); + result = maxFunction.add(event, result); + + // 添加第三个字符串值 + extractedFields.put("value", "ghi"); + result = maxFunction.add(event, result); + + // 验证最大字符串值(按字母顺序最大的是 "ghi") + assertEquals("ghi", result.getMetricsFields().get("maxValue")); + } + + @Test + void testAddMultipleLocalDateTimes() { + // 设置事件中的多个时间值 + Map<String, Object> extractedFields = new HashMap<>(); + + extractedFields.put("value", LocalDateTime.of(2024, 10, 22, 12, 0)); + event.setExtractedFields(extractedFields); + Accumulator result = maxFunction.add(event, acc); + + // 添加第二个时间值 + extractedFields.put("value", LocalDateTime.of(2024, 10, 23, 12, 0)); + result = maxFunction.add(event, result); + + // 添加第三个时间值 + extractedFields.put("value", LocalDateTime.of(2024, 10, 21, 12, 0)); + result = maxFunction.add(event, result); + + // 验证最大时间值(最大的是2024年10月23日12点) + assertEquals(LocalDateTime.of(2024, 10, 23, 12, 0), result.getMetricsFields().get("maxValue")); + } + + @Test + void testMergeAccumulatorsForNumber() { + Accumulator acc1 = new Accumulator(); + Accumulator acc2 = new Accumulator(); + Map map1 = new HashMap(); + map1.put("maxValue", 10L); + Map map2 = new HashMap(); + map2.put("maxValue", 5.0); + acc1.setMetricsFields(map1); + acc2.setMetricsFields(map2); + Accumulator mergedAcc = maxFunction.merge(acc1, acc2); + assertEquals(10L, mergedAcc.getMetricsFields().get("maxValue")); + } + + @Test + void testMergeAccumulatorsForTime() { + Accumulator acc1 = new Accumulator(); + Accumulator acc2 = new Accumulator(); + Map map1 = new HashMap(); + map1.put("maxValue", LocalDateTime.of(2023, 1, 1, 0, 0,1)); + Map map2 = new HashMap(); + map2.put("maxValue", LocalDateTime.of(2023, 1, 1, 0, 0,2)); + acc1.setMetricsFields(map1); + acc2.setMetricsFields(map2); + Accumulator mergedAcc = maxFunction.merge(acc1, acc2); + assertEquals(LocalDateTime.of(2023, 1, 1, 0, 0,2), mergedAcc.getMetricsFields().get("maxValue")); + } + + @Test + void testMergeAccumulatorsForString() { + Accumulator acc1 = new Accumulator(); + Accumulator acc2 = new Accumulator(); + Map map1 = new HashMap(); + map1.put("maxValue", "qwe"); + Map map2 = new HashMap(); + map2.put("maxValue", "abc"); + acc1.setMetricsFields(map1); + acc2.setMetricsFields(map2); + Accumulator mergedAcc = maxFunction.merge(acc1, acc2); + assertEquals("qwe", mergedAcc.getMetricsFields().get("maxValue")); + } +}
\ No newline at end of file diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MeanTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MeanTest.java index cc4eaf0..62efc0a 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MeanTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MeanTest.java @@ -1,18 +1,3 @@ -/** - * Copyright 2017 Hortonworks. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - - * http://www.apache.org/licenses/LICENSE-2.0 - - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - **/ package com.geedgenetworks.core.udf.test.aggregate; @@ -46,8 +31,8 @@ public class MeanTest { } private void testMerge(Number[] arr1,Number[] arr2,int precision) throws ParseException { UDFContext udfContext = new UDFContext(); - udfContext.setLookup_fields(List.of("field")); - udfContext.setOutput_fields(Collections.singletonList("field_mean")); + udfContext.setLookupFields(List.of("field")); + udfContext.setOutputFields(Collections.singletonList("field_mean")); udfContext.setParameters(new HashMap<>()); udfContext.getParameters().put("precision", precision); Mean mean = new Mean(); @@ -62,8 +47,8 @@ public class MeanTest { } private Accumulator getMiddleResult(Number[] arr,int precision) throws ParseException { UDFContext udfContext = new UDFContext(); - udfContext.setLookup_fields(List.of("field")); - udfContext.setOutput_fields(Collections.singletonList("field_mean")); + udfContext.setLookupFields(List.of("field")); + udfContext.setOutputFields(Collections.singletonList("field_mean")); udfContext.setParameters(new HashMap<>()); udfContext.getParameters().put("precision", precision); Mean mean = new Mean(); @@ -87,8 +72,8 @@ public class MeanTest { private void testInt(Number[] arr,int precision) throws ParseException { UDFContext udfContext = new UDFContext(); - udfContext.setLookup_fields(List.of("field")); - udfContext.setOutput_fields(Collections.singletonList("field_mean")); + udfContext.setLookupFields(List.of("field")); + udfContext.setOutputFields(Collections.singletonList("field_mean")); udfContext.setParameters(new HashMap<>()); udfContext.getParameters().put("precision", precision); Mean mean = new Mean(); @@ -113,8 +98,8 @@ public class MeanTest { private void testDouble(Number[] arr,int precision) throws ParseException { UDFContext udfContext = new UDFContext(); - udfContext.setLookup_fields(List.of("field")); - udfContext.setOutput_fields(Collections.singletonList("field_mean")); + udfContext.setLookupFields(List.of("field")); + udfContext.setOutputFields(Collections.singletonList("field_mean")); udfContext.setParameters(new HashMap<>()); udfContext.getParameters().put("precision", precision); Mean mean = new Mean(); @@ -138,8 +123,8 @@ public class MeanTest { private void testNoPrecision(Number[] arr) throws ParseException { UDFContext udfContext = new UDFContext(); - udfContext.setLookup_fields(List.of("field")); - udfContext.setOutput_fields(Collections.singletonList("field_mean")); + udfContext.setLookupFields(List.of("field")); + udfContext.setOutputFields(Collections.singletonList("field_mean")); Mean mean = new Mean(); Map<String, Object> metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MinTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MinTest.java new file mode 100644 index 0000000..e5a1615 --- /dev/null +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/MinTest.java @@ -0,0 +1,123 @@ +package com.geedgenetworks.core.udf.test.aggregate; + + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.core.udf.udaf.Min; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.time.LocalDateTime; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class MinTest { + + private Min minFunction; + private UDFContext udfContext; + private Accumulator acc; + private Event event1, event2, event3; + + @BeforeEach + void setUp() { + minFunction = new Min(); + udfContext = new UDFContext(); + udfContext.setLookupFields(List.of("value")); + udfContext.setOutputFields(List.of("minValue")); + minFunction.open(udfContext); + + acc = new Accumulator(); + acc.setMetricsFields(new HashMap<>()); + + event1 = new Event(); + event2 = new Event(); + event3 = new Event(); + } + + @Test + void testAddNumericValue() { + event1.setExtractedFields(Map.of("value", 5.0)); + event2.setExtractedFields(Map.of("value", 3.0)); + event3.setExtractedFields(Map.of("value", 10.0)); + + acc = minFunction.add(event1, acc); + acc = minFunction.add(event2, acc); + acc = minFunction.add(event3, acc); + + assertEquals(3.0, acc.getMetricsFields().get("minValue")); + } + + @Test + void testAddStringValue() { + event1.setExtractedFields(Map.of("value", "apple")); + event2.setExtractedFields(Map.of("value", "banana")); + event3.setExtractedFields(Map.of("value", "cherry")); + + acc = minFunction.add(event1, acc); + acc = minFunction.add(event2, acc); + acc = minFunction.add(event3, acc); + + assertEquals("apple", acc.getMetricsFields().get("minValue")); + } + + @Test + void testAddLocalDateTime() { + event1.setExtractedFields(Map.of("value", LocalDateTime.of(2023, 1, 1, 0, 0))); + event2.setExtractedFields(Map.of("value", LocalDateTime.of(2022, 1, 1, 0, 0))); + event3.setExtractedFields(Map.of("value", LocalDateTime.of(2024, 1, 1, 0, 0))); + + acc = minFunction.add(event1, acc); + acc = minFunction.add(event2, acc); + acc = minFunction.add(event3, acc); + + assertEquals(LocalDateTime.of(2022, 1, 1, 0, 0), acc.getMetricsFields().get("minValue")); + } + + @Test + void testMergeAccumulatorsForNumber() { + Accumulator acc1 = new Accumulator(); + Accumulator acc2 = new Accumulator(); + Map map1 = new HashMap(); + map1.put("minValue", 10L); + Map map2 = new HashMap(); + map2.put("minValue", 5.0); + acc1.setMetricsFields(map1); + acc2.setMetricsFields(map2); + Accumulator mergedAcc = minFunction.merge(acc1, acc2); + assertEquals(5.0, mergedAcc.getMetricsFields().get("minValue")); + } + + @Test + void testMergeAccumulatorsForTime() { + Accumulator acc1 = new Accumulator(); + Accumulator acc2 = new Accumulator(); + Map map1 = new HashMap(); + map1.put("minValue", LocalDateTime.of(2023, 1, 1, 0, 0,1)); + Map map2 = new HashMap(); + map2.put("minValue", LocalDateTime.of(2023, 1, 1, 0, 0,2)); + acc1.setMetricsFields(map1); + acc2.setMetricsFields(map2); + Accumulator mergedAcc = minFunction.merge(acc1, acc2); + assertEquals(LocalDateTime.of(2023, 1, 1, 0, 0,1), mergedAcc.getMetricsFields().get("minValue")); + } + + @Test + void testMergeAccumulatorsForString() { + Accumulator acc1 = new Accumulator(); + Accumulator acc2 = new Accumulator(); + Map map1 = new HashMap(); + map1.put("minValue", "qwe"); + Map map2 = new HashMap(); + map2.put("minValue", "abc"); + acc1.setMetricsFields(map1); + acc2.setMetricsFields(map2); + Accumulator mergedAcc = minFunction.merge(acc1, acc2); + assertEquals("abc", mergedAcc.getMetricsFields().get("minValue")); + } + + +}
\ No newline at end of file diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/NumberSumTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/NumberSumTest.java index a4072ca..7ccb365 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/NumberSumTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/aggregate/NumberSumTest.java @@ -48,8 +48,8 @@ public class NumberSumTest { private void testMerge(Number[] arr,Number[] arr2) { UDFContext udfContext = new UDFContext(); - udfContext.setLookup_fields(List.of("field")); - udfContext.setOutput_fields(Collections.singletonList("field_sum")); + udfContext.setLookupFields(List.of("field")); + udfContext.setOutputFields(Collections.singletonList("field_sum")); NumberSum numberSum = new NumberSum(); Map<String, Object> metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); @@ -83,8 +83,8 @@ public class NumberSumTest { private static void excute(Number[] arr, Class<? extends Number> clazz) throws ParseException { UDFContext udfContext = new UDFContext(); - udfContext.setLookup_fields(List.of("field")); - udfContext.setOutput_fields(Collections.singletonList("field_sum")); + udfContext.setLookupFields(List.of("field")); + udfContext.setOutputFields(Collections.singletonList("field_sum")); NumberSum numberSum = new NumberSum(); Map<String, Object> metricsFields = new HashMap<>(); Accumulator accumulator = new Accumulator(); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DecodeBase64FunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DecodeBase64FunctionTest.java index 1d3b863..a5f31f7 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DecodeBase64FunctionTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DecodeBase64FunctionTest.java @@ -22,8 +22,8 @@ public class DecodeBase64FunctionTest { @BeforeAll public static void setUp() { udfContext = new UDFContext(); - udfContext.setLookup_fields(Arrays.asList("message", "charset")); - udfContext.setOutput_fields(Collections.singletonList("decodeResult")); + udfContext.setLookupFields(Arrays.asList("message", "charset")); + udfContext.setOutputFields(Collections.singletonList("decodeResult")); Map<String,Object> map = new HashMap<>(); map.put("value_field","message"); map.put("charset_field","charset"); @@ -51,8 +51,8 @@ public class DecodeBase64FunctionTest { DecodeBase64 decodeBase64 = new DecodeBase64(); - udfContext.setLookup_fields(Collections.singletonList("message")); - udfContext.setOutput_fields(Collections.singletonList("decodeResult")); + udfContext.setLookupFields(Collections.singletonList("message")); + udfContext.setOutputFields(Collections.singletonList("decodeResult")); udfContext.getParameters().remove("value_field"); assertThrows(GrootStreamRuntimeException.class, () -> { diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DomainFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DomainFunctionTest.java index 2126117..f8076cc 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DomainFunctionTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DomainFunctionTest.java @@ -24,15 +24,15 @@ public class DomainFunctionTest { udfContext = new UDFContext(); parameters = new HashMap<>(); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("domain")); - udfContext.setOutput_fields(Collections.singletonList("domain1")); + udfContext.setLookupFields(Collections.singletonList("domain")); + udfContext.setOutputFields(Collections.singletonList("domain1")); } @Test public void testInit(){ Domain domain = new Domain(); - udfContext.setLookup_fields(new ArrayList<>()); + udfContext.setLookupFields(new ArrayList<>()); udfContext.setParameters(new HashMap<>()); udfContext.setParameters(null); Assertions.assertThrows(GrootStreamRuntimeException.class, () -> { @@ -44,10 +44,10 @@ public class DomainFunctionTest { domain.open(null, udfContext); }); - udfContext.setLookup_fields(new ArrayList<>()); - udfContext.getLookup_fields().add("v1"); - udfContext.setOutput_fields(new ArrayList<>()); - udfContext.getOutput_fields().add("v2"); + udfContext.setLookupFields(new ArrayList<>()); + udfContext.getLookupFields().add("v1"); + udfContext.setOutputFields(new ArrayList<>()); + udfContext.getOutputFields().add("v2"); udfContext.setParameters(new HashMap<>()); udfContext.getParameters().put("option","other"); Assertions.assertThrows(GrootStreamRuntimeException.class, () -> { @@ -59,16 +59,16 @@ public class DomainFunctionTest { @Test public void testDomainFunctionTopLevelDomain() { parameters.put("option", "TOP_LEVEL_DOMAIN"); - udfContext.setLookup_fields(Collections.singletonList("domain")); - udfContext.setOutput_fields(Collections.singletonList("domain1")); + udfContext.setLookupFields(Collections.singletonList("domain")); + udfContext.setOutputFields(Collections.singletonList("domain1")); Domain domain = new Domain(); domain.open(null, udfContext); Event event = new Event(); Map<String, Object> extractedFields = new HashMap<>(); - extractedFields.put("domain", "www.baidu.com"); + extractedFields.put("domain", "http://www.baidu.com.cn"); event.setExtractedFields(extractedFields); Event result1 = domain.evaluate(event); - assertEquals("com", result1.getExtractedFields().get("domain1")); + assertEquals("com.cn", result1.getExtractedFields().get("domain1")); } @Test @@ -76,8 +76,8 @@ public class DomainFunctionTest { parameters.put("option", "FIRST_SIGNIFICANT_SUBDOMAIN"); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("domain")); - udfContext.setOutput_fields(Collections.singletonList("domain1")); + udfContext.setLookupFields(Collections.singletonList("domain")); + udfContext.setOutputFields(Collections.singletonList("domain1")); Domain domain = new Domain(); domain.open(null, udfContext); Event event = new Event(); @@ -87,4 +87,5 @@ public class DomainFunctionTest { Event result1 = domain.evaluate(event); assertEquals("baidu.com", result1.getExtractedFields().get("domain1")); } + } diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DropFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DropFunctionTest.java index 294a492..027533e 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DropFunctionTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/DropFunctionTest.java @@ -16,6 +16,7 @@ public class DropFunctionTest { @BeforeAll public static void setUp() { udfContext = new UDFContext(); + udfContext.setFilter("true"); udfContext.setParameters(new HashMap<>()); } diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/EncodeBase64FunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/EncodeBase64FunctionTest.java index 2bc96b6..2bd6705 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/EncodeBase64FunctionTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/EncodeBase64FunctionTest.java @@ -21,17 +21,15 @@ public class EncodeBase64FunctionTest { private static UDFContext udfContext; - @BeforeAll - public static void setUp() { - udfContext = new UDFContext(); - udfContext.setOutput_fields(Collections.singletonList("encodeResult")); - Map<String,Object> map = new HashMap<>(); - map.put("value_field","name"); - udfContext.setParameters(map); - } - @Test - public void testEncodeBase64Function() { + @Test + public void testEncodeBase64FunctionForByte() { + udfContext = new UDFContext(); + udfContext.setOutputFields(Collections.singletonList("encodeResult")); + udfContext.setLookupFields(Collections.singletonList("name")); + Map<String,Object> map = new HashMap<>(); + map.put("input_type","byte_array"); + udfContext.setParameters(map); EncodeBase64 encodeBase64 = new EncodeBase64(); encodeBase64.open(null, udfContext); Event event = new Event(); @@ -40,11 +38,28 @@ public class EncodeBase64FunctionTest { event.setExtractedFields(extractedFields); Event result1 = encodeBase64.evaluate(event); assertEquals("aGVsbG8=", result1.getExtractedFields().get("encodeResult")); + + + + } + + @Test + public void testEncodeBase64FunctionForString() { + + udfContext = new UDFContext(); + udfContext.setOutputFields(Collections.singletonList("encodeResult")); + udfContext.setLookupFields(Collections.singletonList("name")); + Map<String,Object> map = new HashMap<>(); + map.put("input_type","string"); + udfContext.setParameters(map); + EncodeBase64 encodeBase64 = new EncodeBase64(); + encodeBase64.open(null, udfContext); + Event event = new Event(); + Map<String, Object> extractedFields = new HashMap<>(); extractedFields.put("name", "hello"); event.setExtractedFields(extractedFields); - Event result2 = encodeBase64.evaluate(event); - assertEquals("", result2.getExtractedFields().get("encodeResult")); + Event result1 = encodeBase64.evaluate(event); + assertEquals("aGVsbG8=", result1.getExtractedFields().get("encodeResult")); } - } diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/EncryptFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/EncryptFunctionTest.java new file mode 100644 index 0000000..e9f1698 --- /dev/null +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/EncryptFunctionTest.java @@ -0,0 +1,245 @@ +package com.geedgenetworks.core.udf.test.simple; + +import cn.hutool.core.util.RandomUtil; +import com.alibaba.fastjson2.JSON; +import com.geedgenetworks.common.Constants; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.config.CommonConfig; +import com.geedgenetworks.common.config.KmsConfig; +import com.geedgenetworks.common.config.SSLConfig; +import com.geedgenetworks.common.exception.GrootStreamRuntimeException; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.core.pojo.KmsKey; +import com.geedgenetworks.core.udf.Encrypt; +import com.geedgenetworks.core.udf.encrypt.EncryptionAlgorithm; +import com.geedgenetworks.core.utils.EncryptionAlgorithmUtils; +import com.geedgenetworks.core.utils.HttpClientPoolUtil; +import com.geedgenetworks.core.utils.KmsUtils; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.SimpleCounter; +import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup; +import org.bouncycastle.jce.provider.BouncyCastleProvider; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.mockito.*; + +import java.io.IOException; +import java.security.Security; +import java.util.*; + +import static org.junit.jupiter.api.Assertions.*; + +public class EncryptFunctionTest { + private static UDFContext udfContext; + private static MockedStatic<HttpClientPoolUtil> httpClientPoolUtilMockedStatic; + private static final String DATA = "13812345678"; + + @BeforeAll + public static void setUp() throws IOException { + Security.addProvider(new BouncyCastleProvider()); + udfContext = new UDFContext(); + udfContext.setLookupFields(Collections.singletonList("phone_number")); + udfContext.setOutputFields(Collections.singletonList("phone_number")); + httpClientPoolUtilMockedStatic = mockSensitiveFields(); + } + + @AfterAll + public static void after() { + httpClientPoolUtilMockedStatic.close(); + } + + @Test + public void testEncryptByVault() throws Exception { + String secretKey = RandomUtil.randomString(32); + MockedStatic<KmsUtils> kmsUtilsMockedStatic = Mockito.mockStatic(KmsUtils.class); + Mockito.when(KmsUtils.getVaultKey(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any())).thenReturn(new KmsKey(secretKey.getBytes(), 1)); + RuntimeContext runtimeContext = mockVaultRuntimeContext(); + Map<String, Object> map = new HashMap<>(); + map.put("identifier", EncryptionAlgorithmUtils.ALGORITHM_AES_256_GCM96_NAME); + udfContext.setParameters(map); + Encrypt encrypt = new Encrypt(); + encrypt.open(runtimeContext, udfContext); + Event event = new Event(); + Map<String, Object> extractedFields = new HashMap<>(); + extractedFields.put("phone_number", DATA); + event.setExtractedFields(extractedFields); + Event result = encrypt.evaluate(event); + EncryptionAlgorithm encryptionAlgorithm = EncryptionAlgorithmUtils.createEncryptionAlgorithm(EncryptionAlgorithmUtils.ALGORITHM_AES_256_GCM96_NAME); + assertNotNull(encryptionAlgorithm); + encryptionAlgorithm.setKmsKey(new KmsKey(secretKey.getBytes(), 1)); + String encrypted = result.getExtractedFields().get("phone_number").toString(); + assertTrue(encrypted.contains("vault:v1:")); + String decrypted = encryptionAlgorithm.decrypt(encrypted.split(":")[2]); + assertEquals(DATA, decrypted); + encrypt.close(); + kmsUtilsMockedStatic.close(); + } + + @Test + public void testEncryptByLocal() throws Exception { + byte[] secretKey = ".........geedgenetworks.........".getBytes(); + RuntimeContext runtimeContext = mockLocalRuntimeContext(); + Map<String, Object> map = new HashMap<>(); + map.put("identifier", EncryptionAlgorithmUtils.ALGORITHM_AES_256_GCM96_NAME); + udfContext.setParameters(map); + Encrypt encrypt = new Encrypt(); + encrypt.open(runtimeContext, udfContext); + Event event = new Event(); + Map<String, Object> extractedFields = new HashMap<>(); + extractedFields.put("phone_number", DATA); + event.setExtractedFields(extractedFields); + Event result = encrypt.evaluate(event); + EncryptionAlgorithm encryptionAlgorithm = EncryptionAlgorithmUtils.createEncryptionAlgorithm(EncryptionAlgorithmUtils.ALGORITHM_AES_256_GCM96_NAME); + assertNotNull(encryptionAlgorithm); + encryptionAlgorithm.setKmsKey(new KmsKey(secretKey, 1)); + String decrypted = encryptionAlgorithm.decrypt((String) result.getExtractedFields().get("phone_number")); + assertEquals(DATA, decrypted); + encrypt.close(); + } + + @Test + public void testEncryptByIdentifier() { + Map<String, Object> map = new HashMap<>(); + map.put("identifier", EncryptionAlgorithmUtils.ALGORITHM_AES_256_GCM96_NAME); + udfContext.setParameters(map); + Encrypt encrypt1 = new Encrypt(); + assertDoesNotThrow(() -> encrypt1.open(mockLocalRuntimeContext(), udfContext)); + encrypt1.close(); + + Encrypt encrypt2 = new Encrypt(); + map.put("identifier", EncryptionAlgorithmUtils.ALGORITHM_AES_128_GCM96_NAME); + udfContext.setParameters(map); + assertDoesNotThrow(() -> encrypt2.open(mockLocalRuntimeContext(), udfContext)); + encrypt2.close(); + + Encrypt encrypt3 = new Encrypt(); + map.put("identifier", EncryptionAlgorithmUtils.ALGORITHM_SM4_GCM96_NAME); + udfContext.setParameters(map); + assertDoesNotThrow(() -> encrypt3.open(mockLocalRuntimeContext(), udfContext)); + encrypt3.close(); + } + + @Test + public void testEncryptionAlgorithm() throws Exception { + EncryptionAlgorithm encryptionAlgorithm = EncryptionAlgorithmUtils.createEncryptionAlgorithm(EncryptionAlgorithmUtils.ALGORITHM_AES_128_GCM96_NAME); + assertNotNull(encryptionAlgorithm); + encryptionAlgorithm.setKmsKey(new KmsKey("aaaaaaaaaaaaaaaa".getBytes(), 1)); + String encryptData = encryptionAlgorithm.encrypt(DATA); + String decryptData = encryptionAlgorithm.decrypt(encryptData); + assertEquals(DATA, decryptData); + + encryptionAlgorithm = EncryptionAlgorithmUtils.createEncryptionAlgorithm(EncryptionAlgorithmUtils.ALGORITHM_AES_256_GCM96_NAME); + assertNotNull(encryptionAlgorithm); + encryptionAlgorithm.setKmsKey(new KmsKey("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa".getBytes(), 1)); + encryptData = encryptionAlgorithm.encrypt(DATA); + decryptData = encryptionAlgorithm.decrypt(encryptData); + assertEquals(DATA, decryptData); + + encryptionAlgorithm = EncryptionAlgorithmUtils.createEncryptionAlgorithm(EncryptionAlgorithmUtils.ALGORITHM_SM4_GCM96_NAME); + assertNotNull(encryptionAlgorithm); + encryptionAlgorithm.setKmsKey(new KmsKey("aaaaaaaaaaaaaaaa".getBytes(), 1)); + encryptData = encryptionAlgorithm.encrypt(DATA); + decryptData = encryptionAlgorithm.decrypt(encryptData); + assertEquals(DATA, decryptData); + + encryptionAlgorithm = EncryptionAlgorithmUtils.createEncryptionAlgorithm("sm4"); + assertNull(encryptionAlgorithm); + } + + @Test + public void testEncryptError() { + RuntimeContext runtimeContext = mockLocalRuntimeContext(); + Encrypt encrypt = new Encrypt(); + udfContext.setParameters(null); + assertThrows(GrootStreamRuntimeException.class, () -> encrypt.open(runtimeContext, udfContext)); + + Map<String, Object> map = new HashMap<>(); + udfContext.setParameters(map); + assertThrows(GrootStreamRuntimeException.class, () -> encrypt.open(runtimeContext, udfContext)); + + map.put("identifier", "aes"); + udfContext.setParameters(map); + assertThrows(GrootStreamRuntimeException.class, () -> encrypt.open(runtimeContext, udfContext)); + } + + static RuntimeContext mockLocalRuntimeContext() { + RuntimeContext runtimeContext = Mockito.mock(RuntimeContext.class); + ExecutionConfig executionConfig = Mockito.mock(ExecutionConfig.class); + Mockito.when(runtimeContext.getExecutionConfig()).thenReturn(executionConfig); + MetricGroup metricGroup = Mockito.mock(OperatorMetricGroup.class); + Mockito.when(runtimeContext.getMetricGroup()).thenReturn(metricGroup); + Mockito.when(metricGroup.addGroup(Mockito.anyString())).thenReturn(metricGroup); + Mockito.when(metricGroup.counter(Mockito.anyString())).thenReturn(new SimpleCounter()); + Configuration configuration = new Configuration(); + CommonConfig commonConfig = new CommonConfig(); + Map<String, KmsConfig> kmsConfigs = new HashMap<>(); + KmsConfig kmsConfig = new KmsConfig(); + kmsConfig.setType(KmsUtils.KMS_TYPE_LOCAL); + kmsConfigs.put(KmsUtils.KMS_TYPE_LOCAL, kmsConfig); + kmsConfig = new KmsConfig(); + kmsConfig.setType(KmsUtils.KMS_TYPE_VAULT); + kmsConfigs.put(KmsUtils.KMS_TYPE_VAULT, kmsConfig); + SSLConfig sslConfig = new SSLConfig(); + sslConfig.setSkipVerification(true); + Map<String, String> propertiesConfig = new HashMap<>(); + propertiesConfig.put("projection.encrypt.schema.registry.uri", "127.0.0.1:9999/v1/schema/session_record?option=encrypt_fields"); + commonConfig.setKmsConfig(kmsConfigs); + commonConfig.setSslConfig(sslConfig); + commonConfig.setPropertiesConfig(propertiesConfig); + configuration.setString(Constants.SYSPROP_GROOTSTREAM_CONFIG, JSON.toJSONString(commonConfig)); + configuration.setString(Constants.SYSPROP_KMS_TYPE_CONFIG, KmsUtils.KMS_TYPE_LOCAL); + Mockito.when(executionConfig.getGlobalJobParameters()).thenReturn(configuration); + return runtimeContext; + } + + static RuntimeContext mockVaultRuntimeContext() { + RuntimeContext runtimeContext = Mockito.mock(RuntimeContext.class); + ExecutionConfig executionConfig = Mockito.mock(ExecutionConfig.class); + Mockito.when(runtimeContext.getExecutionConfig()).thenReturn(executionConfig); + MetricGroup metricGroup = Mockito.mock(OperatorMetricGroup.class); + Mockito.when(runtimeContext.getMetricGroup()).thenReturn(metricGroup); + Mockito.when(metricGroup.addGroup(Mockito.anyString())).thenReturn(metricGroup); + Mockito.when(metricGroup.counter(Mockito.anyString())).thenReturn(new SimpleCounter()); + Configuration configuration = new Configuration(); + CommonConfig commonConfig = new CommonConfig(); + Map<String, KmsConfig> kmsConfigs = new HashMap<>(); + KmsConfig kmsConfig = new KmsConfig(); + kmsConfig.setType(KmsUtils.KMS_TYPE_VAULT); + kmsConfigs.put(KmsUtils.KMS_TYPE_VAULT, kmsConfig); + kmsConfig = new KmsConfig(); + kmsConfig.setType(KmsUtils.KMS_TYPE_LOCAL); + kmsConfigs.put(KmsUtils.KMS_TYPE_LOCAL, kmsConfig); + SSLConfig sslConfig = new SSLConfig(); + sslConfig.setSkipVerification(true); + Map<String, String> propertiesConfig = new HashMap<>(); + propertiesConfig.put("projection.encrypt.schema.registry.uri", "127.0.0.1:9999/v1/schema/session_record?option=encrypt_fields"); + commonConfig.setKmsConfig(kmsConfigs); + commonConfig.setSslConfig(sslConfig); + commonConfig.setPropertiesConfig(propertiesConfig); + configuration.setString(Constants.SYSPROP_GROOTSTREAM_CONFIG, JSON.toJSONString(commonConfig)); + configuration.setString(Constants.SYSPROP_KMS_TYPE_CONFIG, KmsUtils.KMS_TYPE_VAULT); + Mockito.when(executionConfig.getGlobalJobParameters()).thenReturn(configuration); + return runtimeContext; + } + + static MockedStatic<HttpClientPoolUtil> mockSensitiveFields() throws IOException { + String sensitiveFieldsStr = "{\n" + + " \"status\": 200,\n" + + " \"success\": true,\n" + + " \"message\": \"Success\",\n" + + " \"data\": [\n" + + " \"phone_number\",\n" + + " \"server_ip\"\n" + + " ]\n" + + "}"; + HttpClientPoolUtil instance = Mockito.mock(HttpClientPoolUtil.class); + Mockito.when(instance.httpGet(ArgumentMatchers.any(), ArgumentMatchers.any())).thenReturn(sensitiveFieldsStr); + MockedStatic<HttpClientPoolUtil> httpClientPoolUtilMockedStatic = Mockito.mockStatic(HttpClientPoolUtil.class); + Mockito.when(HttpClientPoolUtil.getInstance()).thenReturn(instance); + return httpClientPoolUtilMockedStatic; + } +} diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/FlattenFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/FlattenFunctionTest.java index 61c3975..e829b1d 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/FlattenFunctionTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/FlattenFunctionTest.java @@ -25,7 +25,7 @@ public class FlattenFunctionTest { UDFContext udfContext = new UDFContext(); - udfContext.setLookup_fields(List.of("parent1", "parent2", "parent3", "parent4","parent5","parent6","parent7","parent8")); + udfContext.setLookupFields(List.of("parent1", "parent2", "parent3", "parent4","parent5","parent6","parent7","parent8")); Map<String, Object> params = new HashMap<>(); params.put("prefix", "prefix"); params.put("depth", "4"); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/FromUnixTimestampTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/FromUnixTimestampTest.java index d02303b..6cb1bf8 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/FromUnixTimestampTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/FromUnixTimestampTest.java @@ -19,15 +19,15 @@ public class FromUnixTimestampTest { @BeforeAll public static void setUp() { udfContext = new UDFContext(); - udfContext.setLookup_fields(Arrays.asList("unixTimestamp")); - udfContext.setOutput_fields(Arrays.asList("timestamp")); + udfContext.setLookupFields(Arrays.asList("unixTimestamp")); + udfContext.setOutputFields(Arrays.asList("timestamp")); } @Test - public void testFromUnixTimestampFunction() throws Exception { + public void testFromUnixTimestampMsFunction() throws Exception { Map<String, Object> parameters = new HashMap<>(); parameters.put("precision", "seconds"); - parameters.put("timezone", "UTC"); + parameters.put("timezone", "Asia/Shanghai"); udfContext.setParameters(parameters); FromUnixTimestamp fromUnixTimestamp = new FromUnixTimestamp(); fromUnixTimestamp.open(null, udfContext); @@ -36,11 +36,32 @@ public class FromUnixTimestampTest { extractedFields.put("unixTimestamp", 1577808000000L); event.setExtractedFields(extractedFields); Event result1 = fromUnixTimestamp.evaluate(event); + assertEquals("2020-01-01 00:00:00", result1.getExtractedFields().get("timestamp")); + parameters.put("precision", "milliseconds"); + fromUnixTimestamp.open(null, udfContext); + Event result2 = fromUnixTimestamp.evaluate(event); + assertEquals("2020-01-01 00:00:00.000", result2.getExtractedFields().get("timestamp")); + + } + + @Test + public void testFromUnixTimestampFunction() throws Exception { + + Map<String, Object> parameters = new HashMap<>(); + parameters.put("precision", "seconds"); + udfContext.setParameters(parameters); + FromUnixTimestamp fromUnixTimestamp = new FromUnixTimestamp(); + fromUnixTimestamp.open(null, udfContext); + Event event = new Event(); + Map<String, Object> extractedFields = new HashMap<>(); + extractedFields.put("unixTimestamp", 1577808000L); + event.setExtractedFields(extractedFields); + Event result1 = fromUnixTimestamp.evaluate(event); assertEquals("2019-12-31 16:00:00", result1.getExtractedFields().get("timestamp")); parameters.put("precision", "milliseconds"); fromUnixTimestamp.open(null, udfContext); Event result2 = fromUnixTimestamp.evaluate(event); - assertEquals("2019-12-31 16:00:00:000", result2.getExtractedFields().get("timestamp")); + assertEquals("2019-12-31 16:00:00.000", result2.getExtractedFields().get("timestamp")); } } diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/GenerateStringArrayFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/GenerateStringArrayFunctionTest.java index 5490299..1fbe06c 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/GenerateStringArrayFunctionTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/GenerateStringArrayFunctionTest.java @@ -17,8 +17,8 @@ public class GenerateStringArrayFunctionTest { @BeforeAll public static void setUp() { udfContext = new UDFContext(); - udfContext.setLookup_fields(Arrays.asList("t1", "t2","t3","t4","t5")); - udfContext.setOutput_fields(Collections.singletonList("result_list")); + udfContext.setLookupFields(Arrays.asList("t1", "t2","t3","t4","t5")); + udfContext.setOutputFields(Collections.singletonList("result_list")); } // 测试方法 diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/HmacFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/HmacFunctionTest.java new file mode 100644 index 0000000..5a4d0d3 --- /dev/null +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/HmacFunctionTest.java @@ -0,0 +1,136 @@ +package com.geedgenetworks.core.udf.test.simple; + +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.exception.GrootStreamRuntimeException; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.core.udf.Hmac; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class HmacFunctionTest { + + private static final String SECRET_KEY = ".geedgenetworks."; + private static final String DATA = "13812345678"; + private static UDFContext udfContext; + + @BeforeAll + public static void setUp() { + udfContext = new UDFContext(); + udfContext.setLookupFields(Collections.singletonList("phone_number")); + udfContext.setOutputFields(Collections.singletonList("phone_number_mac")); + } + + @Test + public void testHmacAsBase64() { + Map<String, Object> map = new HashMap<>(); + map.put("secret_key", SECRET_KEY); + map.put("algorithm", "sha256"); + map.put("output_format", "base64"); + udfContext.setParameters(map); + Hmac hmac = new Hmac(); + hmac.open(null, udfContext); + Event event = new Event(); + Map<String, Object> extractedFields = new HashMap<>(); + extractedFields.put("phone_number", DATA); + event.setExtractedFields(extractedFields); + Event result1 = hmac.evaluate(event); + assertEquals("zaj6UKovIsDahIBeRZ2PmgPIfDEr900F2xWu+iQfFrw=", result1.getExtractedFields().get("phone_number_mac")); + } + + @Test + public void testHmacAsHex() { + Map<String, Object> map = new HashMap<>(); + map.put("secret_key", SECRET_KEY); + map.put("algorithm", "sha256"); + map.put("output_format", "hex"); + udfContext.setParameters(map); + Hmac hmac = new Hmac(); + hmac.open(null, udfContext); + Event event = new Event(); + Map<String, Object> extractedFields = new HashMap<>(); + extractedFields.put("phone_number", DATA); + event.setExtractedFields(extractedFields); + Event result1 = hmac.evaluate(event); + assertEquals("cda8fa50aa2f22c0da84805e459d8f9a03c87c312bf74d05db15aefa241f16bc", result1.getExtractedFields().get("phone_number_mac")); + } + + @Test + public void testHmacAlgorithm() { + Map<String, Object> map = new HashMap<>(); + map.put("secret_key", SECRET_KEY); + map.put("algorithm", "sm4"); + map.put("output_format", "base64"); + udfContext.setParameters(map); + Hmac hmac = new Hmac(); + hmac.open(null, udfContext); + Event event = new Event(); + Map<String, Object> extractedFields = new HashMap<>(); + extractedFields.put("phone_number", DATA); + event.setExtractedFields(extractedFields); + Event result = hmac.evaluate(event); + assertEquals("QX1q4Y7y3quYCDje9BuSjg==", result.getExtractedFields().get("phone_number_mac")); + + map = new HashMap<>(); + map.put("secret_key", SECRET_KEY); + map.put("algorithm", "sha1"); + map.put("output_format", "base64"); + udfContext.setParameters(map); + hmac = new Hmac(); + hmac.open(null, udfContext); + event.setExtractedFields(extractedFields); + result = hmac.evaluate(event); + assertEquals("NB1b1TsVZ95/0sE+d/6kdtyUFh0=", result.getExtractedFields().get("phone_number_mac")); + + map = new HashMap<>(); + map.put("secret_key", SECRET_KEY); + map.put("algorithm", "sm3"); + map.put("output_format", "base64"); + udfContext.setParameters(map); + hmac = new Hmac(); + hmac.open(null, udfContext); + event.setExtractedFields(extractedFields); + result = hmac.evaluate(event); + assertEquals("BbQNpwLWE3rkaI1WlPBJgYeD14UyL2OwTxiEoTNA3UU=", result.getExtractedFields().get("phone_number_mac")); + + map = new HashMap<>(); + map.put("secret_key", SECRET_KEY); + map.put("algorithm", "md5"); + map.put("output_format", "base64"); + udfContext.setParameters(map); + hmac = new Hmac(); + hmac.open(null, udfContext); + event.setExtractedFields(extractedFields); + result = hmac.evaluate(event); + assertEquals("BQZzRqD3ZR/nJsDIOO4dBg==", result.getExtractedFields().get("phone_number_mac")); + + map = new HashMap<>(); + map.put("secret_key", SECRET_KEY); + map.put("algorithm", "sha512"); + map.put("output_format", "base64"); + udfContext.setParameters(map); + hmac = new Hmac(); + hmac.open(null, udfContext); + event.setExtractedFields(extractedFields); + result = hmac.evaluate(event); + assertEquals("DWrndzlcqf2qvFTbuDC1gZCGmRhuAUayfsxEqr2ZlpY/QOr9HgGUZNOfytRfA4VT8OZK0BwHwcAg5pgGBvPQ4A==", result.getExtractedFields().get("phone_number_mac")); + } + + @Test + public void testHmacError() { + Map<String, Object> map = new HashMap<>(); + map.put("secret_key", SECRET_KEY); + map.put("algorithm", "sha256"); + map.put("output_format", "hex"); + udfContext.setParameters(map); + Hmac hmac = new Hmac(); + udfContext.getParameters().remove("secret_key"); + assertThrows(GrootStreamRuntimeException.class, () -> hmac.open(null, udfContext)); + } +} diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/JsonExtractFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/JsonExtractFunctionTest.java index 99f3f96..dd661de 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/JsonExtractFunctionTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/JsonExtractFunctionTest.java @@ -25,7 +25,7 @@ public class JsonExtractFunctionTest { @Test public void testInit(){ JsonExtract jsonExtract = new JsonExtract(); - udfContext.setLookup_fields(new ArrayList<>()); + udfContext.setLookupFields(new ArrayList<>()); udfContext.setParameters(new HashMap<>()); udfContext.setParameters(null); Assertions.assertThrows(GrootStreamRuntimeException.class, () -> { @@ -52,8 +52,8 @@ public class JsonExtractFunctionTest { Map<String, Object> parameters = new HashMap<>(); parameters.put("value_expression","$.tags[?(@.tag=='device_group')][0].value"); udfContext.setParameters(parameters); - udfContext.setLookup_fields(Collections.singletonList("device_tag")); - udfContext.setOutput_fields(Collections.singletonList("device_group")); + udfContext.setLookupFields(Collections.singletonList("device_tag")); + udfContext.setOutputFields(Collections.singletonList("device_group")); jsonExtract.open(null, udfContext); Event event = new Event(); String jsonString = "{\"device_tag\":\"{\\\"tags\\\":[{\\\"tag\\\":\\\"data_center\\\",\\\"value\\\":\\\"center-xxg-tsgx\\\"},{\\\"tag\\\":\\\"device_group\\\",\\\"value\\\":\\\"group-xxg-tsgx\\\"}]}\"}"; diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/StringJoinerFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/StringJoinerFunctionTest.java index d80eb97..e9cde8a 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/StringJoinerFunctionTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/StringJoinerFunctionTest.java @@ -20,8 +20,8 @@ public class StringJoinerFunctionTest { @BeforeAll public static void setUp() { udfContext = new UDFContext(); - udfContext.setLookup_fields(Arrays.asList("server_ip", "client_ip")); - udfContext.setOutput_fields(Collections.singletonList("ip_string")); + udfContext.setLookupFields(Arrays.asList("server_ip", "client_ip")); + udfContext.setOutputFields(Collections.singletonList("ip_string")); Map<String, Object> params = new HashMap<>(); params.put("separator",","); params.put("prefix","["); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/UUIDTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/UUIDTest.java new file mode 100644 index 0000000..534569b --- /dev/null +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/UUIDTest.java @@ -0,0 +1,139 @@ +package com.geedgenetworks.core.udf.test.simple; + +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.exception.GrootStreamRuntimeException; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.core.udf.uuid.UUID; +import com.geedgenetworks.core.udf.uuid.UUIDv5; +import com.geedgenetworks.core.udf.uuid.UUIDv7; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.*; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; + +public class UUIDTest { + + private UDFContext udfContext; + private Map<String, Object> parameters ; + + + @Test + public void testInit(){ + udfContext = new UDFContext(); + UUIDv5 uuidv5 = new UUIDv5(); + parameters = new HashMap<>(); + udfContext.setParameters(parameters); + udfContext.setLookupFields(List.of("client_ip","server_ip")); + udfContext.setOutputFields(Collections.singletonList("uuid")); + parameters.put("namespace","NAMESPACE_IP_1"); + Assertions.assertThrows(GrootStreamRuntimeException.class, () -> { + uuidv5.open(null, udfContext); + }); + + } + + @Test + public void testUUID() { + udfContext = new UDFContext(); + UUID uuid = new UUID(); + parameters = new HashMap<>(); + udfContext.setParameters(parameters); + udfContext.setOutputFields(Collections.singletonList("uuid")); + uuid.open(null, udfContext); + Event event = new Event(); + Map<String, Object> extractedFields = new HashMap<>(); + event.setExtractedFields(extractedFields); + Event result1 = uuid.evaluate(event); + assertEquals(36, result1.getExtractedFields().get("uuid").toString().length()); + } + @Test + public void testUUIDV7() { + udfContext = new UDFContext(); + UUIDv7 uuid = new UUIDv7(); + parameters = new HashMap<>(); + udfContext.setParameters(parameters); + udfContext.setOutputFields(Collections.singletonList("uuid")); + uuid.open(null, udfContext); + Event event = new Event(); + Map<String, Object> extractedFields = new HashMap<>(); + event.setExtractedFields(extractedFields); + Event result1 = uuid.evaluate(event); + assertEquals(36, result1.getExtractedFields().get("uuid").toString().length()); + } + @Test + public void testUUIDV5ForNameSpaceIp() { + udfContext = new UDFContext(); + UUIDv5 uuidv5 = new UUIDv5(); + parameters = new HashMap<>(); + udfContext.setParameters(parameters); + udfContext.setLookupFields(List.of("client_ip", "server_ip")); + udfContext.setOutputFields(Collections.singletonList("uuid")); + parameters.put("namespace","NAMESPACE_IP"); + uuidv5.open(null, udfContext); + Event event = new Event(); + Map<String, Object> extractedFields = new HashMap<>(); + extractedFields.put("client_ip", "1.1.1.1"); + extractedFields.put("server_ip", ""); + event.setExtractedFields(extractedFields); + Event result = uuidv5.evaluate(event); + System.out.printf("uuid: %s\n", result.getExtractedFields().get("uuid").toString()); + assertEquals("5394a6a8-b9b8-5147-b5b2-01365f158acb", result.getExtractedFields().get("uuid").toString()); + assertNotEquals("ecc67867-1f76-580c-a4c1-6a3d16ad6d02", result.getExtractedFields().get("uuid").toString()); + } + + @Test + public void testUUIDV5ForNameSpaceDomain() { + udfContext = new UDFContext(); + UUIDv5 uuidv5 = new UUIDv5(); + parameters = new HashMap<>(); + udfContext.setParameters(parameters); + udfContext.setLookupFields(List.of("domain")); + udfContext.setOutputFields(Collections.singletonList("uuid")); + parameters.put("namespace","NAMESPACE_DOMAIN"); + uuidv5.open(null, udfContext); + Event event = new Event(); + Map<String, Object> extractedFields = new HashMap<>(); + extractedFields.put("domain", "baidu"); + event.setExtractedFields(extractedFields); + Event result1 = uuidv5.evaluate(event); + assertEquals("fd67cec1-6b33-5def-835c-fbe32f1ce4a4", result1.getExtractedFields().get("uuid").toString()); + } + @Test + public void testUUIDv5ForNameSpaceApp() { + udfContext = new UDFContext(); + UUIDv5 uuidv5 = new UUIDv5(); + parameters = new HashMap<>(); + udfContext.setParameters(parameters); + udfContext.setLookupFields(List.of("app")); + udfContext.setOutputFields(Collections.singletonList("uuid")); + parameters.put("namespace","NAMESPACE_APP"); + uuidv5.open(null, udfContext); + Event event = new Event(); + Map<String, Object> extractedFields = new HashMap<>(); + extractedFields.put("app", "54321"); + event.setExtractedFields(extractedFields); + Event result1 = uuidv5.evaluate(event); + assertEquals("194b8304-64f0-599e-bf7a-66cbc21bf6ee", result1.getExtractedFields().get("uuid").toString()); + } + + @Test + public void testUUIDV5ForNameSpaceSubscriberID() { + udfContext = new UDFContext(); + UUIDv5 uuidv5 = new UUIDv5(); + parameters = new HashMap<>(); + udfContext.setParameters(parameters); + udfContext.setLookupFields(List.of("subscriber_id")); + udfContext.setOutputFields(Collections.singletonList("uuid")); + parameters.put("namespace","NAMESPACE_SUBSCRIBER"); + uuidv5.open(null, udfContext); + Event event = new Event(); + Map<String, Object> extractedFields = new HashMap<>(); + extractedFields.put("subscriber_id", "test1"); + event.setExtractedFields(extractedFields); + Event result1 = uuidv5.evaluate(event); + assertEquals("9b154520-3c29-541c-bb81-f649354dae67", result1.getExtractedFields().get("uuid").toString()); + } +} diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/UnixTimestampConverterTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/UnixTimestampConverterTest.java index 12a2093..a0d70d7 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/UnixTimestampConverterTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/simple/UnixTimestampConverterTest.java @@ -19,8 +19,8 @@ public class UnixTimestampConverterTest { @BeforeAll public static void setUp() { udfContext = new UDFContext(); - udfContext.setLookup_fields(Arrays.asList("input")); - udfContext.setOutput_fields(Arrays.asList("output")); + udfContext.setLookupFields(Arrays.asList("input")); + udfContext.setOutputFields(Arrays.asList("output")); } @Test diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/JsonUnrollFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/JsonUnrollFunctionTest.java index 02f0b66..3749eb1 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/JsonUnrollFunctionTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/JsonUnrollFunctionTest.java @@ -52,8 +52,8 @@ public class JsonUnrollFunctionTest { JsonUnroll unroll = new JsonUnroll(); Event event = new Event(); event.setExtractedFields(nestedMap); - udfContext.setLookup_fields(List.of("k1")); - udfContext.setOutput_fields(List.of("newk1")); + udfContext.setLookupFields(List.of("k1")); + udfContext.setOutputFields(List.of("newk1")); unroll.open(null, udfContext); List<Event> result3 = unroll.evaluate(event); assertEquals(2, result3.size()); @@ -75,8 +75,8 @@ public class JsonUnrollFunctionTest { udfContext.setParameters(params); params.put("path", "$.k3_1.k3_1_1"); event.setExtractedFields(nestedMap); - udfContext.setLookup_fields(List.of("k3")); - udfContext.setOutput_fields(List.of("newk3")); + udfContext.setLookupFields(List.of("k3")); + udfContext.setOutputFields(List.of("newk3")); unroll.open(null, udfContext); List<Event> result2 = unroll.evaluate(event); assertEquals(2, result2.size()); @@ -95,8 +95,8 @@ public class JsonUnrollFunctionTest { udfContext.setParameters(params); params.put("path", "$.k4_1.k4_1_1"); event.setExtractedFields(nestedMap); - udfContext.setLookup_fields(List.of("k4")); - udfContext.setOutput_fields(List.of("newk4")); + udfContext.setLookupFields(List.of("k4")); + udfContext.setOutputFields(List.of("newk4")); unroll.open(null, udfContext); List<Event> result2 = unroll.evaluate(event); assertEquals(1, result2.size()); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnrollFunctionTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnrollFunctionTest.java index 2f4da76..db66e55 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnrollFunctionTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/test/table/UnrollFunctionTest.java @@ -34,8 +34,8 @@ public class UnrollFunctionTest { UDFContext udfContext = new UDFContext(); - udfContext.setLookup_fields(List.of("k1")); - udfContext.setOutput_fields(List.of("newk1")); + udfContext.setLookupFields(List.of("k1")); + udfContext.setOutputFields(List.of("newk1")); Unroll unroll = new Unroll(); unroll.open(null, udfContext); Event event = new Event(); @@ -60,8 +60,8 @@ public class UnrollFunctionTest { udfContext.setParameters(params); udfContext.setParameters(params); event.setExtractedFields(nestedMap); - udfContext.setLookup_fields(List.of("k2")); - udfContext.setOutput_fields(List.of("k2")); + udfContext.setLookupFields(List.of("k2")); + udfContext.setOutputFields(List.of("k2")); unroll.open(null, udfContext); List<Event> result2 = unroll.evaluate(event); assertEquals(3, result2.size()); @@ -76,8 +76,8 @@ public class UnrollFunctionTest { Unroll unroll = new Unroll(); Event event = new Event(); event.setExtractedFields(nestedMap); - udfContext.setLookup_fields(List.of("k3")); - udfContext.setOutput_fields(List.of("newk3")); + udfContext.setLookupFields(List.of("k3")); + udfContext.setOutputFields(List.of("newk3")); unroll.open(null, udfContext); List<Event> result2 = unroll.evaluate(event); assertEquals(1, result2.size()); diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantileTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantileTest.java index 33f7bad..990186d 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantileTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantileTest.java @@ -1,89 +1,89 @@ -package com.geedgenetworks.core.udf.udaf.HdrHistogram;
-
-import com.geedgenetworks.common.Accumulator;
-import com.geedgenetworks.common.Event;
-import com.geedgenetworks.common.udf.AggregateFunction;
-import com.geedgenetworks.common.udf.UDFContext;
-import com.geedgenetworks.sketch.util.StringUtils;
-import org.HdrHistogram.ArrayHistogram;
-import org.junit.jupiter.api.Test;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.junit.jupiter.api.Assertions.*;
-
-public class HdrHistogramQuantileTest {
- AggregateFunction agg;
- Accumulator acc;
- Event event;
-
- @Test
- public void inputRegular() {
- double probability = 0.5;
- initData( "regular", 2, probability);
- long count = 100000;
- Map<String, Object> fields = event.getExtractedFields();
- for (int i = 1; i <= count; i++) {
- fields.put("ms", i);
- agg.add(event, acc);
- }
-
- long expect = (long) (count * probability);
- long rst = (long)agg.getResult(acc).getMetricsFields().get("ms_his");
- double error = Math.abs(rst - expect) / (double) expect;
- System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error));
- assertTrue(error <= 0.05);
- }
-
- @Test
- public void inputSketch() {
- double probability = 0.5;
- initData( "sketch", 2, probability);
- long count = 100000;
- Map<String, Object> fields = event.getExtractedFields();
-
- ArrayHistogram his = new ArrayHistogram(2);
- for (int i = 1; i <= count; i++) {
- his.recordValue(i);
- }
- fields.put("ms", StringUtils.encodeBase64String(his.toBytes()));
- agg.add(event, acc);
-
- his = new ArrayHistogram(2);
- for (int i = 1; i <= count; i++) {
- his.recordValue(i);
- }
- fields.put("ms", his.toBytes());
- agg.add(event, acc);
-
- long expect = (long) (count * probability);
- long rst = (long)agg.getResult(acc).getMetricsFields().get("ms_his");
- double error = Math.abs(rst - expect) / (double) expect;
- System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error));
- assertTrue(error <= 0.05);
- }
-
- private void initData(String input_type, int numberOfSignificantValueDigits, double probability){
- agg = new HdrHistogramQuantile();
- UDFContext c = new UDFContext();
- Map<String, Object> parameters = new HashMap<>();
- parameters.put("input_type", input_type);
- parameters.put("numberOfSignificantValueDigits", numberOfSignificantValueDigits);
- parameters.put("probability", probability);
- c.setParameters(parameters);
- c.setLookup_fields(Collections.singletonList("ms"));
- c.setOutput_fields(Collections.singletonList("ms_his"));
-
- agg.open(c);
- Map<String, Object> map = new HashMap<>();
- acc = new Accumulator();
- acc.setMetricsFields(map);
- agg.initAccumulator(acc);
-
- event = new Event();
- Map<String, Object> fields = new HashMap<>();
- event.setExtractedFields(fields);
- }
+package com.geedgenetworks.core.udf.udaf.HdrHistogram; + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.AggregateFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.sketch.util.StringUtils; +import org.HdrHistogram.ArrayHistogram; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.*; + +public class HdrHistogramQuantileTest { + AggregateFunction agg; + Accumulator acc; + Event event; + + @Test + public void inputRegular() { + double probability = 0.5; + initData( "regular", 2, probability); + long count = 100000; + Map<String, Object> fields = event.getExtractedFields(); + for (int i = 1; i <= count; i++) { + fields.put("ms", i); + agg.add(event, acc); + } + + long expect = (long) (count * probability); + long rst = (long)agg.getResult(acc).getMetricsFields().get("ms_his"); + double error = Math.abs(rst - expect) / (double) expect; + System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error)); + assertTrue(error <= 0.05); + } + + @Test + public void inputSketch() { + double probability = 0.5; + initData( "sketch", 2, probability); + long count = 100000; + Map<String, Object> fields = event.getExtractedFields(); + + ArrayHistogram his = new ArrayHistogram(2); + for (int i = 1; i <= count; i++) { + his.recordValue(i); + } + fields.put("ms", StringUtils.encodeBase64String(his.toBytes())); + agg.add(event, acc); + + his = new ArrayHistogram(2); + for (int i = 1; i <= count; i++) { + his.recordValue(i); + } + fields.put("ms", his.toBytes()); + agg.add(event, acc); + + long expect = (long) (count * probability); + long rst = (long)agg.getResult(acc).getMetricsFields().get("ms_his"); + double error = Math.abs(rst - expect) / (double) expect; + System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error)); + assertTrue(error <= 0.05); + } + + private void initData(String input_type, int numberOfSignificantValueDigits, double probability){ + agg = new HdrHistogramQuantile(); + UDFContext c = new UDFContext(); + Map<String, Object> parameters = new HashMap<>(); + parameters.put("input_type", input_type); + parameters.put("numberOfSignificantValueDigits", numberOfSignificantValueDigits); + parameters.put("probability", probability); + c.setParameters(parameters); + c.setLookupFields(Collections.singletonList("ms")); + c.setOutputFields(Collections.singletonList("ms_his")); + + agg.open(c); + Map<String, Object> map = new HashMap<>(); + acc = new Accumulator(); + acc.setMetricsFields(map); + agg.initAccumulator(acc); + + event = new Event(); + Map<String, Object> fields = new HashMap<>(); + event.setExtractedFields(fields); + } }
\ No newline at end of file diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantilesTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantilesTest.java index 4eefd9a..a57645d 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantilesTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramQuantilesTest.java @@ -1,98 +1,98 @@ -package com.geedgenetworks.core.udf.udaf.HdrHistogram;
-
-import com.geedgenetworks.common.Accumulator;
-import com.geedgenetworks.common.Event;
-import com.geedgenetworks.common.udf.AggregateFunction;
-import com.geedgenetworks.common.udf.UDFContext;
-import com.geedgenetworks.sketch.util.StringUtils;
-import org.HdrHistogram.ArrayHistogram;
-import org.junit.jupiter.api.Test;
-
-import java.util.*;
-
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-public class HdrHistogramQuantilesTest {
- AggregateFunction agg;
- Accumulator acc;
- Event event;
-
- @Test
- public void inputRegular() {
- double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1};
- initData( "regular", 2, probabilities);
- long count = 100000;
- Map<String, Object> fields = event.getExtractedFields();
- for (int i = 1; i <= count; i++) {
- fields.put("ms", i);
- agg.add(event, acc);
- }
-
- long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray();
-
- List<Long> rsts = (List<Long>)agg.getResult(acc).getMetricsFields().get("ms_his");
- for (int i = 0; i < expects.length; i++) {
- long rst = rsts.get(i);
- long expect = expects[i];
- double probability = probabilities[i];
- double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect;
- System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error));
- assertTrue(error <= 0.05);
- }
- }
-
- @Test
- public void inputSketch() {
- double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1};
- initData( "sketch", 2, probabilities);
- long count = 100000;
- Map<String, Object> fields = event.getExtractedFields();
- long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray();
-
- ArrayHistogram his = new ArrayHistogram(2);
- for (int i = 1; i <= count; i++) {
- his.recordValue(i);
- }
- fields.put("ms", StringUtils.encodeBase64String(his.toBytes()));
- agg.add(event, acc);
-
- his = new ArrayHistogram(2);
- for (int i = 1; i <= count; i++) {
- his.recordValue(i);
- }
- fields.put("ms", his.toBytes());
- agg.add(event, acc);
-
- List<Long> rsts = (List<Long>)agg.getResult(acc).getMetricsFields().get("ms_his");
- for (int i = 0; i < expects.length; i++) {
- long rst = rsts.get(i);
- long expect = expects[i];
- double probability = probabilities[i];
- double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect;
- System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error));
- assertTrue(error <= 0.05);
- }
- }
-
- private void initData(String input_type, int numberOfSignificantValueDigits, double[] probabilities){
- agg = new HdrHistogramQuantiles();
- UDFContext c = new UDFContext();
- Map<String, Object> parameters = new HashMap<>();
- parameters.put("input_type", input_type);
- parameters.put("numberOfSignificantValueDigits", numberOfSignificantValueDigits);
- parameters.put("probabilities", probabilities);
- c.setParameters(parameters);
- c.setLookup_fields(Collections.singletonList("ms"));
- c.setOutput_fields(Collections.singletonList("ms_his"));
-
- agg.open(c);
- Map<String, Object> map = new HashMap<>();
- acc = new Accumulator();
- acc.setMetricsFields(map);
- agg.initAccumulator(acc);
-
- event = new Event();
- Map<String, Object> fields = new HashMap<>();
- event.setExtractedFields(fields);
- }
+package com.geedgenetworks.core.udf.udaf.HdrHistogram; + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.AggregateFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.sketch.util.StringUtils; +import org.HdrHistogram.ArrayHistogram; +import org.junit.jupiter.api.Test; + +import java.util.*; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class HdrHistogramQuantilesTest { + AggregateFunction agg; + Accumulator acc; + Event event; + + @Test + public void inputRegular() { + double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1}; + initData( "regular", 2, probabilities); + long count = 100000; + Map<String, Object> fields = event.getExtractedFields(); + for (int i = 1; i <= count; i++) { + fields.put("ms", i); + agg.add(event, acc); + } + + long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray(); + + List<Long> rsts = (List<Long>)agg.getResult(acc).getMetricsFields().get("ms_his"); + for (int i = 0; i < expects.length; i++) { + long rst = rsts.get(i); + long expect = expects[i]; + double probability = probabilities[i]; + double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect; + System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error)); + assertTrue(error <= 0.05); + } + } + + @Test + public void inputSketch() { + double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1}; + initData( "sketch", 2, probabilities); + long count = 100000; + Map<String, Object> fields = event.getExtractedFields(); + long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray(); + + ArrayHistogram his = new ArrayHistogram(2); + for (int i = 1; i <= count; i++) { + his.recordValue(i); + } + fields.put("ms", StringUtils.encodeBase64String(his.toBytes())); + agg.add(event, acc); + + his = new ArrayHistogram(2); + for (int i = 1; i <= count; i++) { + his.recordValue(i); + } + fields.put("ms", his.toBytes()); + agg.add(event, acc); + + List<Long> rsts = (List<Long>)agg.getResult(acc).getMetricsFields().get("ms_his"); + for (int i = 0; i < expects.length; i++) { + long rst = rsts.get(i); + long expect = expects[i]; + double probability = probabilities[i]; + double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect; + System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error)); + assertTrue(error <= 0.05); + } + } + + private void initData(String input_type, int numberOfSignificantValueDigits, double[] probabilities){ + agg = new HdrHistogramQuantiles(); + UDFContext c = new UDFContext(); + Map<String, Object> parameters = new HashMap<>(); + parameters.put("input_type", input_type); + parameters.put("numberOfSignificantValueDigits", numberOfSignificantValueDigits); + parameters.put("probabilities", probabilities); + c.setParameters(parameters); + c.setLookupFields(Collections.singletonList("ms")); + c.setOutputFields(Collections.singletonList("ms_his")); + + agg.open(c); + Map<String, Object> map = new HashMap<>(); + acc = new Accumulator(); + acc.setMetricsFields(map); + agg.initAccumulator(acc); + + event = new Event(); + Map<String, Object> fields = new HashMap<>(); + event.setExtractedFields(fields); + } }
\ No newline at end of file diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramTest.java index f177ca5..5905138 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/HdrHistogram/HdrHistogramTest.java @@ -1,102 +1,102 @@ -package com.geedgenetworks.core.udf.udaf.HdrHistogram;
-
-import com.geedgenetworks.common.Accumulator;
-import com.geedgenetworks.common.Event;
-import com.geedgenetworks.common.udf.AggregateFunction;
-import com.geedgenetworks.common.udf.UDFContext;
-import com.geedgenetworks.sketch.util.StringUtils;
-import org.HdrHistogram.ArrayHistogram;
-import org.junit.jupiter.api.Test;
-
-import java.nio.charset.StandardCharsets;
-import java.util.*;
-
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-public class HdrHistogramTest {
- AggregateFunction agg;
- Accumulator acc;
- Event event;
-
- @Test
- public void inputRegular() {
- double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1};
- initData( "regular", 2, "base64");
- long count = 100000;
- Map<String, Object> fields = event.getExtractedFields();
- for (int i = 1; i <= count; i++) {
- fields.put("ms", i);
- agg.add(event, acc);
- }
-
- long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray();
- String str = (String) agg.getResult(acc).getMetricsFields().get("ms_his");
- ArrayHistogram his = ArrayHistogram.fromBytes(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
-
- for (int i = 0; i < expects.length; i++) {
- long rst = his.getValueAtPercentile(probabilities[i] * 100);
- long expect = expects[i];
- double probability = probabilities[i];
- double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect;
- System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error));
- assertTrue(error <= 0.05);
- }
- }
-
- @Test
- public void inputSketch() {
- double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1};
- initData( "sketch", 2, "binary");
- long count = 100000;
- Map<String, Object> fields = event.getExtractedFields();
- long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray();
-
- ArrayHistogram his = new ArrayHistogram(2);
- for (int i = 1; i <= count; i++) {
- his.recordValue(i);
- }
- fields.put("ms", StringUtils.encodeBase64String(his.toBytes()));
- agg.add(event, acc);
-
- his = new ArrayHistogram(2);
- for (int i = 1; i <= count; i++) {
- his.recordValue(i);
- }
- fields.put("ms", his.toBytes());
- agg.add(event, acc);
-
- byte[] bytes = (byte[]) agg.getResult(acc).getMetricsFields().get("ms_his");
- ArrayHistogram h = ArrayHistogram.fromBytes(bytes);
-
- for (int i = 0; i < expects.length; i++) {
- long rst = h.getValueAtPercentile(probabilities[i] * 100);
- long expect = expects[i];
- double probability = probabilities[i];
- double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect;
- System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error));
- assertTrue(error <= 0.05);
- }
- }
-
- private void initData(String input_type, int numberOfSignificantValueDigits, String output_format){
- agg = new HdrHistogram();
- UDFContext c = new UDFContext();
- Map<String, Object> parameters = new HashMap<>();
- parameters.put("input_type", input_type);
- parameters.put("numberOfSignificantValueDigits", numberOfSignificantValueDigits);
- parameters.put("output_format", output_format);
- c.setParameters(parameters);
- c.setLookup_fields(Collections.singletonList("ms"));
- c.setOutput_fields(Collections.singletonList("ms_his"));
-
- agg.open(c);
- Map<String, Object> map = new HashMap<>();
- acc = new Accumulator();
- acc.setMetricsFields(map);
- agg.initAccumulator(acc);
-
- event = new Event();
- Map<String, Object> fields = new HashMap<>();
- event.setExtractedFields(fields);
- }
+package com.geedgenetworks.core.udf.udaf.HdrHistogram; + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.AggregateFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.sketch.util.StringUtils; +import org.HdrHistogram.ArrayHistogram; +import org.junit.jupiter.api.Test; + +import java.nio.charset.StandardCharsets; +import java.util.*; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class HdrHistogramTest { + AggregateFunction agg; + Accumulator acc; + Event event; + + @Test + public void inputRegular() { + double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1}; + initData( "regular", 2, "base64"); + long count = 100000; + Map<String, Object> fields = event.getExtractedFields(); + for (int i = 1; i <= count; i++) { + fields.put("ms", i); + agg.add(event, acc); + } + + long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray(); + String str = (String) agg.getResult(acc).getMetricsFields().get("ms_his"); + ArrayHistogram his = ArrayHistogram.fromBytes(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8))); + + for (int i = 0; i < expects.length; i++) { + long rst = his.getValueAtPercentile(probabilities[i] * 100); + long expect = expects[i]; + double probability = probabilities[i]; + double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect; + System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error)); + assertTrue(error <= 0.05); + } + } + + @Test + public void inputSketch() { + double[] probabilities = new double[]{0, 0.1, 0.25, 0.5, 0.75, 1}; + initData( "sketch", 2, "binary"); + long count = 100000; + Map<String, Object> fields = event.getExtractedFields(); + long[] expects = Arrays.stream(probabilities).mapToLong(x -> (long) (count * x)).toArray(); + + ArrayHistogram his = new ArrayHistogram(2); + for (int i = 1; i <= count; i++) { + his.recordValue(i); + } + fields.put("ms", StringUtils.encodeBase64String(his.toBytes())); + agg.add(event, acc); + + his = new ArrayHistogram(2); + for (int i = 1; i <= count; i++) { + his.recordValue(i); + } + fields.put("ms", his.toBytes()); + agg.add(event, acc); + + byte[] bytes = (byte[]) agg.getResult(acc).getMetricsFields().get("ms_his"); + ArrayHistogram h = ArrayHistogram.fromBytes(bytes); + + for (int i = 0; i < expects.length; i++) { + long rst = h.getValueAtPercentile(probabilities[i] * 100); + long expect = expects[i]; + double probability = probabilities[i]; + double error = probability <= 0.01? 0 : Math.abs(rst - expect) / (double) expect; + System.out.println(String.format("%s:%d,%d,%.4f", probability, expect , rst , error)); + assertTrue(error <= 0.05); + } + } + + private void initData(String input_type, int numberOfSignificantValueDigits, String output_format){ + agg = new HdrHistogram(); + UDFContext c = new UDFContext(); + Map<String, Object> parameters = new HashMap<>(); + parameters.put("input_type", input_type); + parameters.put("numberOfSignificantValueDigits", numberOfSignificantValueDigits); + parameters.put("output_format", output_format); + c.setParameters(parameters); + c.setLookupFields(Collections.singletonList("ms")); + c.setOutputFields(Collections.singletonList("ms_his")); + + agg.open(c); + Map<String, Object> map = new HashMap<>(); + acc = new Accumulator(); + acc.setMetricsFields(map); + agg.initAccumulator(acc); + + event = new Event(); + Map<String, Object> fields = new HashMap<>(); + event.setExtractedFields(fields); + } }
\ No newline at end of file diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldApproxCountDistinctTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldApproxCountDistinctTest.java index eae356d..b80d782 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldApproxCountDistinctTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldApproxCountDistinctTest.java @@ -1,87 +1,87 @@ -package com.geedgenetworks.core.udf.udaf.hlld;
-
-
-import com.geedgenetworks.common.Accumulator;
-import com.geedgenetworks.common.Event;
-import com.geedgenetworks.common.udf.AggregateFunction;
-import com.geedgenetworks.common.udf.UDFContext;
-import com.geedgenetworks.sketch.hlld.Hll;
-import com.geedgenetworks.sketch.util.StringUtils;
-import org.junit.jupiter.api.Test;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-
-public class HlldApproxCountDistinctTest {
- AggregateFunction agg;
- Accumulator acc;
- Event event;
-
-
- @Test
- public void inputRegular() {
- initData(14, "regular");
- long count = 100000;
- Map<String, Object> fields = event.getExtractedFields();
- for (int i = 0; i < count; i++) {
- fields.put("ip", i);
- agg.add(event, acc);
- }
-
- long rst = (long)agg.getResult(acc).getMetricsFields().get("ip_cnt");
- double error = Math.abs(rst - count) / (double) count;
- System.out.println(String.format("%d,%d,%.4f", count , rst , error));
- assertTrue(error <= 0.05);
- }
-
- @Test
- public void inputSketch() {
- initData(14, "sketch");
- long count = 150000;
- Map<String, Object> fields = event.getExtractedFields();
-
- Hll hll = new Hll(12);
- for (int i = 0; i < 100000; i++) {
- hll.add(i);
- }
- fields.put("ip", StringUtils.encodeBase64String(hll.toBytes()));
- agg.add(event, acc);
-
- hll = new Hll(13);
- for (int i = 50000; i < 150000; i++) {
- hll.add(i);
- }
- fields.put("ip", hll.toBytes());
- agg.add(event, acc);
-
- long rst = (long)agg.getResult(acc).getMetricsFields().get("ip_cnt");
- double error = Math.abs(rst - count) / (double) count;
- System.out.println(String.format("%d,%d,%.4f", count , rst , error));
- assertTrue(error <= 0.05);
- }
-
- private void initData(int precision, String input_type){
- agg = new HlldApproxCountDistinct();
- UDFContext c = new UDFContext();
- Map<String, Object> parameters = new HashMap<>();
- parameters.put("precision", precision);
- parameters.put("input_type", input_type);
- c.setParameters(parameters);
- c.setLookup_fields(Collections.singletonList("ip"));
- c.setOutput_fields(Collections.singletonList("ip_cnt"));
-
- agg.open(c);
- Map<String, Object> map = new HashMap<>();
- acc = new Accumulator();
- acc.setMetricsFields(map);
- agg.initAccumulator(acc);
-
- event = new Event();
- Map<String, Object> fields = new HashMap<>();
- event.setExtractedFields(fields);
- }
+package com.geedgenetworks.core.udf.udaf.hlld; + + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.AggregateFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.sketch.hlld.Hll; +import com.geedgenetworks.sketch.util.StringUtils; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertTrue; + + +public class HlldApproxCountDistinctTest { + AggregateFunction agg; + Accumulator acc; + Event event; + + + @Test + public void inputRegular() { + initData(14, "regular"); + long count = 100000; + Map<String, Object> fields = event.getExtractedFields(); + for (int i = 0; i < count; i++) { + fields.put("ip", i); + agg.add(event, acc); + } + + long rst = (long)agg.getResult(acc).getMetricsFields().get("ip_cnt"); + double error = Math.abs(rst - count) / (double) count; + System.out.println(String.format("%d,%d,%.4f", count , rst , error)); + assertTrue(error <= 0.05); + } + + @Test + public void inputSketch() { + initData(14, "sketch"); + long count = 150000; + Map<String, Object> fields = event.getExtractedFields(); + + Hll hll = new Hll(12); + for (int i = 0; i < 100000; i++) { + hll.add(i); + } + fields.put("ip", StringUtils.encodeBase64String(hll.toBytes())); + agg.add(event, acc); + + hll = new Hll(13); + for (int i = 50000; i < 150000; i++) { + hll.add(i); + } + fields.put("ip", hll.toBytes()); + agg.add(event, acc); + + long rst = (long)agg.getResult(acc).getMetricsFields().get("ip_cnt"); + double error = Math.abs(rst - count) / (double) count; + System.out.println(String.format("%d,%d,%.4f", count , rst , error)); + assertTrue(error <= 0.05); + } + + private void initData(int precision, String input_type){ + agg = new HlldApproxCountDistinct(); + UDFContext c = new UDFContext(); + Map<String, Object> parameters = new HashMap<>(); + parameters.put("precision", precision); + parameters.put("input_type", input_type); + c.setParameters(parameters); + c.setLookupFields(Collections.singletonList("ip")); + c.setOutputFields(Collections.singletonList("ip_cnt")); + + agg.open(c); + Map<String, Object> map = new HashMap<>(); + acc = new Accumulator(); + acc.setMetricsFields(map); + agg.initAccumulator(acc); + + event = new Event(); + Map<String, Object> fields = new HashMap<>(); + event.setExtractedFields(fields); + } }
\ No newline at end of file diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldTest.java index f489ee4..d6ed4c1 100644 --- a/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldTest.java +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/udaf/hlld/HlldTest.java @@ -1,86 +1,86 @@ -package com.geedgenetworks.core.udf.udaf.hlld;
-
-import com.geedgenetworks.common.Accumulator;
-import com.geedgenetworks.common.Event;
-import com.geedgenetworks.common.udf.AggregateFunction;
-import com.geedgenetworks.common.udf.UDFContext;
-import com.geedgenetworks.sketch.hlld.Hll;
-import com.geedgenetworks.sketch.util.StringUtils;
-import org.junit.jupiter.api.Test;
-
-import java.nio.charset.StandardCharsets;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-public class HlldTest {
- AggregateFunction agg;
- Accumulator acc;
- Event event;
-
- @Test
- public void inputRegular() {
- initData(14, "regular", "base64");
- long count = 100000;
- Map<String, Object> fields = event.getExtractedFields();
- for (int i = 0; i < count; i++) {
- fields.put("ip", i);
- agg.add(event, acc);
- }
-
- String hllStr = (String)agg.getResult(acc).getMetricsFields().get("ip_cnt");
- long rst = (long) Hll.fromBytes(StringUtils.decodeBase64(hllStr.getBytes(StandardCharsets.UTF_8))).size();
- double error = Math.abs(rst - count) / (double) count;
- System.out.println(String.format("%d,%d,%.4f", count , rst , error));
- assertTrue(error <= 0.05);
- }
-
- @Test
- public void inputSketch() {
- initData(14, "sketch", "binary");
- long count = 150000;
- Map<String, Object> fields = event.getExtractedFields();
- for (int i = 0; i < 100000; i++) {
- Hll hll = new Hll(12);
- hll.add(i);
- fields.put("ip", StringUtils.encodeBase64String(hll.toBytes()));
- agg.add(event, acc);
- }
- for (int i = 50000; i < 150000; i++) {
- Hll hll = new Hll(13);
- hll.add(i);
- fields.put("ip", hll.toBytes());
- agg.add(event, acc);
- }
-
- byte[] hllBytes = (byte[])agg.getResult(acc).getMetricsFields().get("ip_cnt");
- long rst = (long) Hll.fromBytes(hllBytes).size();
- double error = Math.abs(rst - count) / (double) count;
- System.out.println(String.format("%d,%d,%.4f", count , rst , error));
- assertTrue(error <= 0.05);
- }
-
- private void initData(int precision, String input_type, String output_format){
- agg = new Hlld();
- UDFContext c = new UDFContext();
- Map<String, Object> parameters = new HashMap<>();
- parameters.put("precision", precision);
- parameters.put("input_type", input_type);
- parameters.put("output_format", output_format);
- c.setParameters(parameters);
- c.setLookup_fields(Collections.singletonList("ip"));
- c.setOutput_fields(Collections.singletonList("ip_cnt"));
-
- agg.open(c);
- Map<String, Object> map = new HashMap<>();
- acc = new Accumulator();
- acc.setMetricsFields(map);
- agg.initAccumulator(acc);
-
- event = new Event();
- Map<String, Object> fields = new HashMap<>();
- event.setExtractedFields(fields);
- }
-}
+package com.geedgenetworks.core.udf.udaf.hlld; + +import com.geedgenetworks.common.Accumulator; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.AggregateFunction; +import com.geedgenetworks.common.udf.UDFContext; +import com.geedgenetworks.sketch.hlld.Hll; +import com.geedgenetworks.sketch.util.StringUtils; +import org.junit.jupiter.api.Test; + +import java.nio.charset.StandardCharsets; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class HlldTest { + AggregateFunction agg; + Accumulator acc; + Event event; + + @Test + public void inputRegular() { + initData(14, "regular", "base64"); + long count = 100000; + Map<String, Object> fields = event.getExtractedFields(); + for (int i = 0; i < count; i++) { + fields.put("ip", i); + agg.add(event, acc); + } + + String hllStr = (String)agg.getResult(acc).getMetricsFields().get("ip_cnt"); + long rst = (long) Hll.fromBytes(StringUtils.decodeBase64(hllStr.getBytes(StandardCharsets.UTF_8))).size(); + double error = Math.abs(rst - count) / (double) count; + System.out.println(String.format("%d,%d,%.4f", count , rst , error)); + assertTrue(error <= 0.05); + } + + @Test + public void inputSketch() { + initData(14, "sketch", "binary"); + long count = 150000; + Map<String, Object> fields = event.getExtractedFields(); + for (int i = 0; i < 100000; i++) { + Hll hll = new Hll(12); + hll.add(i); + fields.put("ip", StringUtils.encodeBase64String(hll.toBytes())); + agg.add(event, acc); + } + for (int i = 50000; i < 150000; i++) { + Hll hll = new Hll(13); + hll.add(i); + fields.put("ip", hll.toBytes()); + agg.add(event, acc); + } + + byte[] hllBytes = (byte[])agg.getResult(acc).getMetricsFields().get("ip_cnt"); + long rst = (long) Hll.fromBytes(hllBytes).size(); + double error = Math.abs(rst - count) / (double) count; + System.out.println(String.format("%d,%d,%.4f", count , rst , error)); + assertTrue(error <= 0.05); + } + + private void initData(int precision, String input_type, String output_format){ + agg = new Hlld(); + UDFContext c = new UDFContext(); + Map<String, Object> parameters = new HashMap<>(); + parameters.put("precision", precision); + parameters.put("input_type", input_type); + parameters.put("output_format", output_format); + c.setParameters(parameters); + c.setLookupFields(Collections.singletonList("ip")); + c.setOutputFields(Collections.singletonList("ip_cnt")); + + agg.open(c); + Map<String, Object> map = new HashMap<>(); + acc = new Accumulator(); + acc.setMetricsFields(map); + agg.initAccumulator(acc); + + event = new Event(); + Map<String, Object> fields = new HashMap<>(); + event.setExtractedFields(fields); + } +} diff --git a/groot-core/src/test/java/com/geedgenetworks/core/udf/udtf/UnrollTest.java b/groot-core/src/test/java/com/geedgenetworks/core/udf/udtf/UnrollTest.java new file mode 100644 index 0000000..3320f38 --- /dev/null +++ b/groot-core/src/test/java/com/geedgenetworks/core/udf/udtf/UnrollTest.java @@ -0,0 +1,109 @@ +package com.geedgenetworks.core.udf.udtf; + +import com.alibaba.fastjson2.JSON; +import com.geedgenetworks.common.Event; +import com.geedgenetworks.common.udf.UDFContext; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.junit.jupiter.api.Assertions.*; + + +public class UnrollTest { + PathUnroll pathUnroll; + Event event; + + @Test + public void explodePathWithNoFileField() { + init("path", "out_path", "."); + Map<String, Object> fields = event.getExtractedFields(); + fields.put("path", "ETHERNET.IPv4.TCP.ssl"); + String[] excepted = new String[]{"ETHERNET","ETHERNET.IPv4","ETHERNET.IPv4.TCP","ETHERNET.IPv4.TCP.ssl"}; + String[] outPaths = pathUnroll.evaluate(event).stream().map(x -> (String)x.getExtractedFields().get("out_path")).toArray(String[]::new); + System.out.println(JSON.toJSONString(outPaths)); + assertArrayEquals(outPaths, excepted); + // 忽略结尾的分隔符 + fields.put("path", "ETHERNET.IPv4.TCP.ssl."); + outPaths = pathUnroll.evaluate(event).stream().map(x -> (String)x.getExtractedFields().get("out_path")).toArray(String[]::new); + System.out.println(JSON.toJSONString(outPaths)); + assertArrayEquals(outPaths, excepted); + // 空路径不输出 + fields.put("path", ""); + outPaths = pathUnroll.evaluate(event).stream().map(x -> (String)x.getExtractedFields().get("out_path")).toArray(String[]::new); + System.out.println(JSON.toJSONString(outPaths)); + assertTrue(outPaths.length == 0); + + init("path", "out_path", "/"); + fields = event.getExtractedFields(); + fields.put("path", "ETHERNET/IPv4/TCP/ssl"); + excepted = new String[]{"ETHERNET","ETHERNET/IPv4","ETHERNET/IPv4/TCP","ETHERNET/IPv4/TCP/ssl"}; + outPaths = pathUnroll.evaluate(event).stream().map(x -> (String)x.getExtractedFields().get("out_path")).toArray(String[]::new); + System.out.println(JSON.toJSONString(outPaths)); + assertArrayEquals(outPaths, excepted); + // 忽略结尾的分隔符 + fields.put("path", "ETHERNET/IPv4/TCP/ssl/"); + outPaths = pathUnroll.evaluate(event).stream().map(x -> (String)x.getExtractedFields().get("out_path")).toArray(String[]::new); + System.out.println(JSON.toJSONString(outPaths)); + assertArrayEquals(outPaths, excepted); + } + + @Test + public void explodePathWithFileField() { + init("path", "file", "out_path", "out_file", "."); + Map<String, Object> fields = event.getExtractedFields(); + fields.put("path", "ETHERNET.IPv4.TCP.ssl"); + fields.put("file", "ssl"); + String[] excepted = new String[]{"ETHERNET", "ETHERNET.IPv4", "ETHERNET.IPv4.TCP", "ETHERNET.IPv4.TCP.ssl"}; + String[] exceptedFile = new String[]{null, null, null, "ssl"}; + String[] outPaths = pathUnroll.evaluate(event).stream().map(x -> (String) x.getExtractedFields().get("out_path")).toArray(String[]::new); + String[] outFiles = pathUnroll.evaluate(event).stream().map(x -> (String) x.getExtractedFields().get("out_file")).toArray(String[]::new); + System.out.println(JSON.toJSONString(outPaths)); + System.out.println(JSON.toJSONString(outFiles)); + assertArrayEquals(outPaths, excepted); + assertArrayEquals(outFiles, exceptedFile); + // 忽略结尾的分隔符 + fields.put("path", "ETHERNET.IPv4.TCP.ssl."); + fields.put("file", "ssl"); + outPaths = pathUnroll.evaluate(event).stream().map(x -> (String) x.getExtractedFields().get("out_path")).toArray(String[]::new); + outFiles = pathUnroll.evaluate(event).stream().map(x -> (String) x.getExtractedFields().get("out_file")).toArray(String[]::new); + System.out.println(JSON.toJSONString(outPaths)); + System.out.println(JSON.toJSONString(outFiles)); + assertArrayEquals(outPaths, excepted); + assertArrayEquals(outFiles, exceptedFile); + + fields.put("path", "ETHERNET.IPv4.TCP.ssl"); + fields.put("file", "ssl.aa"); + excepted = new String[]{"ETHERNET", "ETHERNET.IPv4", "ETHERNET.IPv4.TCP", "ETHERNET.IPv4.TCP.ssl", "ETHERNET.IPv4.TCP.ssl.ssl.aa"}; + exceptedFile = new String[]{null, null, null, null,"ssl.aa"}; + outPaths = pathUnroll.evaluate(event).stream().map(x -> (String) x.getExtractedFields().get("out_path")).toArray(String[]::new); + outFiles = pathUnroll.evaluate(event).stream().map(x -> (String) x.getExtractedFields().get("out_file")).toArray(String[]::new); + System.out.println(JSON.toJSONString(outPaths)); + System.out.println(JSON.toJSONString(outFiles)); + assertArrayEquals(outPaths, excepted); + assertArrayEquals(outFiles, exceptedFile); + } + + private void init(String pathField, String outputPathField, String separator){ + init(pathField, null, outputPathField, null, separator); + } + + private void init(String pathField, String fileField, String outputPathField, String outputFileField, String separator){ + pathUnroll = new PathUnroll(); + UDFContext c = new UDFContext(); + Map<String, Object> parameters = new HashMap<>(); + parameters.put("separator", separator); + c.setParameters(parameters); + c.setLookupFields(Arrays.asList(pathField, fileField).stream().filter(x -> x != null).collect(Collectors.toList())); + c.setOutputFields(Arrays.asList(outputPathField, outputFileField).stream().filter(x -> x != null).collect(Collectors.toList())); + + pathUnroll.open(null, c); + event = new Event(); + Map<String, Object> fields = new HashMap<>(); + event.setExtractedFields(fields); + } +}
\ No newline at end of file diff --git a/groot-core/src/test/java/com/geedgenetworks/core/utils/LoadIntervalDataUtilTest.java b/groot-core/src/test/java/com/geedgenetworks/core/utils/LoadIntervalDataUtilTest.java new file mode 100644 index 0000000..b7c6306 --- /dev/null +++ b/groot-core/src/test/java/com/geedgenetworks/core/utils/LoadIntervalDataUtilTest.java @@ -0,0 +1,80 @@ +package com.geedgenetworks.core.utils;
+
+
+import java.sql.Timestamp;
+
+public class LoadIntervalDataUtilTest {
+
+ public static void main(String[] args) throws Exception{
+ //testNoError();
+ //testNotUpdateDataOnStart();
+ //testWithErrorAndNotFail();
+ testWithErrorAndFail();
+ }
+
+ public static void testNoError() throws Exception{
+ LoadIntervalDataUtil<Timestamp> util = LoadIntervalDataUtil.newInstance(() -> new Timestamp(System.currentTimeMillis()),
+ LoadIntervalDataOptions.defaults("time", 3000));
+
+ System.out.println(new Timestamp(System.currentTimeMillis()) + " - " + util.data());
+
+ for (int i = 0; i < 10; i++) {
+ Thread.sleep(1000);
+ System.out.println(new Timestamp(System.currentTimeMillis()) + " - " + util.data());
+ }
+
+ util.stop();
+ }
+
+ public static void testNotUpdateDataOnStart() throws Exception{
+ LoadIntervalDataUtil<Timestamp> util = LoadIntervalDataUtil.newInstance(() -> new Timestamp(System.currentTimeMillis()),
+ LoadIntervalDataOptions.builder().withName("time").withIntervalMs(3000).withUpdateDataOnStart(false).build());
+
+ System.out.println(new Timestamp(System.currentTimeMillis()) + " - " + util.data());
+
+ for (int i = 0; i < 10; i++) {
+ Thread.sleep(1000);
+ System.out.println(new Timestamp(System.currentTimeMillis()) + " - " + util.data());
+ }
+
+ util.stop();
+ }
+
+ public static void testWithErrorAndNotFail() throws Exception{
+ final long start = System.currentTimeMillis();
+ LoadIntervalDataUtil<Timestamp> util = LoadIntervalDataUtil.newInstance(() -> {
+ if(System.currentTimeMillis() - start >= 5000){
+ throw new RuntimeException(new Timestamp(System.currentTimeMillis()).toString());
+ }
+ return new Timestamp(System.currentTimeMillis());
+ }, LoadIntervalDataOptions.defaults("time", 3000));
+
+ System.out.println(new Timestamp(System.currentTimeMillis()) + " - " + util.data());
+
+ for (int i = 0; i < 10; i++) {
+ Thread.sleep(1000);
+ System.out.println(new Timestamp(System.currentTimeMillis()) + " - " + util.data());
+ }
+
+ util.stop();
+ }
+
+ public static void testWithErrorAndFail() throws Exception{
+ final long start = System.currentTimeMillis();
+ LoadIntervalDataUtil<Timestamp> util = LoadIntervalDataUtil.newInstance(() -> {
+ if(System.currentTimeMillis() - start >= 5000){
+ throw new RuntimeException(new Timestamp(System.currentTimeMillis()).toString());
+ }
+ return new Timestamp(System.currentTimeMillis());
+ }, LoadIntervalDataOptions.builder().withName("time").withIntervalMs(3000).withFailOnException(true).build());
+
+ System.out.println(new Timestamp(System.currentTimeMillis()) + " - " + util.data());
+
+ for (int i = 0; i < 10; i++) {
+ Thread.sleep(1000);
+ System.out.println(new Timestamp(System.currentTimeMillis()) + " - " + util.data());
+ }
+
+ util.stop();
+ }
+}
\ No newline at end of file diff --git a/groot-core/src/test/java/com/geedgenetworks/core/utils/SingleValueMapTest.java b/groot-core/src/test/java/com/geedgenetworks/core/utils/SingleValueMapTest.java new file mode 100644 index 0000000..f5f1e7c --- /dev/null +++ b/groot-core/src/test/java/com/geedgenetworks/core/utils/SingleValueMapTest.java @@ -0,0 +1,98 @@ +package com.geedgenetworks.core.utils;
+
+import org.junit.jupiter.api.Assertions;
+
+import java.sql.Timestamp;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class SingleValueMapTest {
+
+ public static void main(String[] args) throws Exception {
+ //testSingleValue();
+ testSingleValueWithLoadIntervalDataUtil();
+ }
+
+ public static void testSingleValue() throws Exception {
+ Thread[] threads = new Thread[20];
+ for (int i = 0; i < threads.length; i++) {
+ threads[i] = new Thread(() -> {
+ SingleValueMap.Data<ConnDada> connDada = null;
+ try {
+ connDada = SingleValueMap.acquireData("conn_data", () -> new ConnDada(), x -> {
+ System.out.println("close conn");
+ });
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+
+ try {
+ Thread.sleep(ThreadLocalRandom.current().nextInt(5) * 10);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+
+ connDada.release();
+ }, "Thread-" + i);
+ }
+
+ for (int i = 0; i < threads.length; i++) {
+ threads[i].start();
+ }
+
+ for (int i = 0; i < threads.length; i++) {
+ threads[i].join();
+ }
+
+ System.out.println("initCnt:" + ConnDada.initCnt.get());
+ Assertions.assertEquals(ConnDada.initCnt.get(), 1);
+ }
+
+ public static void testSingleValueWithLoadIntervalDataUtil() throws Exception {
+ Thread[] threads = new Thread[20];
+ for (int i = 0; i < threads.length; i++) {
+ threads[i] = new Thread(() -> {
+ SingleValueMap.Data<LoadIntervalDataUtil<Timestamp>> util = null;
+ try {
+ util = SingleValueMap.acquireData("LoadIntervalDataUtil",
+ () -> LoadIntervalDataUtil.newInstance(() -> new Timestamp(System.currentTimeMillis()), LoadIntervalDataOptions.defaults("time", 3000)),
+ LoadIntervalDataUtil::stop);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+
+
+ try {
+ for (int j = 0; j < 10; j++) {
+ Thread.sleep(1000);
+ System.out.println(Thread.currentThread().getName() + " - " + new Timestamp(System.currentTimeMillis()) + " - " + util.getData().data());
+ }
+
+ Thread.sleep(ThreadLocalRandom.current().nextInt(5) * 10);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+
+ util.release();
+ }, "Thread-" + i);
+ }
+
+ for (int i = 0; i < threads.length; i++) {
+ threads[i].start();
+ }
+
+ for (int i = 0; i < threads.length; i++) {
+ threads[i].join();
+ }
+
+ }
+
+ public static class ConnDada {
+ static AtomicInteger initCnt = new AtomicInteger(0);
+ public ConnDada(){
+ System.out.println("ConnDada init");
+ initCnt.incrementAndGet();
+ }
+
+ }
+}
\ No newline at end of file |
