Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Kafka extract header metada #24367

Merged
merged 19 commits into from
Sep 29, 2023
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 4 additions & 0 deletions receiver/kafkareceiver/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,10 @@ type Config struct {

// Controls the way the messages are marked as consumed
MessageMarking MessageMarking `mapstructure:"message_marking"`

// Extract headers
HeaderExtraction bool `mapstructure:"header_extraction"`
VihasMakwana marked this conversation as resolved.
Show resolved Hide resolved
Headers []string `mapstructure:"headers"`
}

const (
Expand Down
1 change: 1 addition & 0 deletions receiver/kafkareceiver/factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -111,6 +111,7 @@ func createDefaultConfig() component.Config {
After: false,
OnError: false,
},
HeaderExtraction: false,
}
}

Expand Down
77 changes: 77 additions & 0 deletions receiver/kafkareceiver/header_extraction.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
// Copyright The OpenTelemetry Authors
// SPDX-License-Identifier: Apache-2.0

package kafkareceiver // import "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/kafkareceiver"

import (
"fmt"

"github.com/Shopify/sarama"
"go.opentelemetry.io/collector/pdata/plog"
"go.opentelemetry.io/collector/pdata/pmetric"
"go.opentelemetry.io/collector/pdata/ptrace"
"go.uber.org/zap"
)

var getAttribute = func(key string) string {
return fmt.Sprintf("kafka.header.%s", key)
}
VihasMakwana marked this conversation as resolved.
Show resolved Hide resolved

type headerExtractor struct {
logger *zap.Logger
headers []string
}

func (ex *headerExtractor) extractHeadersTraces(traces ptrace.Traces, message *sarama.ConsumerMessage) {
VihasMakwana marked this conversation as resolved.
Show resolved Hide resolved
for _, header := range ex.headers {
value, ok := getHeaderValue(message.Headers, header)
if !ok {
ex.logger.Warn("Header key not found in the trace: ", zap.String("key", header))
VihasMakwana marked this conversation as resolved.
Show resolved Hide resolved
continue
}
for i := 0; i < traces.ResourceSpans().Len(); i++ {
rs := traces.ResourceSpans().At(i)
rs.Resource().Attributes().PutStr(getAttribute(header), value)
}
}
}

func (ex *headerExtractor) extractHeadersLogs(logs plog.Logs, message *sarama.ConsumerMessage) {
for _, header := range ex.headers {
value, ok := getHeaderValue(message.Headers, header)
if !ok {
ex.logger.Warn("Header key not found in the logger: ", zap.String("key", header))
continue
}
for i := 0; i < logs.ResourceLogs().Len(); i++ {
rl := logs.ResourceLogs().At(i)
rl.Resource().Attributes().PutStr(getAttribute(header), value)
}
}
}

func (ex *headerExtractor) extractHeadersMetrics(metrics pmetric.Metrics, message *sarama.ConsumerMessage) {
for _, header := range ex.headers {
value, ok := getHeaderValue(message.Headers, header)
if !ok {
ex.logger.Warn("Header key not found in the metric: ", zap.String("key", header))
continue
}
for i := 0; i < metrics.ResourceMetrics().Len(); i++ {
rm := metrics.ResourceMetrics().At(i)
rm.Resource().Attributes().PutStr(getAttribute(header), value)
}
}
}

func getHeaderValue(headers []*sarama.RecordHeader, header string) (string, bool) {
for _, kafkaHeader := range headers {
headerKey := string(kafkaHeader.Key)
if headerKey == header {
// matching header found
return string(kafkaHeader.Value), true
}
}
// no header found matching the key, report to the user
return "", false
}
211 changes: 211 additions & 0 deletions receiver/kafkareceiver/header_extraction_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,211 @@
package kafkareceiver

import (
"context"
VihasMakwana marked this conversation as resolved.
Show resolved Hide resolved
"github.com/Shopify/sarama"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"go.opentelemetry.io/collector/obsreport"
"go.opentelemetry.io/collector/receiver/receivertest"

"bytes"
"fmt"
"github.com/open-telemetry/opentelemetry-collector-contrib/internal/coreinternal/testdata"
"go.opentelemetry.io/collector/consumer"
"go.opentelemetry.io/collector/consumer/consumertest"
"go.opentelemetry.io/collector/pdata/pcommon"
"go.opentelemetry.io/collector/pdata/plog"
"go.opentelemetry.io/collector/pdata/pmetric"
"go.opentelemetry.io/collector/pdata/ptrace"
"go.uber.org/zap"
"math"
"strings"
"sync"
"testing"
)

func TestHeaderExtractionTraces(t *testing.T) {
obsrecv, err := obsreport.NewReceiver(obsreport.ReceiverSettings{ReceiverCreateSettings: receivertest.NewNopCreateSettings()})
nextConsumer := (new(consumertest.TracesSink))
VihasMakwana marked this conversation as resolved.
Show resolved Hide resolved
c := tracesConsumerGroupHandler{
unmarshaler: newPdataTracesUnmarshaler(&ptrace.ProtoUnmarshaler{}, defaultEncoding),
logger: zap.NewNop(),
VihasMakwana marked this conversation as resolved.
Show resolved Hide resolved
ready: make(chan bool),
nextConsumer: nextConsumer,
// nextConsumer: &mockConsumer{},
VihasMakwana marked this conversation as resolved.
Show resolved Hide resolved
obsrecv: obsrecv,
}
headers := []string{"headerKey1", "headerKey2"}
c.headerExtractor = &headerExtractor{
logger: zap.NewNop(),
headers: headers,
}
groupClaim := &testConsumerGroupClaim{
messageChan: make(chan *sarama.ConsumerMessage),
}
ctx, cancelFunc := context.WithCancel(context.Background())
defer close(groupClaim.messageChan)
testSession := testConsumerGroupSession{ctx: ctx}
require.NoError(t, c.Setup(testSession))
wg := sync.WaitGroup{}
wg.Add(1)
go func() {
err = c.ConsumeClaim(testSession, groupClaim)
for _, trace := range nextConsumer.AllTraces() {
for i := 0; i < trace.ResourceSpans().Len(); i++ {
rs := trace.ResourceSpans().At(i)
val, ok := rs.Resource().Attributes().Get("kafka.header.headerKey1")
assert.Equal(t, ok, true)
assert.Equal(t, val.Str(), "headerValue1")
val, ok = rs.Resource().Attributes().Get("kafka.header.headerKey2")
assert.Equal(t, ok, true)
assert.Equal(t, val.Str(), "headerValue2")
}
}
assert.NoError(t, err)
wg.Done()
}()
td := ptrace.NewTraces()
td.ResourceSpans().AppendEmpty().Resource()
td.ResourceSpans().At(0).ScopeSpans().AppendEmpty().Spans().AppendEmpty()
unmarshaler := &ptrace.ProtoMarshaler{}
bts, err := unmarshaler.MarshalTraces(td)
groupClaim.messageChan <- &sarama.ConsumerMessage{
Headers: []*sarama.RecordHeader{
{
Key: []byte("headerKey1"),
Value: []byte("headerValue1"),
},
{
Key: []byte("headerKey2"),
Value: []byte("headerValue2"),
},
},
Value: bts,
}
cancelFunc()
wg.Wait()

}

func TestHeaderExtractionLogs(t *testing.T) {
obsrecv, err := obsreport.NewReceiver(obsreport.ReceiverSettings{ReceiverCreateSettings: receivertest.NewNopCreateSettings()})
nextConsumer := new(consumertest.LogsSink)
unmarshaler := newTextLogsUnmarshaler()
unmarshaler, err = unmarshaler.WithEnc("utf-8")
c := logsConsumerGroupHandler{
unmarshaler: unmarshaler,
logger: zap.NewNop(),
ready: make(chan bool),
nextConsumer: nextConsumer,
obsrecv: obsrecv,
}
headers := []string{"headerKey1", "headerKey2"}
c.headerExtractor = &headerExtractor{
logger: zap.NewNop(),
headers: headers,
}
groupClaim := &testConsumerGroupClaim{
messageChan: make(chan *sarama.ConsumerMessage),
}
ctx, cancelFunc := context.WithCancel(context.Background())
defer close(groupClaim.messageChan)
testSession := testConsumerGroupSession{ctx: ctx}
require.NoError(t, c.Setup(testSession))
wg := sync.WaitGroup{}
wg.Add(1)
go func() {
err = c.ConsumeClaim(testSession, groupClaim)
for _, logs := range nextConsumer.AllLogs() {
for i := 0; i < logs.ResourceLogs().Len(); i++ {
rs := logs.ResourceLogs().At(i)
val, ok := rs.Resource().Attributes().Get("kafka.header.headerKey1")
assert.Equal(t, ok, true)
assert.Equal(t, val.Str(), "headerValue1")
val, ok = rs.Resource().Attributes().Get("kafka.header.headerKey2")
assert.Equal(t, ok, true)
assert.Equal(t, val.Str(), "headerValue2")
}
}
assert.NoError(t, err)
wg.Done()
}()
groupClaim.messageChan <- &sarama.ConsumerMessage{
Headers: []*sarama.RecordHeader{
{
Key: []byte("headerKey1"),
Value: []byte("headerValue1"),
},
{
Key: []byte("headerKey2"),
Value: []byte("headerValue2"),
},
},
Value: []byte("Message"),
}
cancelFunc()
wg.Wait()

}

func TestHeaderExtractionMetrics(t *testing.T) {
obsrecv, err := obsreport.NewReceiver(obsreport.ReceiverSettings{ReceiverCreateSettings: receivertest.NewNopCreateSettings()})
nextConsumer := new(consumertest.MetricsSink)
c := metricsConsumerGroupHandler{
unmarshaler: newPdataMetricsUnmarshaler(&pmetric.ProtoUnmarshaler{}, defaultEncoding),
logger: zap.NewNop(),
ready: make(chan bool),
nextConsumer: nextConsumer,
obsrecv: obsrecv,
}
headers := []string{"headerKey1", "headerKey2"}
c.headerExtractor = &headerExtractor{
logger: zap.NewNop(),
headers: headers,
}
groupClaim := &testConsumerGroupClaim{
messageChan: make(chan *sarama.ConsumerMessage),
}
ctx, cancelFunc := context.WithCancel(context.Background())
defer close(groupClaim.messageChan)
testSession := testConsumerGroupSession{ctx: ctx}
require.NoError(t, c.Setup(testSession))
wg := sync.WaitGroup{}
wg.Add(1)
go func() {
err = c.ConsumeClaim(testSession, groupClaim)
for _, metric := range nextConsumer.AllMetrics() {
for i := 0; i < metric.ResourceMetrics().Len(); i++ {
rs := metric.ResourceMetrics().At(i)
val, ok := rs.Resource().Attributes().Get("kafka.header.headerKey1")
assert.Equal(t, ok, true)
assert.Equal(t, val.Str(), "headerValue1")
val, ok = rs.Resource().Attributes().Get("kafka.header.headerKey2")
assert.Equal(t, ok, true)
assert.Equal(t, val.Str(), "headerValue2")
}
}
assert.NoError(t, err)
wg.Done()
}()
ld := testdata.GenerateMetricsOneMetric()
VihasMakwana marked this conversation as resolved.
Show resolved Hide resolved
unmarshaler := &pmetric.ProtoMarshaler{}
bts, err := unmarshaler.MarshalMetrics(ld)
groupClaim.messageChan <- &sarama.ConsumerMessage{
Headers: []*sarama.RecordHeader{
{
Key: []byte("headerKey1"),
Value: []byte("headerValue1"),
},
{
Key: []byte("headerKey2"),
Value: []byte("headerValue2"),
},
},
Value: bts,
}
// groupClaim.messageChan <- &sarama.ConsumerMessage{}
cancelFunc()
wg.Wait()

}
Loading