Skip to content

Commit 569e38a

Browse files
committed
Added log grpc messages metadata on trace log level for topic writer
1 parent 7ec7b21 commit 569e38a

File tree

11 files changed

+313
-31
lines changed

11 files changed

+313
-31
lines changed

CHANGELOG.md

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,3 +1,5 @@
1+
* Added log grpc messages metadata on trace log level for topic writer
2+
13
## v3.99.5
24
* Fixed error `Empty query text` using prepared statements and `ydb.WithExecuteDataQueryOverQueryClient(true)` option
35
* Prepared statements always send query text on Execute call from now (previous behaviour - send query ID)

internal/grpcwrapper/rawtopic/client.go

Lines changed: 11 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -3,6 +3,8 @@ package rawtopic
33
import (
44
"context"
55
"fmt"
6+
"github.com/google/uuid"
7+
"github.com/ydb-platform/ydb-go-sdk/v3/trace"
68

79
"github.com/ydb-platform/ydb-go-genproto/Ydb_Topic_V1"
810

@@ -99,7 +101,10 @@ func (c *Client) StreamRead(ctxStreamLifeTime context.Context) (rawtopicreader.S
99101
return rawtopicreader.StreamReader{Stream: protoResp}, nil
100102
}
101103

102-
func (c *Client) StreamWrite(ctxStreamLifeTime context.Context) (*rawtopicwriter.StreamWriter, error) {
104+
func (c *Client) StreamWrite(
105+
ctxStreamLifeTime context.Context,
106+
tracer *trace.Topic,
107+
) (*rawtopicwriter.StreamWriter, error) {
103108
protoResp, err := c.service.StreamWrite(ctxStreamLifeTime)
104109
if err != nil {
105110
return nil, xerrors.WithStackTrace(
@@ -109,7 +114,11 @@ func (c *Client) StreamWrite(ctxStreamLifeTime context.Context) (*rawtopicwriter
109114
)
110115
}
111116

112-
return &rawtopicwriter.StreamWriter{Stream: protoResp}, nil
117+
return &rawtopicwriter.StreamWriter{
118+
Stream: protoResp,
119+
Tracer: tracer,
120+
InternalStreamID: uuid.New().String(),
121+
}, nil
113122
}
114123

115124
func (c *Client) UpdateOffsetsInTransaction(

internal/grpcwrapper/rawtopic/rawtopicwriter/streamwriter.go

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3,6 +3,7 @@ package rawtopicwriter
33
import (
44
"errors"
55
"fmt"
6+
"github.com/ydb-platform/ydb-go-sdk/v3/trace"
67
"reflect"
78
"sync"
89
"sync/atomic"
@@ -29,6 +30,12 @@ type StreamWriter struct {
2930

3031
sendCloseMtx sync.Mutex
3132
Stream GrpcStream
33+
34+
Tracer *trace.Topic
35+
InternalStreamID string
36+
readMessagesCount int
37+
writtenMessagesCount int
38+
sessionID string
3239
}
3340

3441
func (w *StreamWriter) Recv() (ServerMessage, error) {
@@ -40,6 +47,8 @@ func (w *StreamWriter) Recv() (ServerMessage, error) {
4047
}
4148

4249
grpcMsg, err := w.Stream.Recv()
50+
w.readMessagesCount++
51+
trace.TopicOnWriterReceiveGRPCMessage(w.Tracer, w.InternalStreamID, w.sessionID, w.readMessagesCount, grpcMsg, err)
4352
if err != nil {
4453
if !xerrors.IsErrorFromServer(err) {
4554
err = xerrors.Transport(err)
@@ -64,6 +73,7 @@ func (w *StreamWriter) Recv() (ServerMessage, error) {
6473
var res InitResult
6574
res.ServerMessageMetadata = meta
6675
res.mustFromProto(v.InitResponse)
76+
w.sessionID = res.SessionID
6777

6878
return &res, nil
6979
case *Ydb_Topic.StreamWriteMessage_FromServer_WriteResponse:
@@ -124,6 +134,8 @@ func (w *StreamWriter) Send(rawMsg ClientMessage) (err error) {
124134
}
125135

126136
err = w.Stream.Send(&protoMsg)
137+
w.writtenMessagesCount++
138+
trace.TopicOnWriterSentGRPCMessage(w.Tracer, w.InternalStreamID, w.sessionID, w.writtenMessagesCount, &protoMsg, err)
127139
if err != nil {
128140
return xerrors.WithStackTrace(xerrors.Wrap(fmt.Errorf("ydb: failed to send grpc message to writer stream: %w", err)))
129141
}

internal/topic/topicclientinternal/client.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -356,11 +356,11 @@ func (c *Client) createWriterConfig(
356356
topicPath string,
357357
opts []topicoptions.WriterOption,
358358
) topicwriterinternal.WriterReconnectorConfig {
359-
var connector topicwriterinternal.ConnectFunc = func(ctx context.Context) (
359+
var connector topicwriterinternal.ConnectFunc = func(ctx context.Context, tracer *trace.Topic) (
360360
topicwriterinternal.RawTopicWriterStream,
361361
error,
362362
) {
363-
return c.rawClient.StreamWrite(ctx)
363+
return c.rawClient.StreamWrite(ctx, tracer)
364364
}
365365

366366
options := []topicoptions.WriterOption{

internal/topic/topicwriterinternal/writer_reconnector.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -517,7 +517,7 @@ func (w *WriterReconnector) connectWithTimeout(streamLifetimeContext context.Con
517517
}
518518
}()
519519

520-
stream, err := w.cfg.Connect(connectCtx)
520+
stream, err := w.cfg.Connect(connectCtx, w.cfg.Tracer)
521521
resCh <- resT{stream: stream, err: err}
522522
}()
523523

@@ -789,7 +789,7 @@ func calculateAllowedCodecs(forceCodec rawtopiccommon.Codec, multiEncoder *Multi
789789
return res
790790
}
791791

792-
type ConnectFunc func(ctx context.Context) (RawTopicWriterStream, error)
792+
type ConnectFunc func(ctx context.Context, tracer *trace.Topic) (RawTopicWriterStream, error)
793793

794794
func createPublicCodecsFromRaw(codecs rawtopiccommon.SupportedCodecs) []topictypes.Codec {
795795
res := make([]topictypes.Codec, len(codecs))

internal/topic/topicwriterinternal/writer_reconnector_test.go

Lines changed: 14 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -6,6 +6,7 @@ import (
66
"context"
77
"errors"
88
"fmt"
9+
"github.com/ydb-platform/ydb-go-sdk/v3/trace"
910
"io"
1011
"math"
1112
"sort"
@@ -446,7 +447,7 @@ func TestWriterImpl_Reconnect(t *testing.T) {
446447
connectCalled := false
447448
connectCalledChan := make(empty.Chan)
448449

449-
w.cfg.Connect = func(streamCtxArg context.Context) (RawTopicWriterStream, error) {
450+
w.cfg.Connect = func(streamCtxArg context.Context, _ *trace.Topic) (RawTopicWriterStream, error) {
450451
close(connectCalledChan)
451452
connectCalled = true
452453
require.NotEqual(t, ctx, streamCtxArg)
@@ -562,7 +563,7 @@ func TestWriterImpl_Reconnect(t *testing.T) {
562563
}
563564

564565
var connectionAttempt atomic.Int64
565-
w.cfg.Connect = func(ctx context.Context) (RawTopicWriterStream, error) {
566+
w.cfg.Connect = func(ctx context.Context, _ *trace.Topic) (RawTopicWriterStream, error) {
566567
attemptIndex := int(connectionAttempt.Add(1)) - 1
567568
t.Logf("connect with attempt index: %v", attemptIndex)
568569
res := connectsResult[attemptIndex]
@@ -1078,17 +1079,18 @@ func newTestEnv(t testing.TB, options *testEnvOptions) *testEnv {
10781079
partitionID: 14,
10791080
}
10801081

1081-
writerOptions := append(defaultTestWriterOptions(), WithConnectFunc(func(ctx context.Context) (
1082-
RawTopicWriterStream,
1083-
error,
1084-
) {
1085-
connectNum := atomic.AddInt64(&res.connectCount, 1)
1086-
if connectNum > 1 {
1087-
t.Fatalf("test: default env support most one connection")
1088-
}
1082+
writerOptions := append(defaultTestWriterOptions(), WithConnectFunc(
1083+
func(ctx context.Context, _ *trace.Topic) (
1084+
RawTopicWriterStream,
1085+
error,
1086+
) {
1087+
connectNum := atomic.AddInt64(&res.connectCount, 1)
1088+
if connectNum > 1 {
1089+
t.Fatalf("test: default env support most one connection")
1090+
}
10891091

1090-
return res.stream, nil
1091-
}))
1092+
return res.stream, nil
1093+
}))
10921094
writerOptions = append(writerOptions, options.writerOptions...)
10931095

10941096
res.writer = newWriterReconnectorStopped(NewWriterReconnectorConfig(writerOptions...))

log/topic.go

Lines changed: 70 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2,6 +2,9 @@ package log
22

33
import (
44
"context"
5+
"github.com/ydb-platform/ydb-go-genproto/protos/Ydb_Topic"
6+
"google.golang.org/protobuf/encoding/protojson"
7+
"google.golang.org/protobuf/proto"
58
"time"
69

710
"github.com/ydb-platform/ydb-go-sdk/v3/internal/kv"
@@ -885,6 +888,39 @@ func internalTopic(l Logger, d trace.Detailer) (t trace.Topic) {
885888
kv.Version(),
886889
)
887890
}
891+
892+
t.OnWriterSentGRPCMessage = func(info trace.TopicWriterSentGRPCMessageInfo) {
893+
if d.Details()&trace.TopicWriterStreamGrpcMessageEvents == 0 {
894+
return
895+
}
896+
897+
ctx := with(context.Background(), TRACE, "ydb", "topic", "writer", "grpc")
898+
l.Log(
899+
ctx, "topic writer sent grpc message (message body and metadata are removed)",
900+
kv.String("topic_stream_internal_id", info.TopicStreamInternalID),
901+
kv.String("session_id", info.SessionID),
902+
kv.Int("message_number", info.MessageNumber),
903+
kv.Stringer("message", lazyProtoStringifer{info.Message}),
904+
kv.Error(info.Error),
905+
kv.Version(),
906+
)
907+
}
908+
t.OnWriterReceiveGRPCMessage = func(info trace.TopicWriterReceiveGRPCMessageInfo) {
909+
if d.Details()&trace.TopicWriterStreamGrpcMessageEvents == 0 {
910+
return
911+
}
912+
913+
ctx := with(context.Background(), TRACE, "ydb", "topic", "writer", "grpc")
914+
l.Log(
915+
ctx, "topic writer received grpc message (message body and metadata are removed)",
916+
kv.String("topic_stream_internal_id", info.TopicStreamInternalID),
917+
kv.String("session_id", info.SessionID),
918+
kv.Int("message_number", info.MessageNumber),
919+
kv.Stringer("message", lazyProtoStringifer{info.Message}),
920+
kv.Error(info.Error),
921+
kv.Version(),
922+
)
923+
}
888924
t.OnWriterReadUnknownGrpcMessage = func(info trace.TopicOnWriterReadUnknownGrpcMessageInfo) {
889925
if d.Details()&trace.TopicWriterStreamEvents == 0 {
890926
return
@@ -899,3 +935,37 @@ func internalTopic(l Logger, d trace.Detailer) (t trace.Topic) {
899935

900936
return t
901937
}
938+
939+
type lazyProtoStringifer struct {
940+
message proto.Message
941+
}
942+
943+
func (s lazyProtoStringifer) String() string {
944+
// cut message data
945+
if writeRequest, ok := s.message.(*Ydb_Topic.StreamWriteMessage_FromClient); ok {
946+
if data := writeRequest.GetWriteRequest(); data != nil {
947+
type messDataType struct {
948+
Data []byte
949+
Metadata []*Ydb_Topic.MetadataItem
950+
}
951+
storage := make([]messDataType, len(data.Messages))
952+
for i := range data.Messages {
953+
storage[i].Data = data.Messages[i].Data
954+
data.Messages[i] = nil
955+
956+
storage[i].Metadata = data.Messages[i].MetadataItems
957+
data.Messages[i].MetadataItems = nil
958+
}
959+
960+
defer func() {
961+
for i := range data.Messages {
962+
data.Messages[i].Data = storage[i].Data
963+
data.Messages[i].MetadataItems = storage[i].Metadata
964+
}
965+
}()
966+
}
967+
}
968+
969+
res := protojson.MarshalOptions{AllowPartial: true}.Format(s.message)
970+
return res
971+
}

tests/integration/topic_read_writer_test.go

Lines changed: 56 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -8,6 +8,7 @@ import (
88
"context"
99
"encoding/binary"
1010
"errors"
11+
"github.com/ydb-platform/ydb-go-sdk/v3/log"
1112
"io"
1213
"os"
1314
"runtime/pprof"
@@ -33,6 +34,61 @@ import (
3334
"github.com/ydb-platform/ydb-go-sdk/v3/trace"
3435
)
3536

37+
func TestTopicWriterLogMessagesWithoutData(t *testing.T) {
38+
scope := newScope(t)
39+
40+
producerID := "dkeujsl"
41+
const seqNoInt = 486812497
42+
seqNoString := "486812497"
43+
data := "kdjwkruowe"
44+
metaKey := "gyoeexiufo"
45+
metaValue := "fjedeikeosbv"
46+
47+
logs := &strings.Builder{}
48+
writer, err := scope.Driver().Topic().StartWriter(
49+
scope.TopicPath(),
50+
topicoptions.WithWriterProducerID(producerID),
51+
topicoptions.WithWriterSetAutoSeqNo(false),
52+
topicoptions.WithWriterWaitServerAck(true),
53+
topicoptions.WithWriterTrace(log.Topic(
54+
log.Default(logs, log.WithMinLevel(log.TRACE)), trace.TopicWriterStreamGrpcMessageEvents),
55+
),
56+
)
57+
58+
scope.Require.NoError(err)
59+
err = writer.Write(scope.Ctx,
60+
topicwriter.Message{
61+
SeqNo: seqNoInt,
62+
Data: strings.NewReader(data),
63+
Metadata: map[string][]byte{
64+
metaKey: []byte(metaValue),
65+
},
66+
},
67+
)
68+
scope.Require.NoError(err)
69+
70+
err = writer.Close(scope.Ctx)
71+
scope.Require.NoError(err)
72+
73+
logsString := logs.String()
74+
scope.Require.Contains(logsString, producerID)
75+
scope.Require.Contains(logsString, seqNoString)
76+
scope.Require.NotContains(logsString, metaKey)
77+
scope.Require.NotContains(logsString, metaValue)
78+
scope.Require.NotContains(logsString, data)
79+
80+
mess, err := scope.TopicReader().ReadMessage(scope.Ctx)
81+
scope.Require.NoError(err)
82+
83+
scope.Require.Equal(producerID, mess.ProducerID)
84+
scope.Require.Equal(int64(seqNoInt), mess.SeqNo)
85+
scope.Require.Equal(metaValue, string(mess.Metadata[metaKey]))
86+
87+
messData, err := io.ReadAll(mess)
88+
scope.Require.NoError(err)
89+
scope.Require.Equal(data, string(messData))
90+
}
91+
3692
func TestSendAsyncMessages(t *testing.T) {
3793
ctx := context.Background()
3894
db := connect(t)

trace/details.go

Lines changed: 15 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -64,6 +64,7 @@ const (
6464

6565
TopicWriterStreamLifeCycleEvents
6666
TopicWriterStreamEvents
67+
TopicWriterStreamGrpcMessageEvents
6768

6869
DatabaseSQLConnectorEvents
6970
DatabaseSQLConnEvents
@@ -116,7 +117,10 @@ const (
116117
TopicReaderPartitionEvents |
117118
TopicReaderStreamLifeCycleEvents
118119

119-
TopicEvents = TopicControlPlaneEvents | TopicReaderEvents
120+
TopicWriterEvents = TopicWriterStreamLifeCycleEvents | TopicWriterStreamEvents |
121+
TopicWriterStreamGrpcMessageEvents
122+
123+
TopicEvents = TopicControlPlaneEvents | TopicReaderEvents | TopicWriterEvents
120124

121125
DatabaseSQLEvents = DatabaseSQLConnectorEvents |
122126
DatabaseSQLConnEvents |
@@ -169,15 +173,16 @@ var (
169173
DatabaseSQLTxEvents: "ydb.database.sql.tx",
170174
DatabaseSQLStmtEvents: "ydb.database.sql.stmt",
171175

172-
TopicEvents: "ydb.topic",
173-
TopicControlPlaneEvents: "ydb.topic.controlplane",
174-
TopicReaderEvents: "ydb.topic.reader",
175-
TopicReaderStreamEvents: "ydb.topic.reader.stream",
176-
TopicReaderMessageEvents: "ydb.topic.reader.message",
177-
TopicReaderPartitionEvents: "ydb.topic.reader.partition",
178-
TopicReaderStreamLifeCycleEvents: "ydb.topic.reader.lifecycle",
179-
TopicWriterStreamLifeCycleEvents: "ydb.topic.writer.lifecycle",
180-
TopicWriterStreamEvents: "ydb.topic.writer.stream",
176+
TopicEvents: "ydb.topic",
177+
TopicControlPlaneEvents: "ydb.topic.controlplane",
178+
TopicReaderEvents: "ydb.topic.reader",
179+
TopicReaderStreamEvents: "ydb.topic.reader.stream",
180+
TopicReaderMessageEvents: "ydb.topic.reader.message",
181+
TopicReaderPartitionEvents: "ydb.topic.reader.partition",
182+
TopicReaderStreamLifeCycleEvents: "ydb.topic.reader.lifecycle",
183+
TopicWriterStreamLifeCycleEvents: "ydb.topic.writer.lifecycle",
184+
TopicWriterStreamEvents: "ydb.topic.writer.stream",
185+
TopicWriterStreamGrpcMessageEvents: "ydb.topic.writer.grpc",
181186
}
182187
defaultDetails = DetailsAll
183188
)

0 commit comments

Comments
 (0)