You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@skywalking.apache.org by GitBox <gi...@apache.org> on 2021/05/26 02:12:14 UTC

[GitHub] [skywalking-satellite] gxthrj opened a new pull request #42: feat: add prometheus fetcher

gxthrj opened a new pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r646264277



##########
File path: plugins/fetcher/prometheus/fetcher.go
##########
@@ -0,0 +1,146 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package prometheus
+
+import (
+	"context"
+	"time"
+
+	"github.com/apache/skywalking-satellite/internal/pkg/config"
+	"github.com/apache/skywalking-satellite/internal/pkg/log"
+	"github.com/apache/skywalking-satellite/internal/satellite/event"
+
+	promConfig "github.com/prometheus/prometheus/config"
+	"github.com/prometheus/prometheus/discovery"
+	"github.com/prometheus/prometheus/scrape"
+	yaml "gopkg.in/yaml.v3"
+	v1 "skywalking.apache.org/repo/goapi/satellite/data/v1"
+)
+
+const (
+	Name      = "prometheus-metrics-fetcher"
+	eventName = "prometheus-metrics-event"
+)
+
+type scrapeConfig struct {
+	JobName        string                   `yaml:"job_name" mapstructure:"job_name"`
+	ScrapeInterval time.Duration            `yaml:"scrape_interval,omitempty" mapstructure:"scrape_interval,omitempty"`
+	StaticConfigs  []map[string]interface{} `yaml:"static_configs" mapstructure:"static_configs"`
+	MetricsPath    string                   `yaml:"metrics_path,omitempty" mapstructure:"metrics_path,omitempty"`
+}
+
+// Fetcher is the struct for Prometheus fetcher
+type Fetcher struct {
+	config.CommonFields
+	// config is the top level configuratScrapeConfigsMapion of prometheus
+	ScrapeConfigsMap []*scrapeConfig `mapstructure:"scrape_configs" yaml:"scrape_configs"`
+
+	ScrapeConfigs []*promConfig.ScrapeConfig
+	// events
+	OutputEvents event.BatchEvents
+	// outputChannel
+	OutputChannel chan *v1.SniffData
+
+	cancelFunc context.CancelFunc
+}
+
+func (f *Fetcher) Name() string {
+	return Name
+}
+
+func (f *Fetcher) Description() string {
+	return "This is a fetcher for Skywalking prometheus metrics format, " +
+		"which will translate Prometheus metrics to Skywalking meter system."
+}
+
+func (f *Fetcher) DefaultConfig() string {
+	return `
+## some config here
+scrape_configs:
+ - job_name: 'prometheus'
+   metrics_path: '/metrics'
+   scrape_interval: 10s
+   static_configs:
+   - targets: ['127.0.0.1:2020']
+`
+}
+
+func (f *Fetcher) Prepare() {}
+
+func (f *Fetcher) Fetch() event.BatchEvents {

Review comment:
       I have modified the Fetcher interface
   1. remove the response, which is not needed any more.
   2. add `ctx` for Fetch(ctx) , which used in conjunction with shutdown(ctx).
   
   cc @EvanLjp 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] wu-sheng commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r642076357



##########
File path: internal/satellite/module/gatherer/api/config.go
##########
@@ -33,6 +33,5 @@ type GathererConfig struct {
 	ServerName     string        `mapstructure:"server_name"` // depends on which server
 
 	// FetcherGatherer
-	FetcherConfig plugin.Config `mapstructure:"fetcher"`        // fetcher plugin config
-	FetchInterval int           `mapstructure:"fetch_interval"` // fetch interval, the time unit is millisecond

Review comment:
       FYI @EvanLjp We discussed and decided how to fetch, and what is the period, should be determined by the fetcher itself. A global config and timer don't make sense.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] nic-chen commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
nic-chen commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r646081013



##########
File path: plugins/fetcher/prometheus/fetcher.go
##########
@@ -0,0 +1,146 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package prometheus
+
+import (
+	"context"
+	"time"
+
+	"github.com/apache/skywalking-satellite/internal/pkg/config"
+	"github.com/apache/skywalking-satellite/internal/pkg/log"
+	"github.com/apache/skywalking-satellite/internal/satellite/event"
+
+	promConfig "github.com/prometheus/prometheus/config"
+	"github.com/prometheus/prometheus/discovery"
+	"github.com/prometheus/prometheus/scrape"
+	yaml "gopkg.in/yaml.v3"
+	v1 "skywalking.apache.org/repo/goapi/satellite/data/v1"
+)
+
+const (
+	Name      = "prometheus-metrics-fetcher"
+	eventName = "prometheus-metrics-event"
+)
+
+type scrapeConfig struct {
+	JobName        string                   `yaml:"job_name" mapstructure:"job_name"`
+	ScrapeInterval time.Duration            `yaml:"scrape_interval,omitempty" mapstructure:"scrape_interval,omitempty"`
+	StaticConfigs  []map[string]interface{} `yaml:"static_configs" mapstructure:"static_configs"`
+	MetricsPath    string                   `yaml:"metrics_path,omitempty" mapstructure:"metrics_path,omitempty"`
+}
+
+// Fetcher is the struct for Prometheus fetcher
+type Fetcher struct {
+	config.CommonFields
+	// config is the top level configuratScrapeConfigsMapion of prometheus
+	ScrapeConfigsMap []*scrapeConfig `mapstructure:"scrape_configs" yaml:"scrape_configs"`
+
+	ScrapeConfigs []*promConfig.ScrapeConfig
+	// events
+	OutputEvents event.BatchEvents
+	// outputChannel
+	OutputChannel chan *v1.SniffData
+
+	cancelFunc context.CancelFunc
+}
+
+func (f *Fetcher) Name() string {
+	return Name
+}
+
+func (f *Fetcher) Description() string {
+	return "This is a fetcher for Skywalking prometheus metrics format, " +
+		"which will translate Prometheus metrics to Skywalking meter system."
+}
+
+func (f *Fetcher) DefaultConfig() string {
+	return `
+## some config here
+scrape_configs:
+ - job_name: 'prometheus'
+   metrics_path: '/metrics'
+   scrape_interval: 10s
+   static_configs:
+   - targets: ['127.0.0.1:2020']
+`
+}
+
+func (f *Fetcher) Prepare() {}
+
+func (f *Fetcher) Fetch() event.BatchEvents {
+	ctx, cancel := context.WithCancel(context.Background())
+	f.cancelFunc = cancel
+	// yaml
+	configDeclare := make(map[string]interface{})
+	configDeclare["scrape_configs"] = f.ScrapeConfigsMap
+	configBytes, err := yaml.Marshal(configDeclare)
+	if err != nil {
+		log.Logger.Fatal("prometheus fetcher configure failed", err.Error())

Review comment:
       should we return here? since it parse failed.

##########
File path: plugins/init.go
##########
@@ -20,7 +20,7 @@ package plugins
 import (
 	"github.com/apache/skywalking-satellite/plugins/client"
 	"github.com/apache/skywalking-satellite/plugins/fallbacker"
-	fetcher "github.com/apache/skywalking-satellite/plugins/fetcher/api"
+	fetcher2 "github.com/apache/skywalking-satellite/plugins/fetcher"

Review comment:
       please use a meaningful name…

##########
File path: plugins/fetcher/prometheus/fetcher.go
##########
@@ -0,0 +1,146 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package prometheus
+
+import (
+	"context"
+	"time"
+
+	"github.com/apache/skywalking-satellite/internal/pkg/config"
+	"github.com/apache/skywalking-satellite/internal/pkg/log"
+	"github.com/apache/skywalking-satellite/internal/satellite/event"
+
+	promConfig "github.com/prometheus/prometheus/config"
+	"github.com/prometheus/prometheus/discovery"
+	"github.com/prometheus/prometheus/scrape"
+	yaml "gopkg.in/yaml.v3"
+	v1 "skywalking.apache.org/repo/goapi/satellite/data/v1"
+)
+
+const (
+	Name      = "prometheus-metrics-fetcher"
+	eventName = "prometheus-metrics-event"
+)
+
+type scrapeConfig struct {
+	JobName        string                   `yaml:"job_name" mapstructure:"job_name"`
+	ScrapeInterval time.Duration            `yaml:"scrape_interval,omitempty" mapstructure:"scrape_interval,omitempty"`
+	StaticConfigs  []map[string]interface{} `yaml:"static_configs" mapstructure:"static_configs"`
+	MetricsPath    string                   `yaml:"metrics_path,omitempty" mapstructure:"metrics_path,omitempty"`
+}
+
+// Fetcher is the struct for Prometheus fetcher
+type Fetcher struct {
+	config.CommonFields
+	// config is the top level configuratScrapeConfigsMapion of prometheus
+	ScrapeConfigsMap []*scrapeConfig `mapstructure:"scrape_configs" yaml:"scrape_configs"`
+
+	ScrapeConfigs []*promConfig.ScrapeConfig
+	// events
+	OutputEvents event.BatchEvents
+	// outputChannel
+	OutputChannel chan *v1.SniffData
+
+	cancelFunc context.CancelFunc
+}
+
+func (f *Fetcher) Name() string {
+	return Name
+}
+
+func (f *Fetcher) Description() string {
+	return "This is a fetcher for Skywalking prometheus metrics format, " +
+		"which will translate Prometheus metrics to Skywalking meter system."
+}
+
+func (f *Fetcher) DefaultConfig() string {
+	return `
+## some config here
+scrape_configs:
+ - job_name: 'prometheus'
+   metrics_path: '/metrics'
+   scrape_interval: 10s
+   static_configs:
+   - targets: ['127.0.0.1:2020']
+`
+}
+
+func (f *Fetcher) Prepare() {}
+
+func (f *Fetcher) Fetch() event.BatchEvents {
+	ctx, cancel := context.WithCancel(context.Background())
+	f.cancelFunc = cancel
+	// yaml
+	configDeclare := make(map[string]interface{})
+	configDeclare["scrape_configs"] = f.ScrapeConfigsMap
+	configBytes, err := yaml.Marshal(configDeclare)
+	if err != nil {
+		log.Logger.Fatal("prometheus fetcher configure failed", err.Error())
+	}
+	log.Logger.Debug(string(configBytes))
+	configStruct, err := promConfig.Load(string(configBytes))
+	if err != nil {
+		log.Logger.Fatal("prometheus fetcher configure load failed", err.Error())
+	}
+	f.ScrapeConfigs = configStruct.ScrapeConfigs
+	return fetch(ctx, f.ScrapeConfigs, f.OutputChannel)
+}
+
+func (f *Fetcher) Channel() <-chan *v1.SniffData {
+	return f.OutputChannel
+}
+
+func (f *Fetcher) Shutdown(context.Context) error {
+	f.cancelFunc()
+	return nil
+}
+
+func fetch(ctx context.Context, scrapeConfigs []*promConfig.ScrapeConfig, outputChannel chan *v1.SniffData) event.BatchEvents {
+	// config of scraper
+	c := make(map[string]discovery.Configs)
+	for _, v := range scrapeConfigs {
+		c[v.JobName] = v.ServiceDiscoveryConfigs
+	}
+	// manager
+	manager := discovery.NewManager(ctx, nil)
+	if err := manager.ApplyConfig(c); err != nil {
+		log.Logger.Errorf("prometheus discovery config error %s", err.Error())
+	}
+	// manager start
+	go func() {
+		if err := manager.Run(); err != nil {
+			log.Logger.Errorf("Discovery manager run failed, error %s", err.Error())
+		}
+	}()
+	// queue store
+	qs := NewQueueStore(ctx, true, Name, outputChannel)
+	scrapeManager := scrape.NewManager(nil, qs)
+	qs.SetScrapeManager(scrapeManager)
+	cfg := &promConfig.Config{ScrapeConfigs: scrapeConfigs}
+	if err := scrapeManager.ApplyConfig(cfg); err != nil {
+		log.Logger.Errorf("scrape failed, error: %s", err.Error())

Review comment:
       why we don't need to return when error?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r646225281



##########
File path: plugins/fetcher/prometheus/fetcher.go
##########
@@ -0,0 +1,146 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package prometheus
+
+import (
+	"context"
+	"time"
+
+	"github.com/apache/skywalking-satellite/internal/pkg/config"
+	"github.com/apache/skywalking-satellite/internal/pkg/log"
+	"github.com/apache/skywalking-satellite/internal/satellite/event"
+
+	promConfig "github.com/prometheus/prometheus/config"
+	"github.com/prometheus/prometheus/discovery"
+	"github.com/prometheus/prometheus/scrape"
+	yaml "gopkg.in/yaml.v3"
+	v1 "skywalking.apache.org/repo/goapi/satellite/data/v1"
+)
+
+const (
+	Name      = "prometheus-metrics-fetcher"
+	eventName = "prometheus-metrics-event"
+)
+
+type scrapeConfig struct {
+	JobName        string                   `yaml:"job_name" mapstructure:"job_name"`
+	ScrapeInterval time.Duration            `yaml:"scrape_interval,omitempty" mapstructure:"scrape_interval,omitempty"`
+	StaticConfigs  []map[string]interface{} `yaml:"static_configs" mapstructure:"static_configs"`
+	MetricsPath    string                   `yaml:"metrics_path,omitempty" mapstructure:"metrics_path,omitempty"`
+}
+
+// Fetcher is the struct for Prometheus fetcher
+type Fetcher struct {
+	config.CommonFields
+	// config is the top level configuratScrapeConfigsMapion of prometheus
+	ScrapeConfigsMap []*scrapeConfig `mapstructure:"scrape_configs" yaml:"scrape_configs"`
+
+	ScrapeConfigs []*promConfig.ScrapeConfig
+	// events
+	OutputEvents event.BatchEvents
+	// outputChannel
+	OutputChannel chan *v1.SniffData
+
+	cancelFunc context.CancelFunc
+}
+
+func (f *Fetcher) Name() string {
+	return Name
+}
+
+func (f *Fetcher) Description() string {
+	return "This is a fetcher for Skywalking prometheus metrics format, " +
+		"which will translate Prometheus metrics to Skywalking meter system."
+}
+
+func (f *Fetcher) DefaultConfig() string {
+	return `
+## some config here
+scrape_configs:
+ - job_name: 'prometheus'
+   metrics_path: '/metrics'
+   scrape_interval: 10s
+   static_configs:
+   - targets: ['127.0.0.1:2020']
+`
+}
+
+func (f *Fetcher) Prepare() {}
+
+func (f *Fetcher) Fetch() event.BatchEvents {
+	ctx, cancel := context.WithCancel(context.Background())
+	f.cancelFunc = cancel
+	// yaml
+	configDeclare := make(map[string]interface{})
+	configDeclare["scrape_configs"] = f.ScrapeConfigsMap
+	configBytes, err := yaml.Marshal(configDeclare)
+	if err != nil {
+		log.Logger.Fatal("prometheus fetcher configure failed", err.Error())

Review comment:
       It is `Fatal` , which will exit the application.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r646261281



##########
File path: plugins/fetcher/prometheus/metric_family.go
##########
@@ -0,0 +1,380 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+package prometheus
+
+import (
+	"fmt"
+	"math"
+	"sort"
+	"strconv"
+	"strings"
+
+	"github.com/prometheus/common/model"
+	v3 "skywalking.apache.org/repo/goapi/collect/language/agent/v3"
+
+	"github.com/prometheus/prometheus/pkg/labels"
+	"github.com/prometheus/prometheus/pkg/textparse"
+	"github.com/prometheus/prometheus/scrape"
+)
+
+type MetricFamily interface {
+	Add(metricName string, ls labels.Labels, t int64, v float64) error
+	IsSameFamily(metricName string) bool
+	// to OTLP metrics
+	// will return 1. metricspb.Metric with timeseries 2. counter all of timeseries 3. count dropped timeseries
+	ToMetric() []*v3.MeterData
+}
+
+type metricFamily struct {
+	name             string
+	mtype            textparse.MetricType
+	mc               MetadataCache
+	labelKeys        map[string]bool
+	labelKeysOrdered []string
+	metadata         *scrape.MetricMetadata
+	groupOrders      map[string]int
+	groups           map[string]*metricGroup
+}
+
+type metricGroup struct {
+	family       *metricFamily
+	name         string
+	ts           int64
+	ls           labels.Labels
+	hasCount     bool
+	count        float64
+	hasSum       bool
+	sum          float64
+	value        float64
+	complexValue []*dataPoint
+}
+
+type dataPoint struct {
+	value    float64
+	boundary float64
+}
+
+func normalizeMetricName(name string) string {
+	for _, s := range trimmableSuffixes {
+		if strings.HasSuffix(name, s) && name != s {
+			return strings.TrimSuffix(name, s)
+		}
+	}
+	return name
+}
+
+func newMetricFamily(metricName string, mc MetadataCache) MetricFamily {
+	familyName := normalizeMetricName(metricName)
+	// lookup metadata based on familyName
+	metadata, ok := mc.Metadata(familyName)
+	if !ok && metricName != familyName {
+		// use the original metricName as metricFamily
+		familyName = metricName
+		// perform a 2nd lookup with the original metric name. it can happen if there's a metric which is not histogram
+		// or summary, but ends with one of those _count/_sum suffixes
+		metadata, ok = mc.Metadata(metricName)
+		// still not found, this can happen when metric has no TYPE HINT
+		if !ok {
+			metadata.Metric = familyName
+			metadata.Type = textparse.MetricTypeUnknown
+		}
+	}
+
+	return &metricFamily{
+		name:             familyName,
+		mtype:            metadata.Type,
+		mc:               mc,
+		labelKeys:        make(map[string]bool),
+		labelKeysOrdered: make([]string, 0),
+		metadata:         &metadata,
+		groupOrders:      make(map[string]int),
+		groups:           make(map[string]*metricGroup),
+	}
+}
+
+func (mf *metricFamily) Add(metricName string, ls labels.Labels, t int64, v float64) error {
+	groupKey := mf.getGroupKey(ls)
+	mg := mf.loadMetricGroupOrCreate(groupKey, ls, t)
+	switch mf.mtype {
+	case textparse.MetricTypeCounter:
+		mg.value = v
+	case textparse.MetricTypeGauge:
+		mg.value = v
+	case textparse.MetricTypeHistogram:
+		if strings.HasSuffix(metricName, metricsSuffixCount) {
+			mg.hasCount = true
+			mg.count = v
+			mg.name = metricName
+		} else if strings.HasSuffix(metricName, metricsSuffixSum) {
+			mg.hasSum = true
+			mg.sum = v
+			mg.name = metricName
+		} else if strings.HasSuffix(metricName, metricsSuffixBucket) {
+			boundary, err := getBoundary(mf.mtype, ls)
+			if err != nil {
+				return err
+			}
+			mg.complexValue = append(mg.complexValue, &dataPoint{value: v, boundary: boundary})
+		}
+		mg.ts = t
+	case textparse.MetricTypeSummary:
+		if strings.HasSuffix(metricName, metricsSuffixCount) {
+			mg.hasCount = true
+			mg.count = v
+			mg.name = metricName
+		} else if strings.HasSuffix(metricName, metricsSuffixSum) {
+			mg.hasSum = true
+			mg.sum = v
+			mg.name = metricName
+		} else {
+			mg.value = v
+			mg.name = metricName
+		}
+		mg.ts = t
+	default:
+		mg.value = v
+		mg.name = metricName
+	}
+	return nil
+}
+
+func getBoundary(metricType textparse.MetricType, lbs labels.Labels) (float64, error) {
+	labelName := ""
+	switch metricType {
+	case textparse.MetricTypeHistogram:
+		labelName = model.BucketLabel
+	case textparse.MetricTypeSummary:
+		labelName = model.QuantileLabel
+	default:
+		return 0, fmt.Errorf("errNoBoundaryLabel")
+	}
+
+	v := lbs.Get(labelName)
+	if v == "" {
+		return 0, fmt.Errorf("errEmptyBoundaryLabel")
+	}
+
+	return strconv.ParseFloat(v, 64)
+}
+
+func (mf *metricFamily) toMeterSingleValue(mg *metricGroup) *v3.MeterSingleValue {
+	var result *v3.MeterSingleValue
+	if mg.hasCount {
+		result = &v3.MeterSingleValue{
+			Name:   mg.name,
+			Labels: mf.convertLabels(mg),
+			Value:  mg.count,
+		}
+	} else if mg.hasSum {
+		result = &v3.MeterSingleValue{
+			Name:   mg.name,
+			Labels: mf.convertLabels(mg),
+			Value:  mg.sum,
+		}
+	} else {
+		result = &v3.MeterSingleValue{
+			Name:   mg.name,
+			Labels: mf.convertLabels(mg),
+			Value:  mg.value,
+		}
+	}
+	return result

Review comment:
       done

##########
File path: plugins/fetcher/prometheus/metric_builder.go
##########
@@ -0,0 +1,122 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package prometheus
+
+import (
+	"fmt"
+	"strings"
+
+	v3 "skywalking.apache.org/repo/goapi/collect/language/agent/v3"
+
+	"github.com/prometheus/common/model"
+
+	"github.com/prometheus/prometheus/pkg/labels"

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r644461836



##########
File path: protocol/gen-codes/skywalking/network/go.mod
##########
@@ -4,6 +4,4 @@ go 1.15
 
 replace google.golang.org/grpc => github.com/grpc/grpc-go v1.36.1
 
-replace google.golang.org/protobuf => github.com/golang/protobuf v1.4.3
-

Review comment:
       recovered




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r644461836



##########
File path: protocol/gen-codes/skywalking/network/go.mod
##########
@@ -4,6 +4,4 @@ go 1.15
 
 replace google.golang.org/grpc => github.com/grpc/grpc-go v1.36.1
 
-replace google.golang.org/protobuf => github.com/golang/protobuf v1.4.3
-

Review comment:
       recovered




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] wu-sheng commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r646238980



##########
File path: plugins/fetcher/prometheus/fetcher.go
##########
@@ -0,0 +1,146 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package prometheus
+
+import (
+	"context"
+	"time"
+
+	"github.com/apache/skywalking-satellite/internal/pkg/config"
+	"github.com/apache/skywalking-satellite/internal/pkg/log"
+	"github.com/apache/skywalking-satellite/internal/satellite/event"
+
+	promConfig "github.com/prometheus/prometheus/config"
+	"github.com/prometheus/prometheus/discovery"
+	"github.com/prometheus/prometheus/scrape"
+	yaml "gopkg.in/yaml.v3"
+	v1 "skywalking.apache.org/repo/goapi/satellite/data/v1"
+)
+
+const (
+	Name      = "prometheus-metrics-fetcher"
+	eventName = "prometheus-metrics-event"
+)
+
+type scrapeConfig struct {
+	JobName        string                   `yaml:"job_name" mapstructure:"job_name"`
+	ScrapeInterval time.Duration            `yaml:"scrape_interval,omitempty" mapstructure:"scrape_interval,omitempty"`
+	StaticConfigs  []map[string]interface{} `yaml:"static_configs" mapstructure:"static_configs"`
+	MetricsPath    string                   `yaml:"metrics_path,omitempty" mapstructure:"metrics_path,omitempty"`
+}
+
+// Fetcher is the struct for Prometheus fetcher
+type Fetcher struct {
+	config.CommonFields
+	// config is the top level configuratScrapeConfigsMapion of prometheus
+	ScrapeConfigsMap []*scrapeConfig `mapstructure:"scrape_configs" yaml:"scrape_configs"`
+
+	ScrapeConfigs []*promConfig.ScrapeConfig
+	// events
+	OutputEvents event.BatchEvents
+	// outputChannel
+	OutputChannel chan *v1.SniffData
+
+	cancelFunc context.CancelFunc
+}
+
+func (f *Fetcher) Name() string {
+	return Name
+}
+
+func (f *Fetcher) Description() string {
+	return "This is a fetcher for Skywalking prometheus metrics format, " +
+		"which will translate Prometheus metrics to Skywalking meter system."
+}
+
+func (f *Fetcher) DefaultConfig() string {
+	return `
+## some config here
+scrape_configs:
+ - job_name: 'prometheus'
+   metrics_path: '/metrics'
+   scrape_interval: 10s
+   static_configs:
+   - targets: ['127.0.0.1:2020']
+`
+}
+
+func (f *Fetcher) Prepare() {}
+
+func (f *Fetcher) Fetch() event.BatchEvents {

Review comment:
       @gxthrj Should we remove this legacy API? @mrproliu This method was called in the core level timer, which has been removed. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] wu-sheng commented on pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#issuecomment-851322549


   Fetcher documentation should be updated from here https://github.com/apache/skywalking-satellite/blob/main/docs/en/setup/plugins/plugin-list.md


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#issuecomment-848647931


   > We need CI passed and UTs.
   
   Yes, I will continue to add code to this PR. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r645457718



##########
File path: plugins/fetcher/prometheus/metric_family.go
##########
@@ -0,0 +1,381 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+package prometheus
+
+import (
+	"fmt"
+	"math"
+	"sort"
+	"strconv"
+	"strings"
+
+	"github.com/prometheus/common/model"
+	v3 "skywalking.apache.org/repo/goapi/collect/language/agent/v3"
+
+	"github.com/prometheus/prometheus/pkg/labels"
+	"github.com/prometheus/prometheus/pkg/textparse"
+	"github.com/prometheus/prometheus/scrape"
+)
+
+type MetricFamily interface {
+	Add(metricName string, ls labels.Labels, t int64, v float64) error
+	IsSameFamily(metricName string) bool
+	// to OTLP metrics
+	// will return 1. metricspb.Metric with timeseries 2. counter all of timeseries 3. count dropped timeseries
+	ToMetric() []*v3.MeterData
+}
+
+type metricFamily struct {
+	name             string
+	mtype            textparse.MetricType
+	mc               MetadataCache
+	labelKeys        map[string]bool
+	labelKeysOrdered []string
+	metadata         *scrape.MetricMetadata
+	groupOrders      map[string]int
+	groups           map[string]*metricGroup
+}
+
+type metricGroup struct {
+	family       *metricFamily
+	name         string
+	ts           int64
+	ls           labels.Labels
+	hasCount     bool
+	count        float64
+	hasSum       bool
+	sum          float64
+	value        float64
+	complexValue []*dataPoint
+}
+
+type dataPoint struct {
+	value    float64
+	boundary float64
+}
+
+func normalizeMetricName(name string) string {
+	for _, s := range trimmableSuffixes {
+		if strings.HasSuffix(name, s) && name != s {
+			return strings.TrimSuffix(name, s)
+		}
+	}
+	return name
+}
+
+func newMetricFamily(metricName string, mc MetadataCache) MetricFamily {
+	familyName := normalizeMetricName(metricName)
+	// lookup metadata based on familyName
+	metadata, ok := mc.Metadata(familyName)
+	if !ok && metricName != familyName {
+		// use the original metricName as metricFamily
+		familyName = metricName
+		// perform a 2nd lookup with the original metric name. it can happen if there's a metric which is not histogram
+		// or summary, but ends with one of those _count/_sum suffixes
+		metadata, ok = mc.Metadata(metricName)
+		// still not found, this can happen when metric has no TYPE HINT
+		if !ok {
+			metadata.Metric = familyName
+			metadata.Type = textparse.MetricTypeUnknown
+		}
+	}
+
+	return &metricFamily{
+		name:             familyName,
+		mtype:            metadata.Type,
+		mc:               mc,
+		labelKeys:        make(map[string]bool),
+		labelKeysOrdered: make([]string, 0),
+		metadata:         &metadata,
+		groupOrders:      make(map[string]int),
+		groups:           make(map[string]*metricGroup),
+	}
+}
+
+func (mf *metricFamily) Add(metricName string, ls labels.Labels, t int64, v float64) error {
+	groupKey := mf.getGroupKey(ls)
+	mg := mf.loadMetricGroupOrCreate(groupKey, ls, t)
+	switch mf.mtype {
+	case textparse.MetricTypeCounter:
+		mg.value = v
+	case textparse.MetricTypeGauge:
+		mg.value = v
+	case textparse.MetricTypeHistogram:
+		if strings.HasSuffix(metricName, metricsSuffixCount) {
+			mg.hasCount = true
+			mg.count = v
+			mg.name = metricName
+		} else if strings.HasSuffix(metricName, metricsSuffixSum) {
+			mg.hasSum = true
+			mg.sum = v
+			mg.name = metricName
+		} else if strings.HasSuffix(metricName, metricsSuffixBucket) {
+			boundary, err := getBoundary(mf.mtype, ls)
+			if err != nil {
+				return err
+			}
+			mg.complexValue = append(mg.complexValue, &dataPoint{value: v, boundary: boundary})
+		}
+		mg.ts = t
+	case textparse.MetricTypeSummary:
+		if strings.HasSuffix(metricName, metricsSuffixCount) {
+			mg.hasCount = true
+			mg.count = v
+			mg.name = metricName
+		} else if strings.HasSuffix(metricName, metricsSuffixSum) {
+			mg.hasSum = true
+			mg.sum = v
+			mg.name = metricName
+		} else {
+			mg.value = v
+			mg.name = metricName
+		}
+		mg.ts = t
+	default:
+		mg.value = v
+		mg.name = metricName
+	}
+	return nil
+}
+
+func getBoundary(metricType textparse.MetricType, lbs labels.Labels) (float64, error) {
+	labelName := ""
+	switch metricType {
+	case textparse.MetricTypeHistogram:
+		labelName = model.BucketLabel
+	case textparse.MetricTypeSummary:
+		labelName = model.QuantileLabel
+	default:
+		return 0, fmt.Errorf("errNoBoundaryLabel")
+	}
+
+	v := lbs.Get(labelName)
+	if v == "" {
+		return 0, fmt.Errorf("errEmptyBoundaryLabel")
+	}
+
+	return strconv.ParseFloat(v, 64)
+}
+
+func (mf *metricFamily) toMeterSingleValue(mg *metricGroup) *v3.MeterSingleValue {
+	var result *v3.MeterSingleValue
+	if mg.hasCount {
+		result = &v3.MeterSingleValue{
+			Name:   mg.name,
+			Labels: mf.convertLabels(mg),
+			Value:  mg.count,
+		}
+	} else if mg.hasSum {
+		result = &v3.MeterSingleValue{
+			Name:   mg.name,
+			Labels: mf.convertLabels(mg),
+			Value:  mg.sum,
+		}
+	} else {
+		result = &v3.MeterSingleValue{
+			Name:   mg.name,
+			Labels: mf.convertLabels(mg),
+			Value:  mg.value,
+		}
+	}
+	return result
+}
+
+func (mf *metricFamily) ToMetric() []*v3.MeterData {
+	result := make([]*v3.MeterData, 0)
+	switch mf.mtype {
+	case textparse.MetricTypeSummary:
+		for _, mg := range mf.getGroups() {
+			msv := mf.toMeterSingleValue(mg)
+			result = append(result, &v3.MeterData{
+				Metric:    &v3.MeterData_SingleValue{SingleValue: msv},
+				Timestamp: mg.ts,
+			})
+		}
+	case textparse.MetricTypeHistogram:
+		for _, mg := range mf.getGroups() {
+			if mg.hasCount {
+				msv := &v3.MeterSingleValue{
+					Name:   mg.name,
+					Labels: mf.convertLabels(mg),
+					Value:  mg.count,
+				}
+				result = append(result, &v3.MeterData{
+					Metric:    &v3.MeterData_SingleValue{SingleValue: msv},
+					Timestamp: mg.ts,
+				})
+				continue
+			}
+			if mg.hasSum {
+				msv := &v3.MeterSingleValue{
+					Name:   mg.name,
+					Labels: mf.convertLabels(mg),
+					Value:  mg.sum,
+				}
+				result = append(result, &v3.MeterData{
+					Metric:    &v3.MeterData_SingleValue{SingleValue: msv},
+					Timestamp: mg.ts,
+				})
+				continue
+			}
+
+			bucketMap := make(map[float64]float64)
+			for _, dp := range mg.complexValue {
+				bucketMap[dp.boundary] = dp.value
+			}
+			sort.Slice(mg.complexValue, func(i, j int) bool {
+				return mg.complexValue[i].boundary < mg.complexValue[j].boundary
+			})
+			mbs := make([]*v3.MeterBucketValue, 0)
+			for index, m := range mg.complexValue {
+				if index == 0 {
+					mbv := &v3.MeterBucketValue{
+						Bucket: float64(math.MinInt64),
+						Count:  int64(m.value),
+					}
+					mbs = append(mbs, mbv)
+				} else {
+					mbv := &v3.MeterBucketValue{
+						Bucket: mg.complexValue[index-1].boundary,
+						Count:  int64(m.value),
+					}
+					mbs = append(mbs, mbv)
+				}
+			}
+			mh := &v3.MeterHistogram{
+				Name:   mf.name,
+				Labels: mf.convertLabels(mg),
+				Values: mbs,
+			}
+			result = append(result, &v3.MeterData{
+				Metric: &v3.MeterData_Histogram{
+					Histogram: mh,
+				},
+				Timestamp: mg.ts,
+			})
+		}
+	default:
+		for _, mg := range mf.getGroups() {
+			msv := &v3.MeterSingleValue{
+				Name:   mf.name,
+				Labels: mf.convertLabels(mg),
+				Value:  mg.value,
+			}
+			result = append(result, &v3.MeterData{
+				Metric: &v3.MeterData_SingleValue{SingleValue: msv},
+				// job, instance will be added in QueueAppender
+				Timestamp: mg.ts,
+			})
+		}
+	}
+	return result
+}
+
+func (mf *metricFamily) convertLabels(mg *metricGroup) []*v3.Label {
+	result := make([]*v3.Label, 0)
+	for k, v := range mg.ls.Map() {
+		label := &v3.Label{
+			Name:  k,
+			Value: v,
+		}
+		result = append(result, label)
+	}
+	return result
+}
+
+func (mf *metricFamily) getGroups() []*metricGroup {
+	groups := make([]*metricGroup, len(mf.groupOrders))
+	for k, v := range mf.groupOrders {
+		groups[v] = mf.groups[k]
+	}
+
+	return groups
+}
+
+func (mf *metricFamily) IsSameFamily(metricName string) bool {
+	// trim known suffix if necessary
+	familyName := normalizeMetricName(metricName)
+	return mf.name == familyName || familyName != metricName && mf.name == metricName
+}
+
+func (mf *metricFamily) getGroupKey(ls labels.Labels) string {
+	mf.updateLabelKeys(ls)
+	return dpgSignature(mf.labelKeysOrdered, ls)
+}
+
+func dpgSignature(orderedKnownLabelKeys []string, ls labels.Labels) string {
+	sign := make([]string, 0, len(orderedKnownLabelKeys))
+	for _, k := range orderedKnownLabelKeys {
+		v := ls.Get(k)
+		if v == "" {
+			continue
+		}
+		sign = append(sign, k+"="+v)
+	}
+	return fmt.Sprintf("%#v", sign)
+}
+
+func (mf *metricFamily) updateLabelKeys(ls labels.Labels) {
+	for _, l := range ls {
+		if isUsefulLabel(mf.mtype, l.Name) {
+			if _, ok := mf.labelKeys[l.Name]; !ok {
+				mf.labelKeys[l.Name] = true
+				// use insertion sort to maintain order
+				i := sort.SearchStrings(mf.labelKeysOrdered, l.Name)
+				labelKeys := append(mf.labelKeysOrdered, "")
+				copy(labelKeys[i+1:], labelKeys[i:])
+				labelKeys[i] = l.Name
+				mf.labelKeysOrdered = labelKeys
+			}
+		}
+	}
+}
+
+// todo

Review comment:
       removed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] mrproliu commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
mrproliu commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r646238683



##########
File path: plugins/fetcher/prometheus/fetcher.go
##########
@@ -0,0 +1,146 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package prometheus
+
+import (
+	"context"
+	"time"
+
+	"github.com/apache/skywalking-satellite/internal/pkg/config"
+	"github.com/apache/skywalking-satellite/internal/pkg/log"
+	"github.com/apache/skywalking-satellite/internal/satellite/event"
+
+	promConfig "github.com/prometheus/prometheus/config"
+	"github.com/prometheus/prometheus/discovery"
+	"github.com/prometheus/prometheus/scrape"
+	yaml "gopkg.in/yaml.v3"
+	v1 "skywalking.apache.org/repo/goapi/satellite/data/v1"
+)
+
+const (
+	Name      = "prometheus-metrics-fetcher"
+	eventName = "prometheus-metrics-event"
+)
+
+type scrapeConfig struct {
+	JobName        string                   `yaml:"job_name" mapstructure:"job_name"`
+	ScrapeInterval time.Duration            `yaml:"scrape_interval,omitempty" mapstructure:"scrape_interval,omitempty"`
+	StaticConfigs  []map[string]interface{} `yaml:"static_configs" mapstructure:"static_configs"`
+	MetricsPath    string                   `yaml:"metrics_path,omitempty" mapstructure:"metrics_path,omitempty"`
+}
+
+// Fetcher is the struct for Prometheus fetcher
+type Fetcher struct {
+	config.CommonFields
+	// config is the top level configuratScrapeConfigsMapion of prometheus
+	ScrapeConfigsMap []*scrapeConfig `mapstructure:"scrape_configs" yaml:"scrape_configs"`
+
+	ScrapeConfigs []*promConfig.ScrapeConfig
+	// events
+	OutputEvents event.BatchEvents
+	// outputChannel
+	OutputChannel chan *v1.SniffData
+
+	cancelFunc context.CancelFunc
+}
+
+func (f *Fetcher) Name() string {
+	return Name
+}
+
+func (f *Fetcher) Description() string {
+	return "This is a fetcher for Skywalking prometheus metrics format, " +
+		"which will translate Prometheus metrics to Skywalking meter system."
+}
+
+func (f *Fetcher) DefaultConfig() string {
+	return `
+## some config here
+scrape_configs:
+ - job_name: 'prometheus'
+   metrics_path: '/metrics'
+   scrape_interval: 10s
+   static_configs:
+   - targets: ['127.0.0.1:2020']
+`
+}
+
+func (f *Fetcher) Prepare() {}
+
+func (f *Fetcher) Fetch() event.BatchEvents {

Review comment:
       I can't see when this method will be called? Do I miss anything?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r642307953



##########
File path: LICENSE
##########
@@ -215,6 +215,7 @@ Apache 2.0 licenses
 The following components are provided under the Apache License. See project link for details.
 The text of each license is the standard Apache 2.0 license.
 
+prometheus(common) v0.15.0 https://github.com/prometheus/common Apache 2.0 licenses

Review comment:
       OK




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r646225606



##########
File path: plugins/init.go
##########
@@ -20,7 +20,7 @@ package plugins
 import (
 	"github.com/apache/skywalking-satellite/plugins/client"
 	"github.com/apache/skywalking-satellite/plugins/fallbacker"
-	fetcher "github.com/apache/skywalking-satellite/plugins/fetcher/api"
+	fetcher2 "github.com/apache/skywalking-satellite/plugins/fetcher"

Review comment:
       Done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] wu-sheng commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r644410395



##########
File path: go.sum
##########
@@ -1,60 +1,161 @@
 cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
 cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
 cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU=
+cloud.google.com/go v0.43.0/go.mod h1:BOSR3VbTLkk6FDC/TcffxP4NF/FFBGA5ku+jvKOP7pg=

Review comment:
       I am feeling you need `go mod tidy` for the go.sum. It keeps growing.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] kezhenxu94 commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r642732816



##########
File path: dist/licenses/LICENSE-prometheus-common.txt
##########
@@ -0,0 +1,201 @@
+                                 Apache License

Review comment:
       > Done, remove this line.
   
   Not remove this line, remove this file




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] kezhenxu94 commented on pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#issuecomment-855535339


   > But we need to make sure, which e2e framework is recommend? Are we going to reuse the e2e driver and validator of the main repo? @kezhenxu94
   
   Use the one in main repo for now.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] kezhenxu94 commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r642924144



##########
File path: dist/licenses/LICENSE-prometheus-common.txt
##########
@@ -0,0 +1,201 @@
+                                 Apache License

Review comment:
       > Is this file deleted because Apache 2.0 does not need to declare a license file?
   
   See my comment https://github.com/apache/skywalking-satellite/pull/42#discussion_r642734047
   
   Because it's standard Apache 2.0 license, we don't need to copy it again because we already have it in our root directory




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] wu-sheng commented on pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#issuecomment-855239971


   @kezhenxu94 @EvanLjp Please review.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] nic-chen commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
nic-chen commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r646242639



##########
File path: plugins/fetcher/prometheus/metric_builder.go
##########
@@ -0,0 +1,122 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package prometheus
+
+import (
+	"fmt"
+	"strings"
+
+	v3 "skywalking.apache.org/repo/goapi/collect/language/agent/v3"
+
+	"github.com/prometheus/common/model"
+
+	"github.com/prometheus/prometheus/pkg/labels"

Review comment:
       ```suggestion
   	"github.com/prometheus/common/model"
   	"github.com/prometheus/prometheus/pkg/labels"
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r645968664



##########
File path: plugins/fetcher/prometheus/queue_appender.go
##########
@@ -0,0 +1,117 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+package prometheus
+
+import (
+	"context"
+	"fmt"
+	"math"
+	"time"
+
+	"github.com/prometheus/common/model"
+	"github.com/prometheus/prometheus/pkg/labels"
+	"github.com/prometheus/prometheus/storage"
+	v1 "skywalking.apache.org/repo/goapi/satellite/data/v1"
+)
+
+// QueueAppender todo appender with queue
+type QueueAppender struct {
+	Ctx                context.Context
+	Ms                 *metadataService
+	isNew              bool
+	job                string
+	instance           string
+	metricBuilder      *metricBuilder
+	useStartTimeMetric bool
+	OutputChannel      chan *v1.SniffData
+}
+
+// NewQueueAppender construct QueueAppender
+func NewQueueAppender(ctx context.Context, ms *metadataService, oc chan *v1.SniffData, useStartTimeMetric bool) *QueueAppender {
+	return &QueueAppender{Ctx: ctx, Ms: ms, OutputChannel: oc, isNew: true, useStartTimeMetric: useStartTimeMetric}
+}
+
+func (qa *QueueAppender) initAppender(ls labels.Labels) error {
+	job, instance := ls.Get(model.JobLabel), ls.Get(model.InstanceLabel)
+	if job == "" || instance == "" {
+		// errNoJobInstance
+		return fmt.Errorf("errNoJobInstance")
+	}
+	// discover the binding target when this method is called for the first time during a transaction
+	mc, err := qa.Ms.Get(job, instance)
+	if err != nil {
+		return err
+	}
+	qa.job = job
+	qa.instance = instance
+	qa.metricBuilder = newMetricBuilder(mc, qa.useStartTimeMetric)
+	qa.isNew = false
+	return nil
+}
+
+var _ storage.Appender = (*QueueAppender)(nil)
+
+// always returns 0 to disable label caching

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] nic-chen commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
nic-chen commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r646244745



##########
File path: plugins/fetcher/prometheus/metadata.go
##########
@@ -0,0 +1,69 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package prometheus
+
+import (
+	"errors"
+
+	"github.com/prometheus/common/model"
+	"github.com/prometheus/prometheus/pkg/labels"
+	"github.com/prometheus/prometheus/scrape"
+)
+
+// MetadataCache is an adapter to prometheus' scrape.Target  and provide only the functionality which is needed
+type MetadataCache interface {
+	Metadata(metricName string) (scrape.MetricMetadata, bool)
+	SharedLabels() labels.Labels
+}
+
+type ScrapeManager interface {
+	TargetsAll() map[string][]*scrape.Target
+}
+
+type metadataService struct {
+	sm ScrapeManager
+}
+
+func (s *metadataService) Get(job, instance string) (MetadataCache, error) {
+	targetGroup, ok := s.sm.TargetsAll()[job]
+	if !ok {
+		return nil, errors.New("unable to find a target group with job=" + job)
+	}
+
+	// from the same targetGroup, instance is not going to be duplicated
+	for _, target := range targetGroup {
+		if target.Labels().Get(model.InstanceLabel) == instance {
+			return &mCache{target}, nil
+		}
+	}
+
+	return nil, errors.New("unable to find a target with job=" + job + ", and instance=" + instance)
+}
+
+// mCache as an adapter to get metadata from scrape.Target
+type mCache struct {
+	t *scrape.Target
+}
+
+func (m *mCache) Metadata(metricName string) (scrape.MetricMetadata, bool) {
+	return m.t.Metadata(metricName)

Review comment:
       may cause panic




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] nic-chen commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
nic-chen commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r646242434



##########
File path: plugins/fetcher/prometheus/fetcher_test.go
##########
@@ -0,0 +1,283 @@
+//
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+package prometheus
+
+import (
+	"context"
+	"fmt"
+	"net/http"
+	"net/http/httptest"
+	"net/url"
+	"reflect"
+	"strings"
+	"sync"
+	"sync/atomic"
+	"testing"
+	"time"
+
+	"github.com/apache/skywalking-satellite/internal/pkg/log"
+
+	"github.com/apache/skywalking-satellite/internal/pkg/plugin"

Review comment:
       ```suggestion
   	"github.com/apache/skywalking-satellite/internal/pkg/log"
   	"github.com/apache/skywalking-satellite/internal/pkg/plugin"
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r646258294



##########
File path: plugins/fetcher/prometheus/fetcher.go
##########
@@ -0,0 +1,146 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package prometheus
+
+import (
+	"context"
+	"time"
+
+	"github.com/apache/skywalking-satellite/internal/pkg/config"
+	"github.com/apache/skywalking-satellite/internal/pkg/log"
+	"github.com/apache/skywalking-satellite/internal/satellite/event"
+
+	promConfig "github.com/prometheus/prometheus/config"
+	"github.com/prometheus/prometheus/discovery"
+	"github.com/prometheus/prometheus/scrape"
+	yaml "gopkg.in/yaml.v3"
+	v1 "skywalking.apache.org/repo/goapi/satellite/data/v1"
+)
+
+const (
+	Name      = "prometheus-metrics-fetcher"
+	eventName = "prometheus-metrics-event"
+)
+
+type scrapeConfig struct {
+	JobName        string                   `yaml:"job_name" mapstructure:"job_name"`
+	ScrapeInterval time.Duration            `yaml:"scrape_interval,omitempty" mapstructure:"scrape_interval,omitempty"`
+	StaticConfigs  []map[string]interface{} `yaml:"static_configs" mapstructure:"static_configs"`
+	MetricsPath    string                   `yaml:"metrics_path,omitempty" mapstructure:"metrics_path,omitempty"`
+}
+
+// Fetcher is the struct for Prometheus fetcher
+type Fetcher struct {
+	config.CommonFields
+	// config is the top level configuratScrapeConfigsMapion of prometheus
+	ScrapeConfigsMap []*scrapeConfig `mapstructure:"scrape_configs" yaml:"scrape_configs"`
+
+	ScrapeConfigs []*promConfig.ScrapeConfig
+	// events
+	OutputEvents event.BatchEvents
+	// outputChannel
+	OutputChannel chan *v1.SniffData
+
+	cancelFunc context.CancelFunc
+}
+
+func (f *Fetcher) Name() string {
+	return Name
+}
+
+func (f *Fetcher) Description() string {
+	return "This is a fetcher for Skywalking prometheus metrics format, " +
+		"which will translate Prometheus metrics to Skywalking meter system."
+}
+
+func (f *Fetcher) DefaultConfig() string {
+	return `
+## some config here
+scrape_configs:
+ - job_name: 'prometheus'
+   metrics_path: '/metrics'
+   scrape_interval: 10s
+   static_configs:
+   - targets: ['127.0.0.1:2020']
+`
+}
+
+func (f *Fetcher) Prepare() {}
+
+func (f *Fetcher) Fetch() event.BatchEvents {

Review comment:
       Done.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] EvanLjp commented on pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
EvanLjp commented on pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#issuecomment-851359435


   ![image](https://user-images.githubusercontent.com/31562192/120173031-38173680-c236-11eb-9ebd-c5561cb765aa.png)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] wu-sheng commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r644409566



##########
File path: protocol/gen-codes/skywalking/network/go.mod
##########
@@ -4,6 +4,4 @@ go 1.15
 
 replace google.golang.org/grpc => github.com/grpc/grpc-go v1.36.1
 
-replace google.golang.org/protobuf => github.com/golang/protobuf v1.4.3
-

Review comment:
       All these replacements seems strange. I am not sure why we remove this.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r646226978



##########
File path: plugins/fetcher/prometheus/fetcher.go
##########
@@ -0,0 +1,146 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package prometheus
+
+import (
+	"context"
+	"time"
+
+	"github.com/apache/skywalking-satellite/internal/pkg/config"
+	"github.com/apache/skywalking-satellite/internal/pkg/log"
+	"github.com/apache/skywalking-satellite/internal/satellite/event"
+
+	promConfig "github.com/prometheus/prometheus/config"
+	"github.com/prometheus/prometheus/discovery"
+	"github.com/prometheus/prometheus/scrape"
+	yaml "gopkg.in/yaml.v3"
+	v1 "skywalking.apache.org/repo/goapi/satellite/data/v1"
+)
+
+const (
+	Name      = "prometheus-metrics-fetcher"
+	eventName = "prometheus-metrics-event"
+)
+
+type scrapeConfig struct {
+	JobName        string                   `yaml:"job_name" mapstructure:"job_name"`
+	ScrapeInterval time.Duration            `yaml:"scrape_interval,omitempty" mapstructure:"scrape_interval,omitempty"`
+	StaticConfigs  []map[string]interface{} `yaml:"static_configs" mapstructure:"static_configs"`
+	MetricsPath    string                   `yaml:"metrics_path,omitempty" mapstructure:"metrics_path,omitempty"`
+}
+
+// Fetcher is the struct for Prometheus fetcher
+type Fetcher struct {
+	config.CommonFields
+	// config is the top level configuratScrapeConfigsMapion of prometheus
+	ScrapeConfigsMap []*scrapeConfig `mapstructure:"scrape_configs" yaml:"scrape_configs"`
+
+	ScrapeConfigs []*promConfig.ScrapeConfig
+	// events
+	OutputEvents event.BatchEvents
+	// outputChannel
+	OutputChannel chan *v1.SniffData
+
+	cancelFunc context.CancelFunc
+}
+
+func (f *Fetcher) Name() string {
+	return Name
+}
+
+func (f *Fetcher) Description() string {
+	return "This is a fetcher for Skywalking prometheus metrics format, " +
+		"which will translate Prometheus metrics to Skywalking meter system."
+}
+
+func (f *Fetcher) DefaultConfig() string {
+	return `
+## some config here
+scrape_configs:
+ - job_name: 'prometheus'
+   metrics_path: '/metrics'
+   scrape_interval: 10s
+   static_configs:
+   - targets: ['127.0.0.1:2020']
+`
+}
+
+func (f *Fetcher) Prepare() {}
+
+func (f *Fetcher) Fetch() event.BatchEvents {
+	ctx, cancel := context.WithCancel(context.Background())
+	f.cancelFunc = cancel
+	// yaml
+	configDeclare := make(map[string]interface{})
+	configDeclare["scrape_configs"] = f.ScrapeConfigsMap
+	configBytes, err := yaml.Marshal(configDeclare)
+	if err != nil {
+		log.Logger.Fatal("prometheus fetcher configure failed", err.Error())
+	}
+	log.Logger.Debug(string(configBytes))
+	configStruct, err := promConfig.Load(string(configBytes))
+	if err != nil {
+		log.Logger.Fatal("prometheus fetcher configure load failed", err.Error())
+	}
+	f.ScrapeConfigs = configStruct.ScrapeConfigs
+	return fetch(ctx, f.ScrapeConfigs, f.OutputChannel)
+}
+
+func (f *Fetcher) Channel() <-chan *v1.SniffData {
+	return f.OutputChannel
+}
+
+func (f *Fetcher) Shutdown(context.Context) error {
+	f.cancelFunc()
+	return nil
+}
+
+func fetch(ctx context.Context, scrapeConfigs []*promConfig.ScrapeConfig, outputChannel chan *v1.SniffData) event.BatchEvents {
+	// config of scraper
+	c := make(map[string]discovery.Configs)
+	for _, v := range scrapeConfigs {
+		c[v.JobName] = v.ServiceDiscoveryConfigs
+	}
+	// manager
+	manager := discovery.NewManager(ctx, nil)
+	if err := manager.ApplyConfig(c); err != nil {
+		log.Logger.Errorf("prometheus discovery config error %s", err.Error())
+	}
+	// manager start
+	go func() {
+		if err := manager.Run(); err != nil {
+			log.Logger.Errorf("Discovery manager run failed, error %s", err.Error())
+		}
+	}()
+	// queue store
+	qs := NewQueueStore(ctx, true, Name, outputChannel)
+	scrapeManager := scrape.NewManager(nil, qs)
+	qs.SetScrapeManager(scrapeManager)
+	cfg := &promConfig.Config{ScrapeConfigs: scrapeConfigs}
+	if err := scrapeManager.ApplyConfig(cfg); err != nil {
+		log.Logger.Errorf("scrape failed, error: %s", err.Error())

Review comment:
       You are right, need to return here. but I think use log.fatal is better.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] kezhenxu94 commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r645427895



##########
File path: plugins/fetcher/prometheus/queue_appender.go
##########
@@ -0,0 +1,117 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+package prometheus
+
+import (
+	"context"
+	"fmt"
+	"math"
+	"time"
+
+	"github.com/prometheus/common/model"
+	"github.com/prometheus/prometheus/pkg/labels"
+	"github.com/prometheus/prometheus/storage"
+	v1 "skywalking.apache.org/repo/goapi/satellite/data/v1"
+)
+
+// QueueAppender todo appender with queue

Review comment:
       Is this `todo` to be done in the future?

##########
File path: plugins/fetcher/prometheus/queue_appender.go
##########
@@ -0,0 +1,117 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+package prometheus
+
+import (
+	"context"
+	"fmt"
+	"math"
+	"time"
+
+	"github.com/prometheus/common/model"
+	"github.com/prometheus/prometheus/pkg/labels"
+	"github.com/prometheus/prometheus/storage"
+	v1 "skywalking.apache.org/repo/goapi/satellite/data/v1"
+)
+
+// QueueAppender todo appender with queue
+type QueueAppender struct {
+	Ctx                context.Context
+	Ms                 *metadataService
+	isNew              bool
+	job                string
+	instance           string
+	metricBuilder      *metricBuilder
+	useStartTimeMetric bool
+	OutputChannel      chan *v1.SniffData
+}
+
+// NewQueueAppender construct QueueAppender
+func NewQueueAppender(ctx context.Context, ms *metadataService, oc chan *v1.SniffData, useStartTimeMetric bool) *QueueAppender {
+	return &QueueAppender{Ctx: ctx, Ms: ms, OutputChannel: oc, isNew: true, useStartTimeMetric: useStartTimeMetric}
+}
+
+func (qa *QueueAppender) initAppender(ls labels.Labels) error {
+	job, instance := ls.Get(model.JobLabel), ls.Get(model.InstanceLabel)
+	if job == "" || instance == "" {
+		// errNoJobInstance
+		return fmt.Errorf("errNoJobInstance")
+	}
+	// discover the binding target when this method is called for the first time during a transaction
+	mc, err := qa.Ms.Get(job, instance)
+	if err != nil {
+		return err
+	}
+	qa.job = job
+	qa.instance = instance
+	qa.metricBuilder = newMetricBuilder(mc, qa.useStartTimeMetric)
+	qa.isNew = false
+	return nil
+}
+
+var _ storage.Appender = (*QueueAppender)(nil)
+
+// always returns 0 to disable label caching

Review comment:
       Comments of public methods should start with the method name.
   The same for the other methods in this PR

##########
File path: plugins/fetcher/prometheus/metric_family.go
##########
@@ -0,0 +1,381 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+package prometheus
+
+import (
+	"fmt"
+	"math"
+	"sort"
+	"strconv"
+	"strings"
+
+	"github.com/prometheus/common/model"
+	v3 "skywalking.apache.org/repo/goapi/collect/language/agent/v3"
+
+	"github.com/prometheus/prometheus/pkg/labels"
+	"github.com/prometheus/prometheus/pkg/textparse"
+	"github.com/prometheus/prometheus/scrape"
+)
+
+type MetricFamily interface {
+	Add(metricName string, ls labels.Labels, t int64, v float64) error
+	IsSameFamily(metricName string) bool
+	// to OTLP metrics
+	// will return 1. metricspb.Metric with timeseries 2. counter all of timeseries 3. count dropped timeseries
+	ToMetric() []*v3.MeterData
+}
+
+type metricFamily struct {
+	name             string
+	mtype            textparse.MetricType
+	mc               MetadataCache
+	labelKeys        map[string]bool
+	labelKeysOrdered []string
+	metadata         *scrape.MetricMetadata
+	groupOrders      map[string]int
+	groups           map[string]*metricGroup
+}
+
+type metricGroup struct {
+	family       *metricFamily
+	name         string
+	ts           int64
+	ls           labels.Labels
+	hasCount     bool
+	count        float64
+	hasSum       bool
+	sum          float64
+	value        float64
+	complexValue []*dataPoint
+}
+
+type dataPoint struct {
+	value    float64
+	boundary float64
+}
+
+func normalizeMetricName(name string) string {
+	for _, s := range trimmableSuffixes {
+		if strings.HasSuffix(name, s) && name != s {
+			return strings.TrimSuffix(name, s)
+		}
+	}
+	return name
+}
+
+func newMetricFamily(metricName string, mc MetadataCache) MetricFamily {
+	familyName := normalizeMetricName(metricName)
+	// lookup metadata based on familyName
+	metadata, ok := mc.Metadata(familyName)
+	if !ok && metricName != familyName {
+		// use the original metricName as metricFamily
+		familyName = metricName
+		// perform a 2nd lookup with the original metric name. it can happen if there's a metric which is not histogram
+		// or summary, but ends with one of those _count/_sum suffixes
+		metadata, ok = mc.Metadata(metricName)
+		// still not found, this can happen when metric has no TYPE HINT
+		if !ok {
+			metadata.Metric = familyName
+			metadata.Type = textparse.MetricTypeUnknown
+		}
+	}
+
+	return &metricFamily{
+		name:             familyName,
+		mtype:            metadata.Type,
+		mc:               mc,
+		labelKeys:        make(map[string]bool),
+		labelKeysOrdered: make([]string, 0),
+		metadata:         &metadata,
+		groupOrders:      make(map[string]int),
+		groups:           make(map[string]*metricGroup),
+	}
+}
+
+func (mf *metricFamily) Add(metricName string, ls labels.Labels, t int64, v float64) error {
+	groupKey := mf.getGroupKey(ls)
+	mg := mf.loadMetricGroupOrCreate(groupKey, ls, t)
+	switch mf.mtype {
+	case textparse.MetricTypeCounter:
+		mg.value = v
+	case textparse.MetricTypeGauge:
+		mg.value = v
+	case textparse.MetricTypeHistogram:
+		if strings.HasSuffix(metricName, metricsSuffixCount) {
+			mg.hasCount = true
+			mg.count = v
+			mg.name = metricName
+		} else if strings.HasSuffix(metricName, metricsSuffixSum) {
+			mg.hasSum = true
+			mg.sum = v
+			mg.name = metricName
+		} else if strings.HasSuffix(metricName, metricsSuffixBucket) {
+			boundary, err := getBoundary(mf.mtype, ls)
+			if err != nil {
+				return err
+			}
+			mg.complexValue = append(mg.complexValue, &dataPoint{value: v, boundary: boundary})
+		}
+		mg.ts = t
+	case textparse.MetricTypeSummary:
+		if strings.HasSuffix(metricName, metricsSuffixCount) {
+			mg.hasCount = true
+			mg.count = v
+			mg.name = metricName
+		} else if strings.HasSuffix(metricName, metricsSuffixSum) {
+			mg.hasSum = true
+			mg.sum = v
+			mg.name = metricName
+		} else {
+			mg.value = v
+			mg.name = metricName
+		}
+		mg.ts = t
+	default:
+		mg.value = v
+		mg.name = metricName
+	}
+	return nil
+}
+
+func getBoundary(metricType textparse.MetricType, lbs labels.Labels) (float64, error) {
+	labelName := ""
+	switch metricType {
+	case textparse.MetricTypeHistogram:
+		labelName = model.BucketLabel
+	case textparse.MetricTypeSummary:
+		labelName = model.QuantileLabel
+	default:
+		return 0, fmt.Errorf("errNoBoundaryLabel")
+	}
+
+	v := lbs.Get(labelName)
+	if v == "" {
+		return 0, fmt.Errorf("errEmptyBoundaryLabel")
+	}
+
+	return strconv.ParseFloat(v, 64)
+}
+
+func (mf *metricFamily) toMeterSingleValue(mg *metricGroup) *v3.MeterSingleValue {
+	var result *v3.MeterSingleValue
+	if mg.hasCount {
+		result = &v3.MeterSingleValue{
+			Name:   mg.name,
+			Labels: mf.convertLabels(mg),
+			Value:  mg.count,
+		}
+	} else if mg.hasSum {
+		result = &v3.MeterSingleValue{
+			Name:   mg.name,
+			Labels: mf.convertLabels(mg),
+			Value:  mg.sum,
+		}
+	} else {
+		result = &v3.MeterSingleValue{
+			Name:   mg.name,
+			Labels: mf.convertLabels(mg),
+			Value:  mg.value,
+		}
+	}
+	return result
+}
+
+func (mf *metricFamily) ToMetric() []*v3.MeterData {
+	result := make([]*v3.MeterData, 0)
+	switch mf.mtype {
+	case textparse.MetricTypeSummary:
+		for _, mg := range mf.getGroups() {
+			msv := mf.toMeterSingleValue(mg)
+			result = append(result, &v3.MeterData{
+				Metric:    &v3.MeterData_SingleValue{SingleValue: msv},
+				Timestamp: mg.ts,
+			})
+		}
+	case textparse.MetricTypeHistogram:
+		for _, mg := range mf.getGroups() {
+			if mg.hasCount {
+				msv := &v3.MeterSingleValue{
+					Name:   mg.name,
+					Labels: mf.convertLabels(mg),
+					Value:  mg.count,
+				}
+				result = append(result, &v3.MeterData{
+					Metric:    &v3.MeterData_SingleValue{SingleValue: msv},
+					Timestamp: mg.ts,
+				})
+				continue
+			}
+			if mg.hasSum {
+				msv := &v3.MeterSingleValue{
+					Name:   mg.name,
+					Labels: mf.convertLabels(mg),
+					Value:  mg.sum,
+				}
+				result = append(result, &v3.MeterData{
+					Metric:    &v3.MeterData_SingleValue{SingleValue: msv},
+					Timestamp: mg.ts,
+				})
+				continue
+			}
+
+			bucketMap := make(map[float64]float64)
+			for _, dp := range mg.complexValue {
+				bucketMap[dp.boundary] = dp.value
+			}
+			sort.Slice(mg.complexValue, func(i, j int) bool {
+				return mg.complexValue[i].boundary < mg.complexValue[j].boundary
+			})
+			mbs := make([]*v3.MeterBucketValue, 0)
+			for index, m := range mg.complexValue {
+				if index == 0 {
+					mbv := &v3.MeterBucketValue{
+						Bucket: float64(math.MinInt64),
+						Count:  int64(m.value),
+					}
+					mbs = append(mbs, mbv)
+				} else {
+					mbv := &v3.MeterBucketValue{
+						Bucket: mg.complexValue[index-1].boundary,
+						Count:  int64(m.value),
+					}
+					mbs = append(mbs, mbv)
+				}
+			}
+			mh := &v3.MeterHistogram{
+				Name:   mf.name,
+				Labels: mf.convertLabels(mg),
+				Values: mbs,
+			}
+			result = append(result, &v3.MeterData{
+				Metric: &v3.MeterData_Histogram{
+					Histogram: mh,
+				},
+				Timestamp: mg.ts,
+			})
+		}
+	default:
+		for _, mg := range mf.getGroups() {
+			msv := &v3.MeterSingleValue{
+				Name:   mf.name,
+				Labels: mf.convertLabels(mg),
+				Value:  mg.value,
+			}
+			result = append(result, &v3.MeterData{
+				Metric: &v3.MeterData_SingleValue{SingleValue: msv},
+				// job, instance will be added in QueueAppender
+				Timestamp: mg.ts,
+			})
+		}
+	}
+	return result
+}
+
+func (mf *metricFamily) convertLabels(mg *metricGroup) []*v3.Label {
+	result := make([]*v3.Label, 0)
+	for k, v := range mg.ls.Map() {
+		label := &v3.Label{
+			Name:  k,
+			Value: v,
+		}
+		result = append(result, label)
+	}
+	return result
+}
+
+func (mf *metricFamily) getGroups() []*metricGroup {
+	groups := make([]*metricGroup, len(mf.groupOrders))
+	for k, v := range mf.groupOrders {
+		groups[v] = mf.groups[k]
+	}
+
+	return groups
+}
+
+func (mf *metricFamily) IsSameFamily(metricName string) bool {
+	// trim known suffix if necessary
+	familyName := normalizeMetricName(metricName)
+	return mf.name == familyName || familyName != metricName && mf.name == metricName
+}
+
+func (mf *metricFamily) getGroupKey(ls labels.Labels) string {
+	mf.updateLabelKeys(ls)
+	return dpgSignature(mf.labelKeysOrdered, ls)
+}
+
+func dpgSignature(orderedKnownLabelKeys []string, ls labels.Labels) string {
+	sign := make([]string, 0, len(orderedKnownLabelKeys))
+	for _, k := range orderedKnownLabelKeys {
+		v := ls.Get(k)
+		if v == "" {
+			continue
+		}
+		sign = append(sign, k+"="+v)
+	}
+	return fmt.Sprintf("%#v", sign)
+}
+
+func (mf *metricFamily) updateLabelKeys(ls labels.Labels) {
+	for _, l := range ls {
+		if isUsefulLabel(mf.mtype, l.Name) {
+			if _, ok := mf.labelKeys[l.Name]; !ok {
+				mf.labelKeys[l.Name] = true
+				// use insertion sort to maintain order
+				i := sort.SearchStrings(mf.labelKeysOrdered, l.Name)
+				labelKeys := append(mf.labelKeysOrdered, "")
+				copy(labelKeys[i+1:], labelKeys[i:])
+				labelKeys[i] = l.Name
+				mf.labelKeysOrdered = labelKeys
+			}
+		}
+	}
+}
+
+// todo

Review comment:
       Remove or finish the todo




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r646261153



##########
File path: plugins/fetcher/prometheus/metadata.go
##########
@@ -0,0 +1,69 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package prometheus
+
+import (
+	"errors"
+
+	"github.com/prometheus/common/model"
+	"github.com/prometheus/prometheus/pkg/labels"
+	"github.com/prometheus/prometheus/scrape"
+)
+
+// MetadataCache is an adapter to prometheus' scrape.Target  and provide only the functionality which is needed
+type MetadataCache interface {
+	Metadata(metricName string) (scrape.MetricMetadata, bool)
+	SharedLabels() labels.Labels
+}
+
+type ScrapeManager interface {
+	TargetsAll() map[string][]*scrape.Target
+}
+
+type metadataService struct {
+	sm ScrapeManager
+}
+
+func (s *metadataService) Get(job, instance string) (MetadataCache, error) {
+	targetGroup, ok := s.sm.TargetsAll()[job]
+	if !ok {
+		return nil, errors.New("unable to find a target group with job=" + job)
+	}
+
+	// from the same targetGroup, instance is not going to be duplicated
+	for _, target := range targetGroup {
+		if target.Labels().Get(model.InstanceLabel) == instance {
+			return &mCache{target}, nil
+		}
+	}
+
+	return nil, errors.New("unable to find a target with job=" + job + ", and instance=" + instance)
+}
+
+// mCache as an adapter to get metadata from scrape.Target
+type mCache struct {
+	t *scrape.Target
+}
+
+func (m *mCache) Metadata(metricName string) (scrape.MetricMetadata, bool) {
+	return m.t.Metadata(metricName)

Review comment:
       The `t` is set before (https://github.com/apache/skywalking-satellite/pull/42/files/0f01d525e7dc208fff2573ae171c330f84481175#diff-ca991bdd94c1589afcfcfdbe1bb4068b2ca7b3421363e8b53e92bfc477e66df8R51)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r642524834



##########
File path: dist/licenses/LICENSE-prometheus-common.txt
##########
@@ -0,0 +1,201 @@
+                                 Apache License

Review comment:
       Done, remove this line.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r642525331



##########
File path: go.mod
##########
@@ -5,17 +5,24 @@ go 1.14
 replace skywalking/network v1.0.0 => ./protocol/gen-codes/skywalking/network
 
 require (
+	github.com/Azure/go-autorest/autorest v0.11.18 // indirect
 	github.com/Shopify/sarama v1.27.2
 	github.com/enriquebris/goconcurrentqueue v0.6.0
 	github.com/golang/protobuf v1.5.2 // indirect
 	github.com/google/go-cmp v0.5.5
+	github.com/gophercloud/gophercloud v0.17.0 // indirect
 	github.com/grandecola/mmap v0.6.0
 	github.com/prometheus/client_golang v1.9.0
+	github.com/prometheus/common v0.15.0

Review comment:
       Done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] EvanLjp commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
EvanLjp commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r644433029



##########
File path: protocol/gen-codes/skywalking/network/go.mod
##########
@@ -4,6 +4,4 @@ go 1.15
 
 replace google.golang.org/grpc => github.com/grpc/grpc-go v1.36.1
 
-replace google.golang.org/protobuf => github.com/golang/protobuf v1.4.3
-

Review comment:
       The replace is useful when generate code with protoc




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r642900716



##########
File path: dist/licenses/LICENSE-prometheus-common.txt
##########
@@ -0,0 +1,201 @@
+                                 Apache License

Review comment:
       Is this file deleted because Apache 2.0 does not need to declare a license file?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] wu-sheng merged pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
wu-sheng merged pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] nic-chen commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
nic-chen commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r646244037



##########
File path: plugins/fetcher/prometheus/metric_family.go
##########
@@ -0,0 +1,380 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+package prometheus
+
+import (
+	"fmt"
+	"math"
+	"sort"
+	"strconv"
+	"strings"
+
+	"github.com/prometheus/common/model"
+	v3 "skywalking.apache.org/repo/goapi/collect/language/agent/v3"
+
+	"github.com/prometheus/prometheus/pkg/labels"
+	"github.com/prometheus/prometheus/pkg/textparse"
+	"github.com/prometheus/prometheus/scrape"
+)
+
+type MetricFamily interface {
+	Add(metricName string, ls labels.Labels, t int64, v float64) error
+	IsSameFamily(metricName string) bool
+	// to OTLP metrics
+	// will return 1. metricspb.Metric with timeseries 2. counter all of timeseries 3. count dropped timeseries
+	ToMetric() []*v3.MeterData
+}
+
+type metricFamily struct {
+	name             string
+	mtype            textparse.MetricType
+	mc               MetadataCache
+	labelKeys        map[string]bool
+	labelKeysOrdered []string
+	metadata         *scrape.MetricMetadata
+	groupOrders      map[string]int
+	groups           map[string]*metricGroup
+}
+
+type metricGroup struct {
+	family       *metricFamily
+	name         string
+	ts           int64
+	ls           labels.Labels
+	hasCount     bool
+	count        float64
+	hasSum       bool
+	sum          float64
+	value        float64
+	complexValue []*dataPoint
+}
+
+type dataPoint struct {
+	value    float64
+	boundary float64
+}
+
+func normalizeMetricName(name string) string {
+	for _, s := range trimmableSuffixes {
+		if strings.HasSuffix(name, s) && name != s {
+			return strings.TrimSuffix(name, s)
+		}
+	}
+	return name
+}
+
+func newMetricFamily(metricName string, mc MetadataCache) MetricFamily {
+	familyName := normalizeMetricName(metricName)
+	// lookup metadata based on familyName
+	metadata, ok := mc.Metadata(familyName)
+	if !ok && metricName != familyName {
+		// use the original metricName as metricFamily
+		familyName = metricName
+		// perform a 2nd lookup with the original metric name. it can happen if there's a metric which is not histogram
+		// or summary, but ends with one of those _count/_sum suffixes
+		metadata, ok = mc.Metadata(metricName)
+		// still not found, this can happen when metric has no TYPE HINT
+		if !ok {
+			metadata.Metric = familyName
+			metadata.Type = textparse.MetricTypeUnknown
+		}
+	}
+
+	return &metricFamily{
+		name:             familyName,
+		mtype:            metadata.Type,
+		mc:               mc,
+		labelKeys:        make(map[string]bool),
+		labelKeysOrdered: make([]string, 0),
+		metadata:         &metadata,
+		groupOrders:      make(map[string]int),
+		groups:           make(map[string]*metricGroup),
+	}
+}
+
+func (mf *metricFamily) Add(metricName string, ls labels.Labels, t int64, v float64) error {
+	groupKey := mf.getGroupKey(ls)
+	mg := mf.loadMetricGroupOrCreate(groupKey, ls, t)
+	switch mf.mtype {
+	case textparse.MetricTypeCounter:
+		mg.value = v
+	case textparse.MetricTypeGauge:
+		mg.value = v
+	case textparse.MetricTypeHistogram:
+		if strings.HasSuffix(metricName, metricsSuffixCount) {
+			mg.hasCount = true
+			mg.count = v
+			mg.name = metricName
+		} else if strings.HasSuffix(metricName, metricsSuffixSum) {
+			mg.hasSum = true
+			mg.sum = v
+			mg.name = metricName
+		} else if strings.HasSuffix(metricName, metricsSuffixBucket) {
+			boundary, err := getBoundary(mf.mtype, ls)
+			if err != nil {
+				return err
+			}
+			mg.complexValue = append(mg.complexValue, &dataPoint{value: v, boundary: boundary})
+		}
+		mg.ts = t
+	case textparse.MetricTypeSummary:
+		if strings.HasSuffix(metricName, metricsSuffixCount) {
+			mg.hasCount = true
+			mg.count = v
+			mg.name = metricName
+		} else if strings.HasSuffix(metricName, metricsSuffixSum) {
+			mg.hasSum = true
+			mg.sum = v
+			mg.name = metricName
+		} else {
+			mg.value = v
+			mg.name = metricName
+		}
+		mg.ts = t
+	default:
+		mg.value = v
+		mg.name = metricName
+	}
+	return nil
+}
+
+func getBoundary(metricType textparse.MetricType, lbs labels.Labels) (float64, error) {
+	labelName := ""
+	switch metricType {
+	case textparse.MetricTypeHistogram:
+		labelName = model.BucketLabel
+	case textparse.MetricTypeSummary:
+		labelName = model.QuantileLabel
+	default:
+		return 0, fmt.Errorf("errNoBoundaryLabel")
+	}
+
+	v := lbs.Get(labelName)
+	if v == "" {
+		return 0, fmt.Errorf("errEmptyBoundaryLabel")
+	}
+
+	return strconv.ParseFloat(v, 64)
+}
+
+func (mf *metricFamily) toMeterSingleValue(mg *metricGroup) *v3.MeterSingleValue {
+	var result *v3.MeterSingleValue
+	if mg.hasCount {
+		result = &v3.MeterSingleValue{
+			Name:   mg.name,
+			Labels: mf.convertLabels(mg),
+			Value:  mg.count,
+		}
+	} else if mg.hasSum {
+		result = &v3.MeterSingleValue{
+			Name:   mg.name,
+			Labels: mf.convertLabels(mg),
+			Value:  mg.sum,
+		}
+	} else {
+		result = &v3.MeterSingleValue{
+			Name:   mg.name,
+			Labels: mf.convertLabels(mg),
+			Value:  mg.value,
+		}
+	}
+	return result

Review comment:
       ```suggestion
   	if mg.hasCount {
   		return &v3.MeterSingleValue{
   			Name:   mg.name,
   			Labels: mf.convertLabels(mg),
   			Value:  mg.count,
   		}
   	}
   
       if mg.hasSum {
   		return &v3.MeterSingleValue{
   			Name:   mg.name,
   			Labels: mf.convertLabels(mg),
   			Value:  mg.sum,
   		}
   	} 
       
       return  &v3.MeterSingleValue{
           Name:   mg.name,
           Labels: mf.convertLabels(mg),
           Value:  mg.value,
       }
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] kezhenxu94 commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r644409207



##########
File path: dist/LICENSE
##########
@@ -241,4 +243,5 @@ The text of each license is also included at licenses/LICENSE-[project].txt.
     Shopify (sarama) v1.27.2: https://github.com/Shopify/sarama  MIT
     enriquebris (goconcurrentqueue) v0.6.0: https://github.com/enriquebris/goconcurrentqueue  MIT
     go.uber.org (automaxprocs) v1.4.0: https://go.uber.org/automaxprocs MIT
-
+    go.uber.org (zap) v1.13.0: https://github.com/uber-go/zap MIT

Review comment:
       Why do we want to have 2 logging frameworks?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] wu-sheng commented on pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#issuecomment-855534432


   > OK, I am going to do the e2e test about apisix-satellite-skywalking.
   
   Prometheus fetcher could work with SkyWalking in a separated e2e, APISIX as source is fine. But we need to make sure, which e2e framework is recommend? Are we going to reuse the e2e driver and validator of the main repo? @kezhenxu94 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r646258059



##########
File path: plugins/fetcher/prometheus/fetcher.go
##########
@@ -0,0 +1,146 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package prometheus
+
+import (
+	"context"
+	"time"
+
+	"github.com/apache/skywalking-satellite/internal/pkg/config"
+	"github.com/apache/skywalking-satellite/internal/pkg/log"
+	"github.com/apache/skywalking-satellite/internal/satellite/event"
+
+	promConfig "github.com/prometheus/prometheus/config"
+	"github.com/prometheus/prometheus/discovery"
+	"github.com/prometheus/prometheus/scrape"
+	yaml "gopkg.in/yaml.v3"
+	v1 "skywalking.apache.org/repo/goapi/satellite/data/v1"
+)
+
+const (
+	Name      = "prometheus-metrics-fetcher"
+	eventName = "prometheus-metrics-event"
+)
+
+type scrapeConfig struct {
+	JobName        string                   `yaml:"job_name" mapstructure:"job_name"`
+	ScrapeInterval time.Duration            `yaml:"scrape_interval,omitempty" mapstructure:"scrape_interval,omitempty"`
+	StaticConfigs  []map[string]interface{} `yaml:"static_configs" mapstructure:"static_configs"`
+	MetricsPath    string                   `yaml:"metrics_path,omitempty" mapstructure:"metrics_path,omitempty"`
+}
+
+// Fetcher is the struct for Prometheus fetcher
+type Fetcher struct {
+	config.CommonFields
+	// config is the top level configuratScrapeConfigsMapion of prometheus
+	ScrapeConfigsMap []*scrapeConfig `mapstructure:"scrape_configs" yaml:"scrape_configs"`
+
+	ScrapeConfigs []*promConfig.ScrapeConfig
+	// events
+	OutputEvents event.BatchEvents
+	// outputChannel
+	OutputChannel chan *v1.SniffData
+
+	cancelFunc context.CancelFunc
+}
+
+func (f *Fetcher) Name() string {
+	return Name
+}
+
+func (f *Fetcher) Description() string {
+	return "This is a fetcher for Skywalking prometheus metrics format, " +
+		"which will translate Prometheus metrics to Skywalking meter system."
+}
+
+func (f *Fetcher) DefaultConfig() string {
+	return `
+## some config here
+scrape_configs:
+ - job_name: 'prometheus'
+   metrics_path: '/metrics'
+   scrape_interval: 10s
+   static_configs:
+   - targets: ['127.0.0.1:2020']
+`
+}
+
+func (f *Fetcher) Prepare() {}
+
+func (f *Fetcher) Fetch() event.BatchEvents {

Review comment:
       There is an error in fetcher_gather.go, the fetch() call is missing. I will add it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] wu-sheng commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r644409566



##########
File path: protocol/gen-codes/skywalking/network/go.mod
##########
@@ -4,6 +4,4 @@ go 1.15
 
 replace google.golang.org/grpc => github.com/grpc/grpc-go v1.36.1
 
-replace google.golang.org/protobuf => github.com/golang/protobuf v1.4.3
-

Review comment:
       All these replacements seems strange. I am not sure why we remove this.

##########
File path: go.sum
##########
@@ -1,60 +1,161 @@
 cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
 cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
 cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU=
+cloud.google.com/go v0.43.0/go.mod h1:BOSR3VbTLkk6FDC/TcffxP4NF/FFBGA5ku+jvKOP7pg=

Review comment:
       I am feeling you need `go mod tidy` for the go.sum. It keeps growing.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] wu-sheng commented on pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#issuecomment-854642810


   Please resolve conflicts.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] kezhenxu94 commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r644409207



##########
File path: dist/LICENSE
##########
@@ -241,4 +243,5 @@ The text of each license is also included at licenses/LICENSE-[project].txt.
     Shopify (sarama) v1.27.2: https://github.com/Shopify/sarama  MIT
     enriquebris (goconcurrentqueue) v0.6.0: https://github.com/enriquebris/goconcurrentqueue  MIT
     go.uber.org (automaxprocs) v1.4.0: https://go.uber.org/automaxprocs MIT
-
+    go.uber.org (zap) v1.13.0: https://github.com/uber-go/zap MIT

Review comment:
       Why do we want to have 2 logging frameworks?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#issuecomment-850999250


   I will add some mock test .


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] kezhenxu94 commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r645440469



##########
File path: dist/LICENSE
##########
@@ -216,6 +216,8 @@ The following components are provided under the Apache License. See project link
 The text of each license is the standard Apache 2.0 license.
     client_golang (prometheus) v1.9.0: https://github.com/prometheus/client_golang Apache 2.0
     grpc-go (grpc) v1.36.1: https://github.com/grpc/grpc-go Apache 2.0
+    prometheus (prometheus) v1.8.2: http://github.com/prometheus/prometheus Apache 2.0
+    go-yaml (yaml) v2: https://github.com/go-yaml/yaml Apache 2.0

Review comment:
       Since https://github.com/apache/skywalking-satellite/pull/49 is also going to use yaml and it uses v3, let's use v3 for this as well, newer version should be more preferable, right?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] wu-sheng commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r642307770



##########
File path: LICENSE
##########
@@ -215,6 +215,7 @@ Apache 2.0 licenses
 The following components are provided under the Apache License. See project link for details.
 The text of each license is the standard Apache 2.0 license.
 
+prometheus(common) v0.15.0 https://github.com/prometheus/common Apache 2.0 licenses

Review comment:
       This is source code LICENSE, you should update https://github.com/apache/skywalking-satellite/blob/main/dist/LICENSE.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r643974311



##########
File path: go.mod
##########
@@ -5,17 +5,24 @@ go 1.14
 replace skywalking/network v1.0.0 => ./protocol/gen-codes/skywalking/network
 
 require (
+	github.com/Azure/go-autorest/autorest v0.11.18 // indirect
 	github.com/Shopify/sarama v1.27.2
 	github.com/enriquebris/goconcurrentqueue v0.6.0
 	github.com/golang/protobuf v1.5.2 // indirect
 	github.com/google/go-cmp v0.5.5
+	github.com/gophercloud/gophercloud v0.17.0 // indirect
 	github.com/grandecola/mmap v0.6.0
 	github.com/prometheus/client_golang v1.9.0
+	github.com/prometheus/common v0.15.0

Review comment:
       Done, thanks.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] wu-sheng commented on pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#issuecomment-848580076


   We need CI passed and UTs.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#issuecomment-855533659


   OK, I am going to do the e2e test about apisix-satellite-skywalking.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r646259371



##########
File path: plugins/fetcher/prometheus/fetcher_test.go
##########
@@ -0,0 +1,283 @@
+//
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+package prometheus
+
+import (
+	"context"
+	"fmt"
+	"net/http"
+	"net/http/httptest"
+	"net/url"
+	"reflect"
+	"strings"
+	"sync"
+	"sync/atomic"
+	"testing"
+	"time"
+
+	"github.com/apache/skywalking-satellite/internal/pkg/log"
+
+	"github.com/apache/skywalking-satellite/internal/pkg/plugin"

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r645458117



##########
File path: plugins/fetcher/prometheus/queue_appender.go
##########
@@ -0,0 +1,117 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+package prometheus
+
+import (
+	"context"
+	"fmt"
+	"math"
+	"time"
+
+	"github.com/prometheus/common/model"
+	"github.com/prometheus/prometheus/pkg/labels"
+	"github.com/prometheus/prometheus/storage"
+	v1 "skywalking.apache.org/repo/goapi/satellite/data/v1"
+)
+
+// QueueAppender todo appender with queue

Review comment:
       It has been done, the `todo` comment removed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r646261728



##########
File path: plugins/fetcher/prometheus/fetcher.go
##########
@@ -0,0 +1,146 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package prometheus
+
+import (
+	"context"
+	"time"
+
+	"github.com/apache/skywalking-satellite/internal/pkg/config"
+	"github.com/apache/skywalking-satellite/internal/pkg/log"
+	"github.com/apache/skywalking-satellite/internal/satellite/event"
+
+	promConfig "github.com/prometheus/prometheus/config"
+	"github.com/prometheus/prometheus/discovery"
+	"github.com/prometheus/prometheus/scrape"
+	yaml "gopkg.in/yaml.v3"
+	v1 "skywalking.apache.org/repo/goapi/satellite/data/v1"
+)
+
+const (
+	Name      = "prometheus-metrics-fetcher"
+	eventName = "prometheus-metrics-event"
+)
+
+type scrapeConfig struct {
+	JobName        string                   `yaml:"job_name" mapstructure:"job_name"`
+	ScrapeInterval time.Duration            `yaml:"scrape_interval,omitempty" mapstructure:"scrape_interval,omitempty"`
+	StaticConfigs  []map[string]interface{} `yaml:"static_configs" mapstructure:"static_configs"`
+	MetricsPath    string                   `yaml:"metrics_path,omitempty" mapstructure:"metrics_path,omitempty"`
+}
+
+// Fetcher is the struct for Prometheus fetcher
+type Fetcher struct {
+	config.CommonFields
+	// config is the top level configuratScrapeConfigsMapion of prometheus
+	ScrapeConfigsMap []*scrapeConfig `mapstructure:"scrape_configs" yaml:"scrape_configs"`
+
+	ScrapeConfigs []*promConfig.ScrapeConfig
+	// events
+	OutputEvents event.BatchEvents
+	// outputChannel
+	OutputChannel chan *v1.SniffData
+
+	cancelFunc context.CancelFunc
+}
+
+func (f *Fetcher) Name() string {
+	return Name
+}
+
+func (f *Fetcher) Description() string {
+	return "This is a fetcher for Skywalking prometheus metrics format, " +
+		"which will translate Prometheus metrics to Skywalking meter system."
+}
+
+func (f *Fetcher) DefaultConfig() string {
+	return `
+## some config here
+scrape_configs:
+ - job_name: 'prometheus'
+   metrics_path: '/metrics'
+   scrape_interval: 10s
+   static_configs:
+   - targets: ['127.0.0.1:2020']
+`
+}
+
+func (f *Fetcher) Prepare() {}
+
+func (f *Fetcher) Fetch() event.BatchEvents {

Review comment:
       > @gxthrj Should we remove this legacy API? @mrproliu This method was called in the core level timer, which has been removed.
   
   the Fech() is still need when booting.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] EvanLjp commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
EvanLjp commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r644433029



##########
File path: protocol/gen-codes/skywalking/network/go.mod
##########
@@ -4,6 +4,4 @@ go 1.15
 
 replace google.golang.org/grpc => github.com/grpc/grpc-go v1.36.1
 
-replace google.golang.org/protobuf => github.com/golang/protobuf v1.4.3
-

Review comment:
       The replace is useful when generate code with protoc




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj edited a comment on pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj edited a comment on pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#issuecomment-850999250


   I will add some mock tests .


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] wu-sheng commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r642312716



##########
File path: dist/licenses/LICENSE-prometheus-common.txt
##########
@@ -0,0 +1,201 @@
+                                 Apache License

Review comment:
       Remove this please.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] kezhenxu94 commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r642734047



##########
File path: go.mod
##########
@@ -5,17 +5,24 @@ go 1.14
 replace skywalking/network v1.0.0 => ./protocol/gen-codes/skywalking/network
 
 require (
+	github.com/Azure/go-autorest/autorest v0.11.18 // indirect
 	github.com/Shopify/sarama v1.27.2
 	github.com/enriquebris/goconcurrentqueue v0.6.0
 	github.com/golang/protobuf v1.5.2 // indirect
 	github.com/google/go-cmp v0.5.5
+	github.com/gophercloud/gophercloud v0.17.0 // indirect
 	github.com/grandecola/mmap v0.6.0
 	github.com/prometheus/client_golang v1.9.0
+	github.com/prometheus/common v0.15.0

Review comment:
       > Done
   
   You need to update https://github.com/apache/skywalking-satellite/blob/d174963d0a2484e259fab360cc7ef5de9c52f541/dist/LICENSE#L211-L218 , and you'll see "The text of each license is the standard Apache 2.0 license", for standard APL-2.0, you don't need to copy the license file.
   
   > The following components are provided under the Apache License. See project link for details. The text of each license is the standard Apache 2.0 license.
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] wu-sheng commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r642080223



##########
File path: go.mod
##########
@@ -5,17 +5,24 @@ go 1.14
 replace skywalking/network v1.0.0 => ./protocol/gen-codes/skywalking/network
 
 require (
+	github.com/Azure/go-autorest/autorest v0.11.18 // indirect
 	github.com/Shopify/sarama v1.27.2
 	github.com/enriquebris/goconcurrentqueue v0.6.0
 	github.com/golang/protobuf v1.5.2 // indirect
 	github.com/google/go-cmp v0.5.5
+	github.com/gophercloud/gophercloud v0.17.0 // indirect
 	github.com/grandecola/mmap v0.6.0
 	github.com/prometheus/client_golang v1.9.0
+	github.com/prometheus/common v0.15.0

Review comment:
       We need a LICENSE update about Prometheus.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r645410529



##########
File path: go.sum
##########
@@ -1,60 +1,161 @@
 cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
 cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
 cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU=
+cloud.google.com/go v0.43.0/go.mod h1:BOSR3VbTLkk6FDC/TcffxP4NF/FFBGA5ku+jvKOP7pg=

Review comment:
       I used `go mod tidy` , but nothing be reduced.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r643974581



##########
File path: dist/licenses/LICENSE-prometheus-common.txt
##########
@@ -0,0 +1,201 @@
+                                 Apache License

Review comment:
       Done, thanks.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r645409553



##########
File path: dist/LICENSE
##########
@@ -241,4 +243,5 @@ The text of each license is also included at licenses/LICENSE-[project].txt.
     Shopify (sarama) v1.27.2: https://github.com/Shopify/sarama  MIT
     enriquebris (goconcurrentqueue) v0.6.0: https://github.com/enriquebris/goconcurrentqueue  MIT
     go.uber.org (automaxprocs) v1.4.0: https://go.uber.org/automaxprocs MIT
-
+    go.uber.org (zap) v1.13.0: https://github.com/uber-go/zap MIT

Review comment:
       done, now only use logrus, zap has been removed.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-satellite] gxthrj commented on a change in pull request #42: feat: add prometheus fetcher

Posted by GitBox <gi...@apache.org>.
gxthrj commented on a change in pull request #42:
URL: https://github.com/apache/skywalking-satellite/pull/42#discussion_r645403193



##########
File path: internal/satellite/module/gatherer/api/config.go
##########
@@ -33,6 +33,5 @@ type GathererConfig struct {
 	ServerName     string        `mapstructure:"server_name"` // depends on which server
 
 	// FetcherGatherer
-	FetcherConfig plugin.Config `mapstructure:"fetcher"`        // fetcher plugin config
-	FetchInterval int           `mapstructure:"fetch_interval"` // fetch interval, the time unit is millisecond

Review comment:
       Yes, fetchInterval now is set in the implements of fetcher.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org