Support dotnet CLRMetric receive-forward. (#122)

diff --git a/CHANGES.md b/CHANGES.md
index 889f6c8..3af21df 100644
--- a/CHANGES.md
+++ b/CHANGES.md
@@ -9,6 +9,7 @@
 * Upgrade to GO 1.18.
 * Add Docker images for arm64 architecture.
 * Support transmit Span Attached Event protocol data.
+* Support dotnet CLRMetric forward.
 
 #### Bug Fixes
 * Fix the missing return data when receive metrics in batch mode.
diff --git a/configs/satellite_config.yaml b/configs/satellite_config.yaml
index 6952167..0eec3e6 100644
--- a/configs/satellite_config.yaml
+++ b/configs/satellite_config.yaml
@@ -306,6 +306,32 @@
       forwarders:
         - plugin_name: native-jvm-grpc-forwarder
   - common_config:
+      pipe_name: clrpipe
+    gatherer:
+      server_name: "grpc-server"
+      receiver:
+        plugin_name: "grpc-native-clr-receiver"
+      queue:
+        plugin_name: "memory-queue"
+        # The maximum buffer event size.
+        event_buffer_size: ${SATELLITE_QUEUE_EVENT_BUFFER_SIZE:5000}
+        # The partition count of queue.
+        partition: ${SATELLITE_QUEUE_PARTITION:4}
+    processor:
+      filters:
+    sender:
+      fallbacker:
+        plugin_name: none-fallbacker
+      # The time interval between two flush operations. And the time unit is millisecond.
+      flush_time: ${SATELLITE_CLRPIPE_SENDER_FLUSH_TIME:1000}
+      # The maximum buffer elements.
+      max_buffer_size: ${SATELLITE_CLRPIPE_SENDER_MAX_BUFFER_SIZE:200}
+      # The minimum flush elements.
+      min_flush_events: ${SATELLITE_CLRPIPE_SENDER_MIN_FLUSH_EVENTS:1}
+      client_name: grpc-client
+      forwarders:
+        - plugin_name: native-clr-grpc-forwarder
+  - common_config:
       pipe_name: meterpipe
     gatherer:
       server_name: "grpc-server"
diff --git a/docs/en/setup/plugins/forwarder_native-clr-grpc-forwarder.md b/docs/en/setup/plugins/forwarder_native-clr-grpc-forwarder.md
new file mode 100644
index 0000000..5acf585
--- /dev/null
+++ b/docs/en/setup/plugins/forwarder_native-clr-grpc-forwarder.md
@@ -0,0 +1,9 @@
+# Forwarder/native-clr-grpc-forwarder
+## Description
+This is a synchronization grpc forwarder with the SkyWalking native Configuration Discovery Service protocol.
+## DefaultConfig
+```yaml```
+## Configuration
+|Name|Type|Description|
+|----|----|-----------|
+
diff --git a/docs/en/setup/plugins/plugin-list.md b/docs/en/setup/plugins/plugin-list.md
index 4a9d231..1c2b47c 100755
--- a/docs/en/setup/plugins/plugin-list.md
+++ b/docs/en/setup/plugins/plugin-list.md
@@ -16,6 +16,7 @@
 	- [Native EBPF Profiling GRPC Forwarder](./forwarder_native-ebpf-profiling-grpc-forwarder.md)
 	- [Native Event GRPC Forwarder](./forwarder_native-event-grpc-forwarder.md)
 	- [Native JVM GRPC Forwarder](./forwarder_native-jvm-grpc-forwarder.md)
+	- [Native CLR GRPC Forwarder](./forwarder_native-clr-grpc-forwarder.md)
 	- [Native Log GRPC Forwarder](./forwarder_native-log-grpc-forwarder.md)
 	- [Native Log Kafka Forwarder](./forwarder_native-log-kafka-forwarder.md)
 	- [Native Management GRPC Forwarder](./forwarder_native-management-grpc-forwarder.md)
@@ -38,6 +39,7 @@
 	- [GRPC Native EBFP Profiling Receiver](./receiver_grpc-native-ebpf-profiling-receiver.md)
 	- [GRPC Native Event Receiver](./receiver_grpc-native-event-receiver.md)
 	- [GRPC Native JVM Receiver](./receiver_grpc-native-jvm-receiver.md)
+	- [GRPC Native CLR Receiver](./receiver_grpc-native-clr-receiver.md)
 	- [GRPC Native Log Receiver](./receiver_grpc-native-log-receiver.md)
 	- [GRPC Native Management Receiver](./receiver_grpc-native-management-receiver.md)
 	- [GRPC Native Meter Receiver](./receiver_grpc-native-meter-receiver.md)
diff --git a/docs/en/setup/plugins/receiver_grpc-native-clr-receiver.md b/docs/en/setup/plugins/receiver_grpc-native-clr-receiver.md
new file mode 100644
index 0000000..caf5249
--- /dev/null
+++ b/docs/en/setup/plugins/receiver_grpc-native-clr-receiver.md
@@ -0,0 +1,11 @@
+# Receiver/grpc-native-clr-receiver
+## Description
+This is a receiver for SkyWalking native clr format, which is defined at https://github.com/apache/skywalking-data-collect-protocol/blob/master/language-agent/CLRMetric.proto.
+## Support Forwarders
+ - [native-clr-grpc-forwarder](forwarder_native-clr-grpc-forwarder.md)
+## DefaultConfig
+```yaml```
+## Configuration
+|Name|Type|Description|
+|----|----|-----------|
+
diff --git a/docs/menu.yml b/docs/menu.yml
index 28b65aa..917c858 100644
--- a/docs/menu.yml
+++ b/docs/menu.yml
@@ -99,6 +99,8 @@
                   path: /en/setup/plugins/forwarder_native-event-grpc-forwarder
                 - name: Native JVM GRPC Forwarder
                   path: /en/setup/plugins/forwarder_native-jvm-grpc-forwarder
+                - name: Native CLR GRPC Forwarder
+                  path: /en/setup/plugins/forwarder_native-clr-grpc-forwarder
                 - name: Native Log GRPC Forwarder
                   path: /en/setup/plugins/forwarder_native-log-grpc-forwarder
                 - name: Native Log Kafka Forwarder
@@ -141,6 +143,8 @@
                   path: /en/setup/plugins/receiver_grpc-native-event-receiver
                 - name: GRPC Native JVM Receiver
                   path: /en/setup/plugins/receiver_grpc-native-jvm-receiver
+                - name: GRPC Native CLR Receiver
+                  path: /en/setup/plugins/receiver_grpc-native-clr-receiver
                 - name: GRPC Native Log Receiver
                   path: /en/setup/plugins/receiver_grpc-native-log-receiver
                 - name: GRPC Native Management Receiver
diff --git a/plugins/forwarder/forwarder_repository.go b/plugins/forwarder/forwarder_repository.go
index 51a47bf..2579552 100644
--- a/plugins/forwarder/forwarder_repository.go
+++ b/plugins/forwarder/forwarder_repository.go
@@ -25,6 +25,7 @@
 	"github.com/apache/skywalking-satellite/plugins/forwarder/grpc/envoymetricsv2"
 	"github.com/apache/skywalking-satellite/plugins/forwarder/grpc/envoymetricsv3"
 	grpc_nativecds "github.com/apache/skywalking-satellite/plugins/forwarder/grpc/nativecds"
+	grpc_nativeclr "github.com/apache/skywalking-satellite/plugins/forwarder/grpc/nativeclr"
 	grpc_nativeebpfprofiling "github.com/apache/skywalking-satellite/plugins/forwarder/grpc/nativeebpfprofiling"
 	grpc_nativeevent "github.com/apache/skywalking-satellite/plugins/forwarder/grpc/nativeevent"
 	grpc_nativejvm "github.com/apache/skywalking-satellite/plugins/forwarder/grpc/nativejvm"
@@ -55,6 +56,7 @@
 		new(grpc_nativecds.Forwarder),
 		new(grpc_nativeevent.Forwarder),
 		new(grpc_nativejvm.Forwarder),
+		new(grpc_nativeclr.Forwarder),
 		new(grpc_nativeprocess.Forwarder),
 		new(grpc_nativeebpfprofiling.Forwarder),
 		new(envoyalsv2.Forwarder),
diff --git a/plugins/forwarder/grpc/nativeclr/sync_forwarder.go b/plugins/forwarder/grpc/nativeclr/sync_forwarder.go
new file mode 100644
index 0000000..c50f9f6
--- /dev/null
+++ b/plugins/forwarder/grpc/nativeclr/sync_forwarder.go
@@ -0,0 +1,92 @@
+// 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 nativeclr
+
+import (
+	"context"
+	"fmt"
+	"reflect"
+
+	"google.golang.org/grpc"
+
+	agent "skywalking.apache.org/repo/goapi/collect/language/agent/v3"
+	v1 "skywalking.apache.org/repo/goapi/satellite/data/v1"
+
+	"github.com/apache/skywalking-satellite/internal/pkg/config"
+	"github.com/apache/skywalking-satellite/internal/satellite/event"
+)
+
+const (
+	Name     = "native-clr-grpc-forwarder"
+	ShowName = "Native CLR GRPC Forwarder"
+)
+
+type Forwarder struct {
+	config.CommonFields
+
+	client agent.CLRMetricReportServiceClient
+}
+
+func (f *Forwarder) Name() string {
+	return Name
+}
+
+func (f *Forwarder) ShowName() string {
+	return ShowName
+}
+
+func (f *Forwarder) Description() string {
+	return "This is a synchronization grpc forwarder with the SkyWalking native Configuration Discovery Service protocol."
+}
+
+func (f *Forwarder) DefaultConfig() string {
+	return ``
+}
+
+func (f *Forwarder) Prepare(connection interface{}) error {
+	client, ok := connection.(*grpc.ClientConn)
+	if !ok {
+		return fmt.Errorf("the %s only accepts a grpc client, but received a %s",
+			f.Name(), reflect.TypeOf(connection).String())
+	}
+	f.client = agent.NewCLRMetricReportServiceClient(client)
+	return nil
+}
+
+func (f *Forwarder) Forward(batch event.BatchEvents) error {
+	for _, e := range batch {
+		clr := e.GetClr()
+		_, err := f.client.Collect(context.Background(), clr)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+func (f *Forwarder) ForwardType() v1.SniffType {
+	return v1.SniffType_CLRMetricType
+}
+
+func (f *Forwarder) SyncForward(_ *v1.SniffData) (*v1.SniffData, error) {
+	return nil, fmt.Errorf("unsupport sync forward")
+}
+
+func (f *Forwarder) SupportedSyncInvoke() bool {
+	return false
+}
diff --git a/plugins/receiver/grpc/nativeclr/clr_report_service.go b/plugins/receiver/grpc/nativeclr/clr_report_service.go
new file mode 100644
index 0000000..2909158
--- /dev/null
+++ b/plugins/receiver/grpc/nativeclr/clr_report_service.go
@@ -0,0 +1,49 @@
+// 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 nativeclr
+
+import (
+	"context"
+	"time"
+
+	common "skywalking.apache.org/repo/goapi/collect/common/v3"
+	agent "skywalking.apache.org/repo/goapi/collect/language/agent/v3"
+	v1 "skywalking.apache.org/repo/goapi/satellite/data/v1"
+)
+
+const eventName = "grpc-clr-event"
+
+type CLRReportService struct {
+	receiveChannel chan *v1.SniffData
+	agent.UnimplementedCLRMetricReportServiceServer
+}
+
+func (j *CLRReportService) Collect(_ context.Context, clr *agent.CLRMetricCollection) (*common.Commands, error) {
+	e := &v1.SniffData{
+		Name:      eventName,
+		Timestamp: time.Now().UnixNano() / 1e6,
+		Meta:      nil,
+		Type:      v1.SniffType_CLRMetricType,
+		Remote:    true,
+		Data: &v1.SniffData_Clr{
+			Clr: clr,
+		},
+	}
+	j.receiveChannel <- e
+	return &common.Commands{}, nil
+}
diff --git a/plugins/receiver/grpc/nativeclr/clr_report_service_compact.go b/plugins/receiver/grpc/nativeclr/clr_report_service_compact.go
new file mode 100644
index 0000000..2fccc8a
--- /dev/null
+++ b/plugins/receiver/grpc/nativeclr/clr_report_service_compact.go
@@ -0,0 +1,35 @@
+// 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 nativeclr
+
+import (
+	"context"
+
+	common "skywalking.apache.org/repo/goapi/collect/common/v3"
+	agent "skywalking.apache.org/repo/goapi/collect/language/agent/v3"
+	agent_compat "skywalking.apache.org/repo/goapi/collect/language/agent/v3/compat"
+)
+
+type CLRReportServiceCompat struct {
+	reportService *CLRReportService
+	agent_compat.UnimplementedCLRMetricReportServiceServer
+}
+
+func (j *CLRReportServiceCompat) Collect(ctx context.Context, clr *agent.CLRMetricCollection) (*common.Commands, error) {
+	return j.reportService.Collect(ctx, clr)
+}
diff --git a/plugins/receiver/grpc/nativeclr/receiver.go b/plugins/receiver/grpc/nativeclr/receiver.go
new file mode 100644
index 0000000..e8f0f98
--- /dev/null
+++ b/plugins/receiver/grpc/nativeclr/receiver.go
@@ -0,0 +1,78 @@
+// 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 nativeclr
+
+import (
+	agent "skywalking.apache.org/repo/goapi/collect/language/agent/v3"
+	agent_compat "skywalking.apache.org/repo/goapi/collect/language/agent/v3/compat"
+	v1 "skywalking.apache.org/repo/goapi/satellite/data/v1"
+
+	"github.com/apache/skywalking-satellite/internal/pkg/config"
+	module "github.com/apache/skywalking-satellite/internal/satellite/module/api"
+	forwarder "github.com/apache/skywalking-satellite/plugins/forwarder/api"
+	forwarder_nativeclr "github.com/apache/skywalking-satellite/plugins/forwarder/grpc/nativeclr"
+	grpcreceiver "github.com/apache/skywalking-satellite/plugins/receiver/grpc"
+)
+
+const (
+	Name     = "grpc-native-clr-receiver"
+	ShowName = "GRPC Native CLR Receiver"
+)
+
+type Receiver struct {
+	config.CommonFields
+	grpcreceiver.CommonGRPCReceiverFields
+	service *CLRReportService // The gRPC request handler for clr data.
+}
+
+func (r *Receiver) Name() string {
+	return Name
+}
+
+func (r *Receiver) ShowName() string {
+	return ShowName
+}
+
+func (r *Receiver) Description() string {
+	return "This is a receiver for SkyWalking native clr format, " +
+		"which is defined at https://github.com/apache/skywalking-data-collect-protocol/blob/master/language-agent/CLRMetric.proto."
+}
+
+func (r *Receiver) DefaultConfig() string {
+	return ""
+}
+
+func (r *Receiver) RegisterHandler(server interface{}) {
+	r.CommonGRPCReceiverFields = *grpcreceiver.InitCommonGRPCReceiverFields(server)
+	r.service = &CLRReportService{receiveChannel: r.OutputChannel}
+	agent.RegisterCLRMetricReportServiceServer(r.Server, r.service)
+	agent_compat.RegisterCLRMetricReportServiceServer(r.Server, &CLRReportServiceCompat{reportService: r.service})
+}
+
+func (r *Receiver) RegisterSyncInvoker(_ module.SyncInvoker) {
+}
+
+func (r *Receiver) Channel() <-chan *v1.SniffData {
+	return r.OutputChannel
+}
+
+func (r *Receiver) SupportForwarders() []forwarder.Forwarder {
+	return []forwarder.Forwarder{
+		new(forwarder_nativeclr.Forwarder),
+	}
+}
diff --git a/plugins/receiver/grpc/nativeclr/receiver_test.go b/plugins/receiver/grpc/nativeclr/receiver_test.go
new file mode 100644
index 0000000..dafdad2
--- /dev/null
+++ b/plugins/receiver/grpc/nativeclr/receiver_test.go
@@ -0,0 +1,74 @@
+// 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 nativeclr
+
+import (
+	"context"
+	"testing"
+	"time"
+
+	"google.golang.org/grpc"
+
+	common "skywalking.apache.org/repo/goapi/collect/common/v3"
+	agent "skywalking.apache.org/repo/goapi/collect/language/agent/v3"
+	v1 "skywalking.apache.org/repo/goapi/satellite/data/v1"
+
+	_ "github.com/apache/skywalking-satellite/internal/satellite/test"
+	receiver_grpc "github.com/apache/skywalking-satellite/plugins/receiver/grpc"
+)
+
+func TestReceiver_RegisterHandler(t *testing.T) {
+	receiver_grpc.TestReceiver(new(Receiver), func(t *testing.T, sequence int, conn *grpc.ClientConn, ctx context.Context) string {
+		client := agent.NewCLRMetricReportServiceClient(conn)
+		data := initData()
+		_, err := client.Collect(ctx, data)
+		if err != nil {
+			t.Fatalf("cannot send data: %v", err)
+		}
+		return data.String()
+	}, func(data *v1.SniffData) string {
+		return data.GetClr().String()
+	}, t)
+}
+
+func initData() *agent.CLRMetricCollection {
+	return &agent.CLRMetricCollection{
+		Service:         "demo-service",
+		ServiceInstance: "demo-instance",
+		Metrics: []*agent.CLRMetric{
+			{
+				Time: time.Now().Unix() / 1e6,
+				Cpu: &common.CPU{
+					UsagePercent: 99.9,
+				},
+				Gc: &agent.ClrGC{
+					Gen0CollectCount: 1,
+					Gen1CollectCount: 2,
+					Gen2CollectCount: 3,
+					HeapMemory:       1024 * 1024 * 1024,
+				},
+				Thread: &agent.ClrThread{
+					AvailableWorkerThreads:         10,
+					AvailableCompletionPortThreads: 10,
+					MaxWorkerThreads:               64,
+					MaxCompletionPortThreads:       64,
+				},
+			},
+		},
+	}
+}
diff --git a/plugins/receiver/receiver_repository.go b/plugins/receiver/receiver_repository.go
index 4335cff..d75ea41 100644
--- a/plugins/receiver/receiver_repository.go
+++ b/plugins/receiver/receiver_repository.go
@@ -27,6 +27,7 @@
 	"github.com/apache/skywalking-satellite/plugins/receiver/grpc/envoymetricsv2"
 	"github.com/apache/skywalking-satellite/plugins/receiver/grpc/envoymetricsv3"
 	grpcnativecds "github.com/apache/skywalking-satellite/plugins/receiver/grpc/nativecds"
+	grpcnativeclr "github.com/apache/skywalking-satellite/plugins/receiver/grpc/nativeclr"
 	grpcnativeebpfprofiling "github.com/apache/skywalking-satellite/plugins/receiver/grpc/nativeebpfprofiling"
 	grpcnativeevent "github.com/apache/skywalking-satellite/plugins/receiver/grpc/nativeevent"
 	grpcnativejvm "github.com/apache/skywalking-satellite/plugins/receiver/grpc/nativejvm"
@@ -52,6 +53,7 @@
 		new(grpcnativecds.Receiver),
 		new(httpnavtivelog.Receiver),
 		new(grpcnativejvm.Receiver),
+		new(grpcnativeclr.Receiver),
 		new(grpcnativeevent.Receiver),
 		new(grpcnativemeter.Receiver),
 		new(grpcnativeprocess.Receiver),