|
@@ -1,54 +1,31 @@
|
|
|
package com.cecf.observe.agentstream;
|
|
|
|
|
|
-import com.cecf.observe.agentstream.common.Config;
|
|
|
-import com.cecf.observe.agentstream.metrics.ServiceInstanceMeta;
|
|
|
-import com.cecf.observe.agentstream.metrics.ServiceInstanceMetric;
|
|
|
-import com.cecf.observe.agentstream.metrics.ServiceMetricAggregator;
|
|
|
+import com.cecf.observe.agentstream.common.StringUtils;
|
|
|
import com.cecf.observe.agentstream.metrics.*;
|
|
|
-import com.cecf.observe.agentstream.otel.*;
|
|
|
-import com.cecf.observe.agentstream.traceurl.PojoSpan2TraceURLMapper;
|
|
|
-import com.cecf.observe.agentstream.traceurl.TraceURL;
|
|
|
-import com.cecf.observe.agentstream.traceurl.TraceURLCHConverter;
|
|
|
-import com.cecf.observe.agentstream.traceurl.URLAggregator;
|
|
|
-import io.opentelemetry.proto.collector.trace.v1.ExportTraceServiceRequest;
|
|
|
+import com.cecf.observe.agentstream.otel.PojoSpan;
|
|
|
+import com.cecf.observe.agentstream.otel.ServiceNameFilter;
|
|
|
import io.opentelemetry.proto.trace.v1.Status;
|
|
|
-import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner;
|
|
|
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
|
|
|
import org.apache.flink.api.common.functions.FlatMapFunction;
|
|
|
import org.apache.flink.api.common.functions.MapFunction;
|
|
|
-import org.apache.flink.api.common.serialization.DeserializationSchema;
|
|
|
-import org.apache.flink.api.common.serialization.SimpleStringSchema;
|
|
|
import org.apache.flink.api.java.tuple.Tuple2;
|
|
|
import org.apache.flink.api.java.typeutils.TypeExtractor;
|
|
|
-import org.apache.flink.api.java.utils.ParameterTool;
|
|
|
-import org.apache.flink.configuration.Configuration;
|
|
|
-import org.apache.flink.configuration.RestartStrategyOptions;
|
|
|
import org.apache.flink.connector.jdbc.JdbcConnectionOptions;
|
|
|
import org.apache.flink.connector.jdbc.JdbcSink;
|
|
|
-import org.apache.flink.connector.kafka.source.KafkaSource;
|
|
|
import org.apache.flink.streaming.api.datastream.DataStream;
|
|
|
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
|
|
|
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
|
|
|
-import org.apache.flink.streaming.api.functions.ProcessFunction;
|
|
|
import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
|
|
|
import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
|
|
|
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
|
|
|
import org.apache.flink.util.Collector;
|
|
|
-import org.apache.flink.util.OutputTag;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
-import org.yaml.snakeyaml.Yaml;
|
|
|
import ru.ivi.opensource.flinkclickhousesink.ClickHouseSink;
|
|
|
-import ru.ivi.opensource.flinkclickhousesink.model.ClickHouseClusterSettings;
|
|
|
-import ru.ivi.opensource.flinkclickhousesink.model.ClickHouseSinkConst;
|
|
|
|
|
|
-import java.io.FileInputStream;
|
|
|
-import java.io.FileNotFoundException;
|
|
|
-import java.io.InputStream;
|
|
|
import java.sql.Timestamp;
|
|
|
import java.time.Duration;
|
|
|
-import java.util.Collections;
|
|
|
-import java.util.HashMap;
|
|
|
+import java.util.ArrayList;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
|
|
@@ -57,128 +34,9 @@ public class Main {
|
|
|
|
|
|
public static final Logger logger = LoggerFactory.getLogger(Main.class);
|
|
|
|
|
|
- public static DataStream<PojoSpan> buildPojoSourceStream(StreamExecutionEnvironment env, OasConfig oasConfig) {
|
|
|
- SingleOutputStreamOperator<PojoSpan> pbOp = null;
|
|
|
- SingleOutputStreamOperator<PojoSpan> jsonOp = null;
|
|
|
- boolean enablePbSource = oasConfig.getPbSource().isEnabled();
|
|
|
- boolean enableJsonSource = oasConfig.getJsonSource().isEnabled();
|
|
|
- if (!enableJsonSource && !enablePbSource) {
|
|
|
- throw new RuntimeException("至少配置一个Source");
|
|
|
- }
|
|
|
- OasConfig.JobConfig jobConfig = oasConfig.getJobConfig();
|
|
|
- WatermarkStrategy<PojoSpan> watermarkStrategy = WatermarkStrategy
|
|
|
- .<PojoSpan>forBoundedOutOfOrderness(Duration.ofSeconds(jobConfig.getWatermarkOrderlessSeconds()))
|
|
|
- .withTimestampAssigner(
|
|
|
- (SerializableTimestampAssigner<PojoSpan>) (element, recordTimestamp) -> element.Timestamp.getTime()
|
|
|
- ).withIdleness(
|
|
|
- Duration.ofSeconds(jobConfig.getWatermarkIdlenessSeconds())
|
|
|
- );
|
|
|
- if (enablePbSource) {
|
|
|
- OasConfig.KafkaSourceConfig pbKafkaSourceConfig = oasConfig.getPbSource();
|
|
|
- DeserializationSchema<ExportTraceServiceRequest> pbDeserializationSchema = new ExportTraceServiceRequestSchema();
|
|
|
- KafkaSource<ExportTraceServiceRequest> pbSource =
|
|
|
- (new KafkaSourceBuilder<ExportTraceServiceRequest>()).setBrokers(pbKafkaSourceConfig.getBrokers())
|
|
|
- .setTopics(pbKafkaSourceConfig.getTopics())
|
|
|
- .setGroupID(pbKafkaSourceConfig.getConsumerGroupID())
|
|
|
- .setDeserializationSchema(pbDeserializationSchema)
|
|
|
- .build();
|
|
|
- pbOp = env.fromSource(pbSource, WatermarkStrategy.noWatermarks(), "pbSource")
|
|
|
- .setParallelism(oasConfig.getJobConfig().getKafkaSourceSinkParallelism())
|
|
|
- .flatMap(new TraceServiceRequest2PojoSpanFlatMap())
|
|
|
- .setParallelism(jobConfig.getPojoParseParallelism())
|
|
|
- .name("TraceServiceRequestFlatMap2PojoSpan")
|
|
|
- .assignTimestampsAndWatermarks(watermarkStrategy)
|
|
|
- .name("assignTimestampAndWatermark")
|
|
|
- .setParallelism(jobConfig.getPojoParseParallelism());
|
|
|
- }
|
|
|
-
|
|
|
- if (enableJsonSource) {
|
|
|
- OasConfig.KafkaSourceConfig jsonKafkaSourceConfig = oasConfig.getJsonSource();
|
|
|
- KafkaSource<String> jsonSource =
|
|
|
- (new KafkaSourceBuilder<String>()).setBrokers(jsonKafkaSourceConfig.getBrokers())
|
|
|
- .setTopics(jsonKafkaSourceConfig.getTopics())
|
|
|
- .setGroupID(jsonKafkaSourceConfig.getConsumerGroupID())
|
|
|
- .setDeserializationSchema(new SimpleStringSchema())
|
|
|
- .build();
|
|
|
- jsonOp = env.fromSource(jsonSource, WatermarkStrategy.noWatermarks(), "jsonSource")
|
|
|
- .setParallelism(oasConfig.getJobConfig().getKafkaSourceSinkParallelism())
|
|
|
- .flatMap(new JSONString2PojoSpanFlatMapFunc())
|
|
|
- .setParallelism(jobConfig.getPojoParseParallelism())
|
|
|
- .name("JsonFlatMap2PojoSpan")
|
|
|
- .assignTimestampsAndWatermarks(watermarkStrategy)
|
|
|
- .name("assignTimestampAndWatermark")
|
|
|
- .setParallelism(jobConfig.getPojoParseParallelism());
|
|
|
- }
|
|
|
- if (pbOp == null) {
|
|
|
- return jsonOp;
|
|
|
- }
|
|
|
- if (jsonOp == null) {
|
|
|
- return pbOp;
|
|
|
- }
|
|
|
- return pbOp.union(jsonOp);
|
|
|
- }
|
|
|
-
|
|
|
- private static Map<String, String> getClickhouseGlobalConfig(OasConfig.ClickhouseStreamConfig clickhouseStreamConfig) {
|
|
|
- Map<String, String> globalParameters = new HashMap<>();
|
|
|
- String chHosts = clickhouseStreamConfig.getChHost();
|
|
|
- globalParameters.put(ClickHouseClusterSettings.CLICKHOUSE_HOSTS, chHosts);
|
|
|
- globalParameters.put(ClickHouseClusterSettings.CLICKHOUSE_USER, clickhouseStreamConfig.getChUser());
|
|
|
- globalParameters.put(ClickHouseClusterSettings.CLICKHOUSE_PASSWORD, clickhouseStreamConfig.getChPwd());
|
|
|
-
|
|
|
- globalParameters.put(ClickHouseSinkConst.TIMEOUT_SEC, clickhouseStreamConfig.getChTimeoutSeconds().toString());
|
|
|
- globalParameters.put(ClickHouseSinkConst.FAILED_RECORDS_PATH, "./failed_record");
|
|
|
- globalParameters.put(ClickHouseSinkConst.NUM_WRITERS, clickhouseStreamConfig.getChWriterNum().toString());
|
|
|
- globalParameters.put(ClickHouseSinkConst.NUM_RETRIES, clickhouseStreamConfig.getChWriteRetry().toString());
|
|
|
- globalParameters.put(ClickHouseSinkConst.QUEUE_MAX_CAPACITY, clickhouseStreamConfig.getChWriteQueueSize().toString());
|
|
|
- globalParameters.put(ClickHouseSinkConst.IGNORING_CLICKHOUSE_SENDING_EXCEPTION_ENABLED, "true");
|
|
|
- return globalParameters;
|
|
|
- }
|
|
|
-
|
|
|
- private static StreamExecutionEnvironment buildExecuteEnv(ParameterTool parameters, OasConfig oasConfig) {
|
|
|
- OasConfig.JobConfig jobConfig = oasConfig.getJobConfig();
|
|
|
- Configuration envConfig = new Configuration();
|
|
|
- envConfig.set(RestartStrategyOptions.RESTART_STRATEGY, "fixed-delay");
|
|
|
- envConfig.set(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS, Integer.MAX_VALUE); // number of restart attempts
|
|
|
- envConfig.set(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_DELAY, Duration.ofSeconds(10)); // delay
|
|
|
-
|
|
|
-// envConfig.set(StateBackendOptions.STATE_BACKEND, "rocksdb");
|
|
|
-//
|
|
|
-// envConfig.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem");
|
|
|
-// envConfig.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, "hdfs:///checkpoints-data/");
|
|
|
-
|
|
|
-
|
|
|
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(envConfig);
|
|
|
- env.getConfig().setGlobalJobParameters(parameters);
|
|
|
- if (jobConfig.isDisableOperatorChaining()) {
|
|
|
- env.disableOperatorChaining();
|
|
|
- }
|
|
|
-
|
|
|
-// CheckpointConfig checkPointConfig = env.getCheckpointConfig();
|
|
|
-// checkPointConfig.setExternalizedCheckpointCleanup(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
|
|
|
-
|
|
|
- return env;
|
|
|
- }
|
|
|
-
|
|
|
- public static OasConfig parseConfig() {
|
|
|
- String oasConfigPath = Config.getString("ob.config.path", "/conf/oas.yaml");
|
|
|
- Yaml yaml = new Yaml();
|
|
|
- try {
|
|
|
- InputStream inputStream = new FileInputStream(oasConfigPath);
|
|
|
- OasConfig oasConfig = yaml.loadAs(inputStream, OasConfig.class);
|
|
|
- logger.info("[配置]: {}", oasConfig.toString());
|
|
|
- oasConfig.fillDefaults();
|
|
|
- return oasConfig;
|
|
|
- } catch (FileNotFoundException e) {
|
|
|
- throw new RuntimeException(String.format("读取配置[%s]错误:%s", oasConfigPath, e));
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
public static void main(String[] args) throws Exception {
|
|
|
- OasConfig oasConfig = parseConfig();
|
|
|
-
|
|
|
- ParameterTool parameters = ParameterTool.fromMap(getClickhouseGlobalConfig(oasConfig.getClickhouseStreamConfig()));
|
|
|
- StreamExecutionEnvironment env = buildExecuteEnv(parameters, oasConfig);
|
|
|
-
|
|
|
+ OasConfig oasConfig = OasConfig.fromPath();
|
|
|
+ StreamExecutionEnvironment env = ExecuteEnvBuilder.buildExecuteEnv(oasConfig);
|
|
|
FillAppNameMapFunc fillAppNameMapFunc = FillAppNameMapFunc.buildFromConfig(oasConfig.getMySqlConfig());
|
|
|
|
|
|
// build base stream
|
|
@@ -189,7 +47,7 @@ public class Main {
|
|
|
.setWriteBufSize(oasConfig.getClickhouseStreamConfig().getChWriteBufSize().toString())
|
|
|
.setConverter(new PojoSpanSpanCSVConverter())
|
|
|
.build();
|
|
|
- DataStream<PojoSpan> baseStream = buildPojoSourceStream(env, oasConfig)
|
|
|
+ DataStream<PojoSpan> baseStream = PojoSourceBuilder.buildPojoSourceStream(env, oasConfig)
|
|
|
.map(fillAppNameMapFunc)
|
|
|
.name("添加缺失的AppName")
|
|
|
.setParallelism(jobConfig.getAttachAppNameParallelism())
|
|
@@ -197,29 +55,166 @@ public class Main {
|
|
|
.name("过滤ServiceName")
|
|
|
.setParallelism(jobConfig.getFilterByServiceNameParallelism());
|
|
|
// clickhouse traces stream
|
|
|
+// SlotSharingGroup clickhouseSinkSSG = SlotSharingGroup.newBuilder("clickhouseSSG")
|
|
|
+// .setCpuCores(1)
|
|
|
+// .setManagedMemoryMB(300)
|
|
|
+// .setTaskOffHeapMemoryMB(500)
|
|
|
+// .setTaskHeapMemoryMB(2000)
|
|
|
+// .build();
|
|
|
+// env.registerSlotSharingGroup(clickhouseSinkSSG);
|
|
|
baseStream
|
|
|
.addSink(tracesClickhouseSink)
|
|
|
.setParallelism(jobConfig.getClickhouseSinkParallelism())
|
|
|
+// .slotSharingGroup(clickhouseSinkSSG)
|
|
|
.name("ClickhouseSink");
|
|
|
|
|
|
- DataStream<PojoSpan> endpointStream = baseStream.filter(new EndpointPojoFilter());
|
|
|
+ OasConfig.SpanMetricConfig configSpanMetric = oasConfig.getSpanMetric();
|
|
|
+ JdbcConnectionOptions jdbcConnectionOptions = new JdbcConnectionOptions.JdbcConnectionOptionsBuilder()
|
|
|
+ .withUrl(configSpanMetric.getJdbcUrl())
|
|
|
+ .withDriverName("org.postgresql.Driver")
|
|
|
+ .withUsername(configSpanMetric.getPgUser())
|
|
|
+ .withPassword(configSpanMetric.getPgPwd())
|
|
|
+ .build();
|
|
|
+ (new SpanTagsProcess.SpanTagsProcessBuilder())
|
|
|
+ .setWindowDuration(Duration.ofMinutes(1))
|
|
|
+ .setJdbcConnectionOptions(jdbcConnectionOptions)
|
|
|
+ .buildStream(baseStream);
|
|
|
+
|
|
|
+ DataStream<PojoSpan> endpointStream = baseStream
|
|
|
+ .filter(new EndpointPojoFilter())
|
|
|
+ .setParallelism(jobConfig.getEndpointSpanFilterParallelism())
|
|
|
+ .name("filterEndpointSpans");
|
|
|
DataStream<ServiceMetric> serviceMetricStreamLevel1 = endpointStream
|
|
|
.keyBy(PojoSpan::getServiceName)
|
|
|
.window(TumblingEventTimeWindows.of(Duration.ofSeconds(15)))
|
|
|
.aggregate(new ServiceMetricAggregator(), new ServiceMetricTimestampAdder());
|
|
|
- sinkServiceMetric(serviceMetricStreamLevel1, oasConfig.getSpanMetric(), Duration.ofSeconds(15));
|
|
|
|
|
|
- DataStream<ServiceMetric> serviceMetricDataStream5m = nextLevelServiceMetric(serviceMetricStreamLevel1,
|
|
|
- jobConfig.getWatermarkIdlenessSeconds(), 5);
|
|
|
- sinkServiceMetric(serviceMetricDataStream5m, oasConfig.getSpanMetric(), Duration.ofMinutes(5));
|
|
|
+ Duration du15s = Duration.ofSeconds(15);
|
|
|
+ DataStream<Tuple2<ServiceMetricMeta, EndpointMetrics>> serviceOnlyStream = serviceMetricStreamLevel1
|
|
|
+ .map((MapFunction<ServiceMetric, Tuple2<ServiceMetricMeta, EndpointMetrics>>) (v) -> new Tuple2<>(
|
|
|
+ new ServiceMetricMeta(v.getAppName(), v.getTimestamp(), v.getServiceName()), v.getServerMetrics(du15s)))
|
|
|
+ .name("serviceMetric2ServiceMeta+EndpointMetrics")
|
|
|
+ .returns(
|
|
|
+ TypeExtractor.getForObject(new Tuple2<>(new ServiceMetricMeta(), new EndpointMetrics()))
|
|
|
+ );
|
|
|
+ sinkServiceOnlyMetric(serviceOnlyStream, jdbcConnectionOptions, du15s);
|
|
|
+
|
|
|
+ DataStream<Tuple2<AppNameMetricMeta, EndpointMetrics>> appNameMetricsStream = serviceOnlyStream
|
|
|
+ .assignTimestampsAndWatermarks(WatermarkStrategy.forBoundedOutOfOrderness(Duration.ofSeconds(10)))
|
|
|
+ .keyBy(t -> t.f0.getAppName())
|
|
|
+ .window(TumblingEventTimeWindows.of(du15s))
|
|
|
+ .process(new ProcessWindowFunction<>() {
|
|
|
+ @Override
|
|
|
+ public void process(String s, ProcessWindowFunction<Tuple2<ServiceMetricMeta, EndpointMetrics>,
|
|
|
+ Tuple2<AppNameMetricMeta, EndpointMetrics>, String, TimeWindow>.Context context,
|
|
|
+ Iterable<Tuple2<ServiceMetricMeta, EndpointMetrics>> elements,
|
|
|
+ Collector<Tuple2<AppNameMetricMeta, EndpointMetrics>> out) throws Exception {
|
|
|
+ List<EndpointMetrics> endpointMetrics = new ArrayList<>();
|
|
|
+ for (Tuple2<ServiceMetricMeta, EndpointMetrics> element : elements) {
|
|
|
+ endpointMetrics.add(element.f1);
|
|
|
+ }
|
|
|
+ EndpointMetrics agg = new EndpointMetrics(endpointMetrics);
|
|
|
+ out.collect(new Tuple2<>(new AppNameMetricMeta(s, context.window().getEnd()), agg));
|
|
|
+ }
|
|
|
+ });
|
|
|
+ sinkAppNameMetric(appNameMetricsStream, jdbcConnectionOptions, du15s);
|
|
|
|
|
|
- DataStream<ServiceMetric> serviceMetricDataStream20m = nextLevelServiceMetric(serviceMetricDataStream5m,
|
|
|
- jobConfig.getWatermarkIdlenessSeconds(), 20);
|
|
|
- sinkServiceMetric(serviceMetricDataStream20m, oasConfig.getSpanMetric(), Duration.ofMinutes(20));
|
|
|
|
|
|
- DataStream<ServiceMetric> serviceMetricDataStream1h = nextLevelServiceMetric(serviceMetricDataStream20m,
|
|
|
- jobConfig.getWatermarkIdlenessSeconds(), 60);
|
|
|
- sinkServiceMetric(serviceMetricDataStream1h, oasConfig.getSpanMetric(), Duration.ofHours(1));
|
|
|
+ DataStream<Tuple2<ServiceInstanceMeta, EndpointMetrics>> serviceInstanceOnlyStream = serviceMetricStreamLevel1
|
|
|
+ .flatMap((FlatMapFunction<ServiceMetric, Tuple2<ServiceInstanceMeta, EndpointMetrics>>) (value, out) -> {
|
|
|
+ for (ServiceInstanceMetric instanceMetric : value.getServiceInstanceMetrics()) {
|
|
|
+ out.collect(new Tuple2<>(new ServiceInstanceMeta(value.getAppName(), value.getTimestamp(), value.getServiceName(),
|
|
|
+ instanceMetric.getInstanceID()), instanceMetric.getInstanceMetrics(du15s)));
|
|
|
+ }
|
|
|
+ })
|
|
|
+ .name("serviceMetric2InstanceMeta+EndpointMetrics")
|
|
|
+ .returns(
|
|
|
+ TypeExtractor.getForObject(new Tuple2<>(new ServiceInstanceMeta(), new EndpointMetrics()))
|
|
|
+ );
|
|
|
+ sinkServiceInstanceOnlyMetric(serviceInstanceOnlyStream, jdbcConnectionOptions, du15s);
|
|
|
+ DataStream<Tuple2<EndpointMeta, EndpointMetrics>> endpointOnlyMetricStream = serviceMetricStreamLevel1
|
|
|
+ .flatMap((FlatMapFunction<ServiceMetric, Tuple2<EndpointMeta, EndpointMetrics>>) (value, out) -> {
|
|
|
+ for (ServiceInstanceMetric serviceInstanceMetric : value.getServiceInstanceMetrics()) {
|
|
|
+ for (EndpointMetric endpointMetric : serviceInstanceMetric.getEndpointMetrics()) {
|
|
|
+ out.collect(new Tuple2<>(
|
|
|
+ new EndpointMeta(value.getAppName(), value.getTimestamp(), value.getServiceName(), serviceInstanceMetric.getInstanceID(),
|
|
|
+ endpointMetric.getEndpoint()), endpointMetric.getMetrics(du15s)));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ })
|
|
|
+ .name("serviceMetric2EndpointMeta+EndpointMetrics")
|
|
|
+ .returns(
|
|
|
+ TypeExtractor.getForObject(new Tuple2<>(new EndpointMeta(), new EndpointMetrics()))
|
|
|
+ );
|
|
|
+ sinkEndpointOnlyMetric(endpointOnlyMetricStream, jdbcConnectionOptions, du15s);
|
|
|
+
|
|
|
+ List<Duration> granularity = List.of(Duration.ofMinutes(5), Duration.ofMinutes(20), Duration.ofMinutes(60));
|
|
|
+ DataStream<Tuple2<ServiceMetricMeta, EndpointMetrics>> preLevelServiceOnlyStream = serviceOnlyStream;
|
|
|
+ DataStream<Tuple2<ServiceInstanceMeta, EndpointMetrics>> preLevelServiceInstanceOnlyStream = serviceInstanceOnlyStream;
|
|
|
+ DataStream<Tuple2<EndpointMeta, EndpointMetrics>> preLevelEndpointOnlyMetricStream = endpointOnlyMetricStream;
|
|
|
+ for (Duration du : granularity) {
|
|
|
+ String duStr = StringUtils.getDurationString(du);
|
|
|
+ DataStream<Tuple2<ServiceMetricMeta, EndpointMetrics>> serviceMetricDataStreamDu = preLevelServiceOnlyStream
|
|
|
+ .keyBy((t) -> t.f0)
|
|
|
+ .window(TumblingEventTimeWindows.of(du))
|
|
|
+ .process(new ProcessWindowFunction<Tuple2<ServiceMetricMeta, EndpointMetrics>, Tuple2<ServiceMetricMeta, EndpointMetrics>, ServiceMetricMeta, TimeWindow>() {
|
|
|
+ @Override
|
|
|
+ public void process(ServiceMetricMeta serviceMetricMeta,
|
|
|
+ ProcessWindowFunction<Tuple2<ServiceMetricMeta, EndpointMetrics>, Tuple2<ServiceMetricMeta, EndpointMetrics>, ServiceMetricMeta, TimeWindow>.Context context,
|
|
|
+ Iterable<Tuple2<ServiceMetricMeta, EndpointMetrics>> elements, Collector<Tuple2<ServiceMetricMeta, EndpointMetrics>> out) throws Exception {
|
|
|
+ List<EndpointMetrics> endpointMetrics = new ArrayList<>();
|
|
|
+ for (Tuple2<ServiceMetricMeta, EndpointMetrics> element : elements) {
|
|
|
+ endpointMetrics.add(element.f1);
|
|
|
+ }
|
|
|
+ EndpointMetrics agg = new EndpointMetrics(endpointMetrics);
|
|
|
+ serviceMetricMeta.setTimestamp(context.window().getEnd());
|
|
|
+ out.collect(new Tuple2<>(serviceMetricMeta, agg));
|
|
|
+ }
|
|
|
+ })
|
|
|
+ .name(String.format("serviceMetric%s", duStr));
|
|
|
+ sinkServiceOnlyMetric(serviceMetricDataStreamDu, jdbcConnectionOptions, du);
|
|
|
+ preLevelServiceOnlyStream = serviceMetricDataStreamDu;
|
|
|
+ DataStream<Tuple2<ServiceInstanceMeta, EndpointMetrics>> serviceInstanceStreamDu = preLevelServiceInstanceOnlyStream
|
|
|
+ .keyBy((t) -> t.f0)
|
|
|
+ .window(TumblingEventTimeWindows.of(du))
|
|
|
+ .process(new ProcessWindowFunction<Tuple2<ServiceInstanceMeta, EndpointMetrics>, Tuple2<ServiceInstanceMeta, EndpointMetrics>, ServiceInstanceMeta, TimeWindow>() {
|
|
|
+ @Override
|
|
|
+ public void process(ServiceInstanceMeta serviceMetricMeta,
|
|
|
+ ProcessWindowFunction<Tuple2<ServiceInstanceMeta, EndpointMetrics>, Tuple2<ServiceInstanceMeta, EndpointMetrics>, ServiceInstanceMeta, TimeWindow>.Context context,
|
|
|
+ Iterable<Tuple2<ServiceInstanceMeta, EndpointMetrics>> elements, Collector<Tuple2<ServiceInstanceMeta, EndpointMetrics>> out) throws Exception {
|
|
|
+ List<EndpointMetrics> endpointMetrics = new ArrayList<>();
|
|
|
+ for (Tuple2<ServiceInstanceMeta, EndpointMetrics> element : elements) {
|
|
|
+ endpointMetrics.add(element.f1);
|
|
|
+ }
|
|
|
+ EndpointMetrics agg = new EndpointMetrics(endpointMetrics);
|
|
|
+ serviceMetricMeta.setTimestamp(context.window().getEnd());
|
|
|
+ out.collect(new Tuple2<>(serviceMetricMeta, agg));
|
|
|
+ }
|
|
|
+ })
|
|
|
+ .name(String.format("instanceMetric%s", duStr));
|
|
|
+ sinkServiceInstanceOnlyMetric(serviceInstanceStreamDu, jdbcConnectionOptions, du);
|
|
|
+ preLevelServiceInstanceOnlyStream = serviceInstanceStreamDu;
|
|
|
+ DataStream<Tuple2<EndpointMeta, EndpointMetrics>> endpointStreamDu = preLevelEndpointOnlyMetricStream
|
|
|
+ .keyBy((t) -> t.f0)
|
|
|
+ .window(TumblingEventTimeWindows.of(du))
|
|
|
+ .process(new ProcessWindowFunction<Tuple2<EndpointMeta, EndpointMetrics>, Tuple2<EndpointMeta, EndpointMetrics>, EndpointMeta, TimeWindow>() {
|
|
|
+ @Override
|
|
|
+ public void process(EndpointMeta serviceMetricMeta,
|
|
|
+ ProcessWindowFunction<Tuple2<EndpointMeta, EndpointMetrics>, Tuple2<EndpointMeta, EndpointMetrics>, EndpointMeta, TimeWindow>.Context context,
|
|
|
+ Iterable<Tuple2<EndpointMeta, EndpointMetrics>> elements, Collector<Tuple2<EndpointMeta, EndpointMetrics>> out) throws Exception {
|
|
|
+ List<EndpointMetrics> endpointMetrics = new ArrayList<>();
|
|
|
+ for (Tuple2<EndpointMeta, EndpointMetrics> element : elements) {
|
|
|
+ endpointMetrics.add(element.f1);
|
|
|
+ }
|
|
|
+ EndpointMetrics agg = new EndpointMetrics(endpointMetrics);
|
|
|
+ serviceMetricMeta.setTimestamp(context.window().getEnd());
|
|
|
+ out.collect(new Tuple2<>(serviceMetricMeta, agg));
|
|
|
+ }
|
|
|
+ })
|
|
|
+ .name(String.format("endpointMetric%s", duStr));
|
|
|
+ sinkEndpointOnlyMetric(endpointStreamDu, jdbcConnectionOptions, du);
|
|
|
+ preLevelEndpointOnlyMetricStream = endpointStreamDu;
|
|
|
+ }
|
|
|
|
|
|
// // traceURL stream
|
|
|
// handelTraceURLStream(oasConfig.getTraceURLConfig(), baseStream);
|
|
@@ -230,9 +225,66 @@ public class Main {
|
|
|
env.execute(oasConfig.getJobConfig().getJobName());
|
|
|
}
|
|
|
|
|
|
- private static void sinkServiceMetric(DataStream<ServiceMetric> serviceMetricDataStream, OasConfig.SpanMetricConfig pgConfig, Duration du) {
|
|
|
- String tableSuffix = du.toHours() >= 1 ? String.format("%dh", du.toHours()) :
|
|
|
- (du.toMinutes() >= 1 ? String.format("%dm", du.toMinutes()) : String.format("%ds", du.getSeconds()));
|
|
|
+ private static void sinkAppNameMetric(DataStream<Tuple2<AppNameMetricMeta, EndpointMetrics>> serviceMetricDataStream,
|
|
|
+ JdbcConnectionOptions jdbcConnectionOptions, Duration du) {
|
|
|
+ String tableSuffix = StringUtils.getDurationString(du);
|
|
|
+ String serviceSql = String.format("insert into app_metrics_%s " +
|
|
|
+ "(timestamp, app_name, apdex, http_code_300_count, " +
|
|
|
+ "http_code_400_count, http_code_500_count, span_status_error_count, " +
|
|
|
+ "exception_count, max_latency, min_latency, avg_latency, request_count," +
|
|
|
+ " rpm, count_le_500, count_le_1000, count_le_5000, quantile_50, " +
|
|
|
+ "quantile_90, quantile_95, quantile_99) " +
|
|
|
+ "values (?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?)", tableSuffix);
|
|
|
+ serviceMetricDataStream
|
|
|
+ .addSink(JdbcSink.sink(
|
|
|
+ serviceSql,
|
|
|
+ (ps, t) -> {
|
|
|
+ AppNameMetricMeta meta = t.f0;
|
|
|
+ EndpointMetrics em = t.f1;
|
|
|
+ int httpCode300Count = 0;
|
|
|
+ int httpCode400Count = 0;
|
|
|
+ int httpCode500Count = 0;
|
|
|
+ for (Map.Entry<Long, Integer> entry : em.getHttpCode2Count().entrySet()) {
|
|
|
+ Long code = entry.getKey();
|
|
|
+ if (code >= 300 && code < 400) {
|
|
|
+ httpCode300Count += entry.getValue();
|
|
|
+ } else if (code >= 400 && code < 500) {
|
|
|
+ httpCode400Count += entry.getValue();
|
|
|
+ } else if (code >= 500 && code < 600) {
|
|
|
+ httpCode500Count += entry.getValue();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ int exceptionCount = 0;
|
|
|
+ for (Integer value : em.getExceptionType2Count().values()) {
|
|
|
+ exceptionCount += value;
|
|
|
+ }
|
|
|
+ int spanErrorCount = em.getSpanStatusCode2Count().getOrDefault(Status.StatusCode.STATUS_CODE_ERROR.getNumber(), 0);
|
|
|
+ ps.setTimestamp(1, new Timestamp(meta.getTimestamp()));
|
|
|
+ ps.setString(2, meta.getAppName());
|
|
|
+ ps.setDouble(3, em.getApdex());
|
|
|
+ ps.setInt(4, httpCode300Count);
|
|
|
+ ps.setInt(5, httpCode400Count);
|
|
|
+ ps.setInt(6, httpCode500Count);
|
|
|
+ ps.setInt(7, spanErrorCount);
|
|
|
+ ps.setInt(8, exceptionCount);
|
|
|
+ ps.setLong(9, em.getMaxLatencyMS());
|
|
|
+ ps.setLong(10, em.getMinLatencyMS());
|
|
|
+ ps.setLong(11, em.getAvgLatencyMS());
|
|
|
+ ps.setLong(12, em.getRequestCount());
|
|
|
+ ps.setLong(13, em.getRPM());
|
|
|
+ ps.setLong(14, em.getLatencyLe500());
|
|
|
+ ps.setLong(15, em.getLatencyLe1000());
|
|
|
+ ps.setLong(16, em.getLatencyLe5000());
|
|
|
+ ps.setDouble(17, em.getQuantileLatencyMS50());
|
|
|
+ ps.setDouble(18, em.getQuantileLatencyMS90());
|
|
|
+ ps.setDouble(19, em.getQuantileLatencyMS95());
|
|
|
+ ps.setDouble(20, em.getQuantileLatencyMS99());
|
|
|
+ },
|
|
|
+ jdbcConnectionOptions));
|
|
|
+ }
|
|
|
+
|
|
|
+ private static void sinkServiceOnlyMetric(DataStream<Tuple2<ServiceMetricMeta, EndpointMetrics>> serviceMetricDataStream, JdbcConnectionOptions jdbcConnectionOptions, Duration du) {
|
|
|
+ String tableSuffix = StringUtils.getDurationString(du);
|
|
|
String serviceSql = String.format("insert into service_metrics_%s " +
|
|
|
"(timestamp, app_name, service_name, apdex, http_code_300_count, " +
|
|
|
"http_code_400_count, http_code_500_count, span_status_error_count, " +
|
|
@@ -240,18 +292,7 @@ public class Main {
|
|
|
" rpm, count_le_500, count_le_1000, count_le_5000, quantile_50, " +
|
|
|
"quantile_90, quantile_95, quantile_99) " +
|
|
|
"values (?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?)", tableSuffix);
|
|
|
- JdbcConnectionOptions jdbcConnectionOptions = new JdbcConnectionOptions.JdbcConnectionOptionsBuilder()
|
|
|
- .withUrl(pgConfig.getJdbcUrl())
|
|
|
- .withDriverName("org.postgresql.Driver")
|
|
|
- .withUsername(pgConfig.getPgUser())
|
|
|
- .withPassword(pgConfig.getPgPwd())
|
|
|
- .build();
|
|
|
serviceMetricDataStream
|
|
|
- .map((MapFunction<ServiceMetric, Tuple2<ServiceMetricMeta, EndpointMetrics>>) (v) -> new Tuple2<>(
|
|
|
- new ServiceMetricMeta(v.getAppName(), v.getTimestamp(), v.getServiceName()), v.getServerMetrics(du)))
|
|
|
- .returns(
|
|
|
- TypeExtractor.getForObject(new Tuple2<>(new ServiceMetricMeta(), new EndpointMetrics()))
|
|
|
- )
|
|
|
.addSink(JdbcSink.sink(
|
|
|
serviceSql,
|
|
|
(ps, t) -> {
|
|
@@ -298,26 +339,22 @@ public class Main {
|
|
|
ps.setDouble(21, em.getQuantileLatencyMS99());
|
|
|
},
|
|
|
jdbcConnectionOptions));
|
|
|
- String instanceSql = String.format("insert into service_instance_metrics_%s " +
|
|
|
+ }
|
|
|
+
|
|
|
+ private static void sinkEndpointOnlyMetric(DataStream<Tuple2<EndpointMeta, EndpointMetrics>> endpointOnlyMetricStream, JdbcConnectionOptions jdbcConnectionOptions, Duration du15s) {
|
|
|
+ String tableSuffix = StringUtils.getDurationString(du15s);
|
|
|
+ String endpointSql = String.format("insert into endpoint_metrics_%s " +
|
|
|
"(timestamp, app_name, service_name, apdex, http_code_300_count, " +
|
|
|
"http_code_400_count, http_code_500_count, span_status_error_count, " +
|
|
|
"exception_count, max_latency, min_latency, avg_latency, request_count," +
|
|
|
" rpm, count_le_500, count_le_1000, count_le_5000, quantile_50, " +
|
|
|
- "quantile_90, quantile_95, quantile_99, instanceID) " +
|
|
|
- "values (?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?)", tableSuffix);
|
|
|
- serviceMetricDataStream.flatMap((FlatMapFunction<ServiceMetric, Tuple2<ServiceInstanceMeta, EndpointMetrics>>) (value, out) -> {
|
|
|
- for (ServiceInstanceMetric instanceMetric : value.getServiceInstanceMetrics()) {
|
|
|
- out.collect(new Tuple2<>(new ServiceInstanceMeta(value.getAppName(), value.getTimestamp(), value.getServiceName(),
|
|
|
- instanceMetric.getInstanceID()), instanceMetric.getInstanceMetrics(du)));
|
|
|
- }
|
|
|
- })
|
|
|
- .returns(
|
|
|
- TypeExtractor.getForObject(new Tuple2<>(new ServiceInstanceMeta(), new EndpointMetrics()))
|
|
|
- )
|
|
|
+ "quantile_90, quantile_95, quantile_99, instanceID, endpoint) " +
|
|
|
+ "values (?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?)", tableSuffix);
|
|
|
+ endpointOnlyMetricStream
|
|
|
.addSink(JdbcSink.sink(
|
|
|
- instanceSql,
|
|
|
+ endpointSql,
|
|
|
(ps, t) -> {
|
|
|
- ServiceInstanceMeta meta = t.f0;
|
|
|
+ EndpointMeta meta = t.f0;
|
|
|
EndpointMetrics em = t.f1;
|
|
|
int httpCode300Count = 0;
|
|
|
int httpCode400Count = 0;
|
|
@@ -360,32 +397,27 @@ public class Main {
|
|
|
ps.setDouble(20, em.getQuantileLatencyMS95());
|
|
|
ps.setDouble(21, em.getQuantileLatencyMS99());
|
|
|
ps.setString(22, meta.getInstanceId());
|
|
|
+ String endpoint = meta.getEndpoint();
|
|
|
+ ps.setString(23, endpoint.length() > 1024 ? endpoint.substring(0, 1024) : endpoint);
|
|
|
},
|
|
|
jdbcConnectionOptions
|
|
|
));
|
|
|
- String endpointSql = String.format("insert into endpoint_metrics_%s " +
|
|
|
+ }
|
|
|
+
|
|
|
+ private static void sinkServiceInstanceOnlyMetric(DataStream<Tuple2<ServiceInstanceMeta, EndpointMetrics>> serviceMetricDataStream, JdbcConnectionOptions jdbcConnectionOptions, Duration du) {
|
|
|
+ String tableSuffix = StringUtils.getDurationString(du);
|
|
|
+ String instanceSql = String.format("insert into service_instance_metrics_%s " +
|
|
|
"(timestamp, app_name, service_name, apdex, http_code_300_count, " +
|
|
|
"http_code_400_count, http_code_500_count, span_status_error_count, " +
|
|
|
"exception_count, max_latency, min_latency, avg_latency, request_count," +
|
|
|
" rpm, count_le_500, count_le_1000, count_le_5000, quantile_50, " +
|
|
|
- "quantile_90, quantile_95, quantile_99, instanceID, endpoint) " +
|
|
|
- "values (?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?)", tableSuffix);
|
|
|
- serviceMetricDataStream.flatMap((FlatMapFunction<ServiceMetric, Tuple2<EndpointMeta, EndpointMetrics>>) (value, out) -> {
|
|
|
- for (ServiceInstanceMetric serviceInstanceMetric : value.getServiceInstanceMetrics()) {
|
|
|
- for (EndpointMetric endpointMetric : serviceInstanceMetric.getEndpointMetrics()) {
|
|
|
- out.collect(new Tuple2<>(
|
|
|
- new EndpointMeta(value.getAppName(), value.getTimestamp(), value.getServiceName(), serviceInstanceMetric.getInstanceID(),
|
|
|
- endpointMetric.getEndpoint()), endpointMetric.getMetrics(du)));
|
|
|
- }
|
|
|
- }
|
|
|
- })
|
|
|
- .returns(
|
|
|
- TypeExtractor.getForObject(new Tuple2<>(new EndpointMeta(), new EndpointMetrics()))
|
|
|
- )
|
|
|
+ "quantile_90, quantile_95, quantile_99, instanceID) " +
|
|
|
+ "values (?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?)", tableSuffix);
|
|
|
+ serviceMetricDataStream
|
|
|
.addSink(JdbcSink.sink(
|
|
|
- endpointSql,
|
|
|
+ instanceSql,
|
|
|
(ps, t) -> {
|
|
|
- EndpointMeta meta = t.f0;
|
|
|
+ ServiceInstanceMeta meta = t.f0;
|
|
|
EndpointMetrics em = t.f1;
|
|
|
int httpCode300Count = 0;
|
|
|
int httpCode400Count = 0;
|
|
@@ -428,28 +460,11 @@ public class Main {
|
|
|
ps.setDouble(20, em.getQuantileLatencyMS95());
|
|
|
ps.setDouble(21, em.getQuantileLatencyMS99());
|
|
|
ps.setString(22, meta.getInstanceId());
|
|
|
- ps.setString(23, meta.getEndpoint());
|
|
|
},
|
|
|
jdbcConnectionOptions
|
|
|
));
|
|
|
}
|
|
|
|
|
|
- private static DataStream<ServiceMetric> nextLevelServiceMetric(DataStream<ServiceMetric> base, int idleness, int windowMinutes) {
|
|
|
- WatermarkStrategy<ServiceMetric> watermarkStrategyLevel2 = WatermarkStrategy
|
|
|
- .<ServiceMetric>forBoundedOutOfOrderness(Duration.ofSeconds(0))
|
|
|
- .withTimestampAssigner(
|
|
|
- (SerializableTimestampAssigner<ServiceMetric>) (element, recordTimestamp) -> element.getTimestamp()
|
|
|
- ).withIdleness(
|
|
|
- Duration.ofSeconds(idleness)
|
|
|
- );
|
|
|
-
|
|
|
- return base
|
|
|
- .assignTimestampsAndWatermarks(watermarkStrategyLevel2)
|
|
|
- .keyBy(ServiceMetric::getServiceName)
|
|
|
- .window(TumblingEventTimeWindows.of(Duration.ofMinutes(windowMinutes)))
|
|
|
- .aggregate(new ServiceMetricGranularityAggregator(), new ServiceMetricTimestampAdder());
|
|
|
-
|
|
|
- }
|
|
|
|
|
|
private static void handleSpanMetrics(OasConfig.SpanMetricConfig spanMetric, SingleOutputStreamOperator<PojoSpan> mainStreamOperator) {
|
|
|
if (!spanMetric.isEnabled()) {
|
|
@@ -477,92 +492,5 @@ public class Main {
|
|
|
//
|
|
|
}
|
|
|
|
|
|
- private static void handelTraceURLStream(OasConfig.TraceURLConfig traceURL, SingleOutputStreamOperator<PojoSpan> mainStreamOperator) {
|
|
|
- if (!traceURL.isEnabled()) {
|
|
|
- return;
|
|
|
- }
|
|
|
- final OutputTag<PojoSpan> traceURLOutputTag = new OutputTag<>("traceURL") {
|
|
|
- };
|
|
|
- mainStreamOperator.process(new PojoSpanOutputTagProcessFunc(Collections.singletonList(traceURLOutputTag)));
|
|
|
- String traceURLTableName = traceURL.getSinkTableName();
|
|
|
- Integer traceURLWriteBufSize = traceURL.getWriteBufSize();
|
|
|
- ClickHouseSink<TraceURL> traceURLClickHouseSink = (new ClickhouseSinkBuilder<TraceURL>()).
|
|
|
- setTableName(traceURLTableName).
|
|
|
- setWriteBufSize(traceURLWriteBufSize.toString())
|
|
|
- .setConverter(new TraceURLCHConverter())
|
|
|
- .build();
|
|
|
-
|
|
|
- final OutputTag<TraceURL> otURLMappingOutput = new OutputTag<>("otURLMapping") {
|
|
|
- };
|
|
|
- SingleOutputStreamOperator<TraceURL> traceURLSingleOutputStreamOperator = mainStreamOperator
|
|
|
- .getSideOutput(traceURLOutputTag)
|
|
|
- .filter(new SpanKindFilter(OtelConst.SpanKindServer))
|
|
|
- .name("filterSpanKindServer")
|
|
|
- .filter(new ScopeTypeFilter(SpanScopeType.SPAN_SCOPE_TYPE_HTTP_SERVER))
|
|
|
- .name("filterHTTPScope")
|
|
|
- .map(new PojoSpan2TraceURLMapper())
|
|
|
- .name("pojoSpan2TraceURL")
|
|
|
- .process(new ProcessFunction<>() {
|
|
|
- @Override
|
|
|
- public void processElement(TraceURL traceURL, ProcessFunction<TraceURL, TraceURL>.Context context, Collector<TraceURL> collector) {
|
|
|
- collector.collect(traceURL);
|
|
|
- context.output(otURLMappingOutput, traceURL);
|
|
|
- }
|
|
|
- });
|
|
|
- traceURLSingleOutputStreamOperator.windowAll(TumblingEventTimeWindows.of(Duration.ofSeconds(30)))
|
|
|
- .aggregate(new URLAggregator())
|
|
|
- .name("aggregateByASMR")
|
|
|
- .print();
|
|
|
- traceURLSingleOutputStreamOperator.addSink(traceURLClickHouseSink).name("Sink2Clickhouse");
|
|
|
- }
|
|
|
|
|
|
-// private static void handleI6000Stream(OasConfig oasConfig, SingleOutputStreamOperator<PojoSpan> baseStream) {
|
|
|
-// Properties i6000KafkaProducerSettings = new Properties();
|
|
|
-// i6000KafkaProducerSettings.put("batch.size", "0");
|
|
|
-// i6000KafkaProducerSettings.put("linger.ms", "0");
|
|
|
-// OasConfig.I6000Config i6000Config = oasConfig.getI6000Config();
|
|
|
-// OasConfig.JobConfig jobConfig = oasConfig.getJobConfig();
|
|
|
-// final OutputTag<PojoSpan> httpMetricOutput = new OutputTag<>("http-metric-stream") {
|
|
|
-// };
|
|
|
-// Set<String> i6000IncludeAppNameSet = new HashSet<>(i6000Config.getIncludeAppNames());
|
|
|
-// SingleOutputStreamOperator<PojoSpan> outputStreamOperator = baseStream
|
|
|
-// .process(new ProcessFunction<PojoSpan, PojoSpan>() {
|
|
|
-// @Override
|
|
|
-// public void processElement(PojoSpan value, ProcessFunction<PojoSpan, PojoSpan>.Context ctx, Collector<PojoSpan> out) {
|
|
|
-// out.collect(value);
|
|
|
-// if (!i6000IncludeAppNameSet.contains(value.AppName)) {
|
|
|
-// return;
|
|
|
-// }
|
|
|
-// if (value.isHttpServerScopeType()) {
|
|
|
-// ctx.output(httpMetricOutput, value);
|
|
|
-// }
|
|
|
-// }
|
|
|
-// })
|
|
|
-// .name("旁路输出httpMetric/topo&Filter")
|
|
|
-// .setParallelism(jobConfig.getProcessOutputTagParallelism());
|
|
|
-//
|
|
|
-// OasConfig.HttpMetricConfig httpMetricConfig = oasConfig.getHttpMetricConfig();
|
|
|
-// Integer i6000Metric2BytesParallelism = oasConfig.getJobConfig().getI6000Metric2BytesParallelism();
|
|
|
-//
|
|
|
-//
|
|
|
-// String kafkaBootstrapServers = i6000Config.getKafkaBootstrapServer();
|
|
|
-// KafkaSink<byte[]> i6000MetricSink = KafkaSink.<byte[]>builder()
|
|
|
-// .setBootstrapServers(kafkaBootstrapServers)
|
|
|
-// .setKafkaProducerConfig(i6000KafkaProducerSettings)
|
|
|
-// .setRecordSerializer(
|
|
|
-// KafkaRecordSerializationSchema.builder()
|
|
|
-// .setTopic(i6000Config.getKafkaTopicMetric())
|
|
|
-// .setValueSerializationSchema(new I6000MessageSerializer()).build()
|
|
|
-// )
|
|
|
-// .build();
|
|
|
-// outputStreamOperator.getSideOutput(httpMetricOutput)
|
|
|
-// .windowAll(TumblingEventTimeWindows.of(Duration.ofSeconds(httpMetricConfig.getWindowIntervalSeconds())))
|
|
|
-// .aggregate(new ServiceMetricAggregator(), new TimeWindowAllAdder<>())
|
|
|
-// .name("aggEndPointMetrics")
|
|
|
-// .flatMap(new Metric2BytesMapper(i6000Config.getMetricIntervalMinutes(), i6000Config.getAppName2CmdbID(),
|
|
|
-// i6000Config.getIncludeAppNames()))
|
|
|
-// .setParallelism(i6000Metric2BytesParallelism).name("i6000Metrics2Bytes")
|
|
|
-// .sinkTo(i6000MetricSink).setParallelism(jobConfig.getI6000MetricKafkaSinkParallelism()).name("i6000MetricKafka");
|
|
|
-// // todo 拆解i6000与metric生成
|
|
|
-// }
|
|
|
}
|