diff --git a/backend/pkg/api/connect/service/console/service.go b/backend/pkg/api/connect/service/console/service.go index 2a18627c5..2743aa167 100644 --- a/backend/pkg/api/connect/service/console/service.go +++ b/backend/pkg/api/connect/service/console/service.go @@ -105,6 +105,7 @@ func (api *Service) ListMessages(ctx context.Context, req *connect.Request[v1alp FilterInterpreterCode: interpreterCode, Troubleshoot: req.Msg.GetTroubleshoot(), IncludeRawPayload: req.Msg.GetIncludeOriginalRawPayload(), + IgnoreMaxSizeLimit: req.Msg.GetIgnoreMaxSizeLimit(), KeyDeserializer: fromProtoEncoding(req.Msg.GetKeyDeserializer()), ValueDeserializer: fromProtoEncoding(req.Msg.GetValueDeserializer()), } diff --git a/backend/pkg/api/handle_topic_messages_integration_test.go b/backend/pkg/api/handle_topic_messages_integration_test.go index b6b6bcbf9..d01b1fe2b 100644 --- a/backend/pkg/api/handle_topic_messages_integration_test.go +++ b/backend/pkg/api/handle_topic_messages_integration_test.go @@ -13,11 +13,15 @@ package api import ( "context" + "encoding/base64" + "encoding/json" "errors" "fmt" + "math/rand" "net/http" "os" "path/filepath" + "sort" "testing" "time" @@ -37,7 +41,7 @@ import ( things "github.com/redpanda-data/console/backend/pkg/testutil/testdata/proto/gen/things/v1" ) -// this is a complex test, no reson to refactor it +// this is a complex test, no reason to refactor it func (s *APIIntegrationTestSuite) TestListMessages() { t := s.T() @@ -57,10 +61,34 @@ func (s *APIIntegrationTestSuite) TestListMessages() { testutil.CreateTestData(t, context.Background(), s.kafkaClient, s.kafkaAdminClient, topicName) + topicNameBig := testutil.TopicNameForTest("list_messages_big_0") + testutil.CreateTestData(t, context.Background(), s.kafkaClient, s.kafkaAdminClient, + topicNameBig) + defer func() { s.kafkaAdminClient.DeleteTopics(context.Background(), topicName) + s.kafkaAdminClient.DeleteTopics(context.Background(), topicName) }() + // produce too big of a message + order := testutil.Order{ID: randomString(21000)} + serializedOrder, err := json.Marshal(order) + require.NoError(err) + + r := &kgo.Record{ + Key: []byte("too_big_0"), + Value: serializedOrder, + Topic: topicNameBig, + Headers: []kgo.RecordHeader{ + { + Key: "revision", + Value: []byte("0"), + }, + }, + } + results := s.kafkaClient.ProduceSync(ctx, r) + require.NoError(results.FirstErr()) + t.Run("simple happy path", func(t *testing.T) { stream, err := client.ListMessages(ctx, connect.NewRequest(&v1pb.ListMessagesRequest{ Topic: topicName, @@ -262,6 +290,310 @@ func (s *APIIntegrationTestSuite) TestListMessages() { assert.True(seenZeroOffset) assert.GreaterOrEqual(progressCount, 0) }) + + t.Run("with key filter", func(t *testing.T) { + filterCode := base64.StdEncoding.EncodeToString([]byte( + `return key.endsWith('2') || key.endsWith('3')`, + )) + + stream, err := client.ListMessages(ctx, connect.NewRequest(&v1pb.ListMessagesRequest{ + Topic: topicName, + StartOffset: -2, + PartitionId: -1, + MaxResults: 100, + FilterInterpreterCode: filterCode, + })) + require.NoError(err) + + keys := make([]string, 0, 4) + phaseCount := 0 + doneCount := 0 + progressCount := 0 + errorCount := 0 + + for stream.Receive() { + msg := stream.Msg() + switch cm := msg.GetControlMessage().(type) { + case *v1pb.ListMessagesResponse_Data: + assert.NotEmpty(cm.Data.Timestamp) + assert.NotEmpty(cm.Data.Compression) + assert.NotEmpty(cm.Data.Headers) + + for _, h := range cm.Data.Headers { + h := h + assert.NotEmpty(h) + assert.NotEmpty(h.Key) + assert.NotEmpty(h.Value) + } + + key := string(cm.Data.GetKey().GetNormalizedPayload()) + keys = append(keys, key) + + assert.NotEmpty(cm.Data.GetKey()) + assert.NotEmpty(cm.Data.GetKey().GetNormalizedPayload()) + assert.Empty(cm.Data.GetKey().GetOriginalPayload()) + assert.NotEmpty(cm.Data.GetKey().GetPayloadSize()) + assert.Equal(v1pb.PayloadEncoding_PAYLOAD_ENCODING_TEXT, cm.Data.GetKey().GetEncoding()) + assert.False(cm.Data.GetKey().GetIsPayloadTooLarge()) + assert.Empty(cm.Data.GetKey().GetTroubleshootReport()) + + assert.NotEmpty(cm.Data.GetValue()) + assert.NotEmpty(cm.Data.GetValue().GetNormalizedPayload()) + assert.Empty(cm.Data.GetValue().GetOriginalPayload()) + assert.NotEmpty(cm.Data.GetValue().GetPayloadSize()) + assert.Equal(v1pb.PayloadEncoding_PAYLOAD_ENCODING_JSON, cm.Data.GetValue().GetEncoding()) + assert.False(cm.Data.GetValue().GetIsPayloadTooLarge()) + assert.Empty(cm.Data.GetValue().GetTroubleshootReport()) + + assert.Equal(fmt.Sprintf(`{"ID":%q}`, key), string(cm.Data.GetValue().GetNormalizedPayload())) + case *v1pb.ListMessagesResponse_Done: + doneCount++ + + assert.NotEmpty(cm.Done.GetBytesConsumed()) + assert.NotEmpty(cm.Done.GetMessagesConsumed()) + assert.NotEmpty(cm.Done.GetElapsedMs()) + assert.False(cm.Done.GetIsCancelled()) + case *v1pb.ListMessagesResponse_Phase: + if phaseCount == 0 { + assert.Equal("Get Partitions", cm.Phase.GetPhase()) + } else if phaseCount == 1 { + assert.Equal("Get Watermarks and calculate consuming requests", cm.Phase.GetPhase()) + } else if phaseCount == 2 { + assert.Equal("Consuming messages", cm.Phase.GetPhase()) + } else { + assert.Fail("Unknown phase.") + } + + phaseCount++ + case *v1pb.ListMessagesResponse_Progress: + progressCount++ + + assert.NotEmpty(cm.Progress) + assert.NotEmpty(cm.Progress.GetBytesConsumed()) + assert.NotEmpty(cm.Progress.GetMessagesConsumed()) + case *v1pb.ListMessagesResponse_Error: + errorCount++ + } + } + + assert.Nil(stream.Err()) + assert.Nil(stream.Close()) + + sort.Strings(keys) + + assert.Equal([]string{"12", "13", "2", "3"}, keys) + assert.Equal(3, phaseCount) + assert.Equal(0, errorCount) + assert.Equal(1, doneCount) + assert.GreaterOrEqual(progressCount, 0) + }) + + t.Run("too big", func(t *testing.T) { + stream, err := client.ListMessages(ctx, connect.NewRequest(&v1pb.ListMessagesRequest{ + Topic: topicNameBig, + StartOffset: -1, + PartitionId: -1, + MaxResults: 5, + })) + require.NoError(err) + + keys := make([]string, 0, 5) + phaseCount := 0 + doneCount := 0 + progressCount := 0 + errorCount := 0 + seenZeroOffset := false + for stream.Receive() { + msg := stream.Msg() + switch cm := msg.GetControlMessage().(type) { + case *v1pb.ListMessagesResponse_Data: + if seenZeroOffset { + assert.NotEmpty(cm.Data.Offset) + } + + if cm.Data.Offset == 0 { + seenZeroOffset = true + } + + assert.NotEmpty(cm.Data.Timestamp) + assert.NotEmpty(cm.Data.Compression) + assert.NotEmpty(cm.Data.Headers) + + for _, h := range cm.Data.Headers { + h := h + assert.NotEmpty(h) + assert.NotEmpty(h.Key) + assert.NotEmpty(h.Value) + } + + key := string(cm.Data.GetKey().GetNormalizedPayload()) + keys = append(keys, key) + + assert.NotEmpty(cm.Data.GetKey()) + assert.NotEmpty(cm.Data.GetKey().GetNormalizedPayload()) + assert.Empty(cm.Data.GetKey().GetOriginalPayload()) + assert.NotEmpty(cm.Data.GetKey().GetPayloadSize()) + assert.Equal(v1pb.PayloadEncoding_PAYLOAD_ENCODING_TEXT, cm.Data.GetKey().GetEncoding()) + assert.False(cm.Data.GetKey().GetIsPayloadTooLarge()) + assert.Empty(cm.Data.GetKey().GetTroubleshootReport()) + + assert.NotEmpty(cm.Data.GetValue()) + assert.Empty(cm.Data.GetValue().GetOriginalPayload()) + assert.NotEmpty(cm.Data.GetValue().GetPayloadSize()) + assert.Equal(v1pb.PayloadEncoding_PAYLOAD_ENCODING_JSON, cm.Data.GetValue().GetEncoding()) + assert.Empty(cm.Data.GetValue().GetTroubleshootReport()) + + if key == "too_big_0" { + assert.True(cm.Data.GetValue().GetIsPayloadTooLarge()) + assert.Empty(cm.Data.GetValue().GetNormalizedPayload()) + } else { + assert.False(cm.Data.GetValue().GetIsPayloadTooLarge()) + assert.NotEmpty(cm.Data.GetValue().GetNormalizedPayload()) + } + + case *v1pb.ListMessagesResponse_Done: + doneCount++ + + assert.NotEmpty(cm.Done.GetBytesConsumed()) + assert.NotEmpty(cm.Done.GetMessagesConsumed()) + assert.NotEmpty(cm.Done.GetElapsedMs()) + assert.False(cm.Done.GetIsCancelled()) + case *v1pb.ListMessagesResponse_Phase: + if phaseCount == 0 { + assert.Equal("Get Partitions", cm.Phase.GetPhase()) + } else if phaseCount == 1 { + assert.Equal("Get Watermarks and calculate consuming requests", cm.Phase.GetPhase()) + } else if phaseCount == 2 { + assert.Equal("Consuming messages", cm.Phase.GetPhase()) + } else { + assert.Fail("Unknown phase.") + } + + phaseCount++ + case *v1pb.ListMessagesResponse_Progress: + progressCount++ + + assert.NotEmpty(cm.Progress) + assert.NotEmpty(cm.Progress.GetBytesConsumed()) + assert.NotEmpty(cm.Progress.GetMessagesConsumed()) + case *v1pb.ListMessagesResponse_Error: + errorCount++ + } + } + + assert.Nil(stream.Err()) + assert.Nil(stream.Close()) + assert.Equal([]string{"16", "17", "18", "19", "too_big_0"}, keys) + assert.Equal(3, phaseCount) + assert.Equal(0, errorCount) + assert.Equal(1, doneCount) + assert.False(seenZeroOffset) + assert.GreaterOrEqual(progressCount, 0) + }) + + t.Run("too big with ignore", func(t *testing.T) { + stream, err := client.ListMessages(ctx, connect.NewRequest(&v1pb.ListMessagesRequest{ + Topic: topicNameBig, + StartOffset: -1, + PartitionId: -1, + MaxResults: 5, + IgnoreMaxSizeLimit: true, + })) + require.NoError(err) + + keys := make([]string, 0, 5) + phaseCount := 0 + doneCount := 0 + progressCount := 0 + errorCount := 0 + seenZeroOffset := false + for stream.Receive() { + msg := stream.Msg() + switch cm := msg.GetControlMessage().(type) { + case *v1pb.ListMessagesResponse_Data: + if seenZeroOffset { + assert.NotEmpty(cm.Data.Offset) + } + + if cm.Data.Offset == 0 { + seenZeroOffset = true + } + + assert.NotEmpty(cm.Data.Timestamp) + assert.NotEmpty(cm.Data.Compression) + assert.NotEmpty(cm.Data.Headers) + + for _, h := range cm.Data.Headers { + h := h + assert.NotEmpty(h) + assert.NotEmpty(h.Key) + assert.NotEmpty(h.Value) + } + + key := string(cm.Data.GetKey().GetNormalizedPayload()) + keys = append(keys, key) + + assert.NotEmpty(cm.Data.GetKey()) + assert.NotEmpty(cm.Data.GetKey().GetNormalizedPayload()) + assert.Empty(cm.Data.GetKey().GetOriginalPayload()) + assert.NotEmpty(cm.Data.GetKey().GetPayloadSize()) + assert.Equal(v1pb.PayloadEncoding_PAYLOAD_ENCODING_TEXT, cm.Data.GetKey().GetEncoding()) + assert.False(cm.Data.GetKey().GetIsPayloadTooLarge()) + assert.Empty(cm.Data.GetKey().GetTroubleshootReport()) + + assert.NotEmpty(cm.Data.GetValue()) + assert.Empty(cm.Data.GetValue().GetOriginalPayload()) + assert.NotEmpty(cm.Data.GetValue().GetPayloadSize()) + assert.Equal(v1pb.PayloadEncoding_PAYLOAD_ENCODING_JSON, cm.Data.GetValue().GetEncoding()) + assert.Empty(cm.Data.GetValue().GetTroubleshootReport()) + assert.False(cm.Data.GetValue().GetIsPayloadTooLarge()) + assert.NotEmpty(cm.Data.GetValue().GetNormalizedPayload()) + + if key == "too_big_0" { + // {"ID":"..."} + // large string of 21000 chars + 9 extra chars + assert.Len(cm.Data.GetValue().GetNormalizedPayload(), 21009) + } + + case *v1pb.ListMessagesResponse_Done: + doneCount++ + + assert.NotEmpty(cm.Done.GetBytesConsumed()) + assert.NotEmpty(cm.Done.GetMessagesConsumed()) + assert.NotEmpty(cm.Done.GetElapsedMs()) + assert.False(cm.Done.GetIsCancelled()) + case *v1pb.ListMessagesResponse_Phase: + if phaseCount == 0 { + assert.Equal("Get Partitions", cm.Phase.GetPhase()) + } else if phaseCount == 1 { + assert.Equal("Get Watermarks and calculate consuming requests", cm.Phase.GetPhase()) + } else if phaseCount == 2 { + assert.Equal("Consuming messages", cm.Phase.GetPhase()) + } else { + assert.Fail("Unknown phase.") + } + + phaseCount++ + case *v1pb.ListMessagesResponse_Progress: + progressCount++ + + assert.NotEmpty(cm.Progress) + assert.NotEmpty(cm.Progress.GetBytesConsumed()) + assert.NotEmpty(cm.Progress.GetMessagesConsumed()) + case *v1pb.ListMessagesResponse_Error: + errorCount++ + } + } + + assert.Nil(stream.Err()) + assert.Nil(stream.Close()) + assert.Equal([]string{"16", "17", "18", "19", "too_big_0"}, keys) + assert.Equal(3, phaseCount) + assert.Equal(0, errorCount) + assert.Equal(1, doneCount) + assert.False(seenZeroOffset) + assert.GreaterOrEqual(progressCount, 0) + }) } func (s *APIIntegrationTestSuite) TestPublishMessages() { @@ -590,3 +922,13 @@ func (s *APIIntegrationTestSuite) TestPublishMessages() { assert.Equal(timestamppb.New(objTime), obj2.CreatedAt) }) } + +const letterBytes = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ" + +func randomString(n int) string { + b := make([]byte, n) + for i := range b { + b[i] = letterBytes[rand.Intn(len(letterBytes))] + } + return string(b) +} diff --git a/backend/pkg/api/handle_topics_integration_test.go b/backend/pkg/api/handle_topics_integration_test.go index bfc4eae0e..bf4473133 100644 --- a/backend/pkg/api/handle_topics_integration_test.go +++ b/backend/pkg/api/handle_topics_integration_test.go @@ -217,6 +217,13 @@ func (s *APIIntegrationTestSuite) TestHandleGetTopics() { // switch s.api.ConsoleSvc = newConsoleSvc + // undo switch + defer func() { + if oldConsoleSvc != nil { + s.api.ConsoleSvc = oldConsoleSvc + } + }() + // call the fake control and expect function fakeCluster.Control(func(req kmsg.Request) (kmsg.Response, error, bool) { fakeCluster.KeepControl() @@ -248,13 +255,6 @@ func (s *APIIntegrationTestSuite) TestHandleGetTopics() { } }) - // undo switch - defer func() { - if oldConsoleSvc != nil { - s.api.ConsoleSvc = oldConsoleSvc - } - }() - // make the request ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) defer cancel() @@ -312,6 +312,13 @@ func (s *APIIntegrationTestSuite) TestHandleGetTopics() { // switch s.api.ConsoleSvc = newConsoleSvc + // undo switch + defer func() { + if oldConsoleSvc != nil { + s.api.ConsoleSvc = oldConsoleSvc + } + }() + // call the fake control and expect function fakeCluster.Control(func(req kmsg.Request) (kmsg.Response, error, bool) { fakeCluster.KeepControl() @@ -356,13 +363,6 @@ func (s *APIIntegrationTestSuite) TestHandleGetTopics() { } }) - // undo switch - defer func() { - if oldConsoleSvc != nil { - s.api.ConsoleSvc = oldConsoleSvc - } - }() - // make the request ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) defer cancel() diff --git a/backend/pkg/console/list_messages.go b/backend/pkg/console/list_messages.go index 967544bea..f3dc0b71f 100644 --- a/backend/pkg/console/list_messages.go +++ b/backend/pkg/console/list_messages.go @@ -48,6 +48,7 @@ type ListMessageRequest struct { FilterInterpreterCode string Troubleshoot bool IncludeRawPayload bool + IgnoreMaxSizeLimit bool KeyDeserializer serde.PayloadEncoding ValueDeserializer serde.PayloadEncoding } @@ -138,6 +139,7 @@ func (s *Service) ListMessages(ctx context.Context, listReq ListMessageRequest, FilterInterpreterCode: listReq.FilterInterpreterCode, Troubleshoot: listReq.Troubleshoot, IncludeRawPayload: listReq.IncludeRawPayload, + IgnoreMaxSizeLimit: listReq.IgnoreMaxSizeLimit, KeyDeserializer: listReq.KeyDeserializer, ValueDeserializer: listReq.ValueDeserializer, } diff --git a/backend/pkg/kafka/consumer.go b/backend/pkg/kafka/consumer.go index 8c2890c98..7b0e501b7 100644 --- a/backend/pkg/kafka/consumer.go +++ b/backend/pkg/kafka/consumer.go @@ -79,6 +79,7 @@ type TopicConsumeRequest struct { FilterInterpreterCode string Troubleshoot bool IncludeRawPayload bool + IgnoreMaxSizeLimit bool KeyDeserializer serde.PayloadEncoding ValueDeserializer serde.PayloadEncoding } diff --git a/backend/pkg/kafka/consumer_worker.go b/backend/pkg/kafka/consumer_worker.go index 8ea8fdeac..d445b110c 100644 --- a/backend/pkg/kafka/consumer_worker.go +++ b/backend/pkg/kafka/consumer_worker.go @@ -59,11 +59,12 @@ func (s *Service) startMessageWorker(ctx context.Context, wg *sync.WaitGroup, ctx, record, serde.DeserializationOptions{ - MaxPayloadSize: s.Config.Console.MaxDeserializationPayloadSize, - Troubleshoot: consumeReq.Troubleshoot, - IncludeRawData: consumeReq.IncludeRawPayload, - KeyEncoding: consumeReq.KeyDeserializer, - ValueEncoding: consumeReq.ValueDeserializer, + MaxPayloadSize: s.Config.Console.MaxDeserializationPayloadSize, + Troubleshoot: consumeReq.Troubleshoot, + IncludeRawData: consumeReq.IncludeRawPayload, + IgnoreMaxSizeLimit: consumeReq.IgnoreMaxSizeLimit, + KeyEncoding: consumeReq.KeyDeserializer, + ValueEncoding: consumeReq.ValueDeserializer, }) headersByKey := make(map[string][]byte, len(deserializedRec.Headers)) diff --git a/backend/pkg/protogen/redpanda/api/console/v1alpha1/list_messages.pb.go b/backend/pkg/protogen/redpanda/api/console/v1alpha1/list_messages.pb.go index 6e3316d6c..c5c353d74 100644 --- a/backend/pkg/protogen/redpanda/api/console/v1alpha1/list_messages.pb.go +++ b/backend/pkg/protogen/redpanda/api/console/v1alpha1/list_messages.pb.go @@ -39,6 +39,7 @@ type ListMessagesRequest struct { IncludeOriginalRawPayload bool `protobuf:"varint,9,opt,name=include_original_raw_payload,json=includeOriginalRawPayload,proto3" json:"include_original_raw_payload,omitempty"` // Optionally include original raw payload. KeyDeserializer *PayloadEncoding `protobuf:"varint,10,opt,name=key_deserializer,json=keyDeserializer,proto3,enum=redpanda.api.console.v1alpha1.PayloadEncoding,oneof" json:"key_deserializer,omitempty"` // Optionally specify key payload deserialization strategy to use. ValueDeserializer *PayloadEncoding `protobuf:"varint,11,opt,name=value_deserializer,json=valueDeserializer,proto3,enum=redpanda.api.console.v1alpha1.PayloadEncoding,oneof" json:"value_deserializer,omitempty"` // Optionally specify value payload deserialization strategy to use. + IgnoreMaxSizeLimit bool `protobuf:"varint,12,opt,name=ignore_max_size_limit,json=ignoreMaxSizeLimit,proto3" json:"ignore_max_size_limit,omitempty"` // Optionally ignore configured maximum payload size limit. } func (x *ListMessagesRequest) Reset() { @@ -150,6 +151,13 @@ func (x *ListMessagesRequest) GetValueDeserializer() PayloadEncoding { return PayloadEncoding_PAYLOAD_ENCODING_UNSPECIFIED } +func (x *ListMessagesRequest) GetIgnoreMaxSizeLimit() bool { + if x != nil { + return x.IgnoreMaxSizeLimit + } + return false +} + // ListMessagesResponse is the response for ListMessages call. type ListMessagesResponse struct { state protoimpl.MessageState @@ -712,7 +720,7 @@ var file_redpanda_api_console_v1alpha1_list_messages_proto_rawDesc = []byte{ 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x2a, 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x63, - 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x93, 0x05, 0x0a, 0x13, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xc6, 0x05, 0x0a, 0x13, 0x4c, 0x69, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x0a, 0xba, 0x48, 0x07, 0x72, 0x05, 0x10, 0x01, 0x18, 0x80, 0x01, 0x52, 0x05, @@ -751,141 +759,144 @@ var file_redpanda_api_console_v1alpha1_list_messages_proto_rawDesc = []byte{ 0x73, 0x6f, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x45, 0x6e, 0x63, 0x6f, 0x64, 0x69, 0x6e, 0x67, 0x48, 0x01, 0x52, 0x11, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x44, 0x65, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, - 0x65, 0x72, 0x88, 0x01, 0x01, 0x42, 0x13, 0x0a, 0x11, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x64, 0x65, - 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x72, 0x42, 0x15, 0x0a, 0x13, 0x5f, 0x76, - 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x64, 0x65, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, - 0x72, 0x22, 0xa1, 0x0a, 0x0a, 0x14, 0x4c, 0x69, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x55, 0x0a, 0x04, 0x64, 0x61, - 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x72, 0x65, 0x64, 0x70, 0x61, - 0x6e, 0x64, 0x61, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x2e, - 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x44, 0x61, - 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x64, 0x61, 0x74, - 0x61, 0x12, 0x58, 0x0a, 0x05, 0x70, 0x68, 0x61, 0x73, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x40, 0x2e, 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x65, 0x72, 0x88, 0x01, 0x01, 0x12, 0x31, 0x0a, 0x15, 0x69, 0x67, 0x6e, 0x6f, 0x72, 0x65, 0x5f, + 0x6d, 0x61, 0x78, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x0c, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x69, 0x67, 0x6e, 0x6f, 0x72, 0x65, 0x4d, 0x61, 0x78, 0x53, + 0x69, 0x7a, 0x65, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x42, 0x13, 0x0a, 0x11, 0x5f, 0x6b, 0x65, 0x79, + 0x5f, 0x64, 0x65, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x72, 0x42, 0x15, 0x0a, + 0x13, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x64, 0x65, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, + 0x69, 0x7a, 0x65, 0x72, 0x22, 0xa1, 0x0a, 0x0a, 0x14, 0x4c, 0x69, 0x73, 0x74, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x55, 0x0a, + 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x72, 0x65, + 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x6f, + 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x2e, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, + 0x64, 0x61, 0x74, 0x61, 0x12, 0x58, 0x0a, 0x05, 0x70, 0x68, 0x61, 0x73, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, + 0x68, 0x61, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x50, 0x68, 0x61, 0x73, 0x65, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x05, 0x70, 0x68, 0x61, 0x73, 0x65, 0x12, 0x61, + 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x43, 0x2e, 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x50, 0x68, 0x61, 0x73, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x48, 0x00, 0x52, 0x05, 0x70, 0x68, 0x61, 0x73, 0x65, 0x12, 0x61, 0x0a, 0x08, 0x70, - 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x43, 0x2e, - 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6e, - 0x73, 0x6f, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4c, 0x69, - 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x2e, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x60, - 0x0a, 0x04, 0x64, 0x6f, 0x6e, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4a, 0x2e, 0x72, - 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6e, 0x73, - 0x6f, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4c, 0x69, 0x73, - 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, - 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x64, 0x6f, 0x6e, 0x65, - 0x12, 0x58, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x40, 0x2e, 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, + 0x73, 0x12, 0x60, 0x0a, 0x04, 0x64, 0x6f, 0x6e, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x4a, 0x2e, 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x48, 0x00, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x1a, 0xbd, 0x03, 0x0a, 0x0b, 0x44, - 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, - 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x16, 0x0a, - 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6f, - 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, - 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, - 0x61, 0x6d, 0x70, 0x12, 0x50, 0x0a, 0x0b, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, - 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x72, 0x65, 0x64, 0x70, 0x61, - 0x6e, 0x64, 0x61, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x2e, - 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, - 0x73, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0b, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, - 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x29, 0x0a, 0x10, 0x69, 0x73, 0x5f, 0x74, 0x72, 0x61, 0x6e, - 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x0f, 0x69, 0x73, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, - 0x12, 0x4a, 0x0a, 0x07, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x30, 0x2e, 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x61, 0x70, 0x69, - 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, - 0x31, 0x2e, 0x4b, 0x61, 0x66, 0x6b, 0x61, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x48, 0x65, 0x61, - 0x64, 0x65, 0x72, 0x52, 0x07, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x12, 0x43, 0x0a, 0x03, - 0x6b, 0x65, 0x79, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x72, 0x65, 0x64, 0x70, - 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, - 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4b, 0x61, 0x66, 0x6b, 0x61, 0x52, - 0x65, 0x63, 0x6f, 0x72, 0x64, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x03, 0x6b, 0x65, - 0x79, 0x12, 0x47, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x31, 0x2e, 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x61, 0x70, 0x69, 0x2e, - 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, - 0x2e, 0x4b, 0x61, 0x66, 0x6b, 0x61, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x50, 0x61, 0x79, 0x6c, - 0x6f, 0x61, 0x64, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x1a, 0x24, 0x0a, 0x0c, 0x50, 0x68, - 0x61, 0x73, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x70, 0x68, - 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x70, 0x68, 0x61, 0x73, 0x65, - 0x1a, 0x65, 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x4d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x12, 0x2b, 0x0a, 0x11, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x5f, - 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x64, - 0x12, 0x25, 0x0a, 0x0e, 0x62, 0x79, 0x74, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, - 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x62, 0x79, 0x74, 0x65, 0x73, 0x43, - 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x64, 0x1a, 0xae, 0x01, 0x0a, 0x16, 0x53, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x65, 0x6c, 0x61, 0x70, 0x73, 0x65, 0x64, 0x5f, 0x6d, 0x73, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x65, 0x6c, 0x61, 0x70, 0x73, 0x65, 0x64, 0x4d, - 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x73, 0x5f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x6c, 0x65, - 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x69, 0x73, 0x43, 0x61, 0x6e, 0x63, 0x65, - 0x6c, 0x6c, 0x65, 0x64, 0x12, 0x2b, 0x0a, 0x11, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, - 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x10, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, - 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x62, 0x79, 0x74, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, - 0x6d, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x62, 0x79, 0x74, 0x65, 0x73, - 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x64, 0x1a, 0x28, 0x0a, 0x0c, 0x45, 0x72, 0x72, 0x6f, - 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x42, 0x11, 0x0a, 0x0f, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x5f, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xd8, 0x03, 0x0a, 0x12, 0x4b, 0x61, 0x66, 0x6b, 0x61, 0x52, - 0x65, 0x63, 0x6f, 0x72, 0x64, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x2e, 0x0a, 0x10, - 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x61, 0x6c, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x48, 0x00, 0x52, 0x0f, 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, - 0x61, 0x6c, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x88, 0x01, 0x01, 0x12, 0x32, 0x0a, 0x12, - 0x6e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, - 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x48, 0x01, 0x52, 0x11, 0x6e, 0x6f, 0x72, 0x6d, - 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x88, 0x01, 0x01, - 0x12, 0x4a, 0x0a, 0x08, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x61, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6d, 0x70, 0x6c, + 0x65, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x64, + 0x6f, 0x6e, 0x65, 0x12, 0x58, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, - 0x61, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x45, 0x6e, 0x63, 0x6f, 0x64, 0x69, - 0x6e, 0x67, 0x52, 0x08, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x69, 0x6e, 0x67, 0x12, 0x20, 0x0a, 0x09, - 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x48, - 0x02, 0x52, 0x08, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x49, 0x64, 0x88, 0x01, 0x01, 0x12, 0x21, - 0x0a, 0x0c, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x69, 0x7a, - 0x65, 0x12, 0x2f, 0x0a, 0x14, 0x69, 0x73, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x5f, - 0x74, 0x6f, 0x6f, 0x5f, 0x6c, 0x61, 0x72, 0x67, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x11, 0x69, 0x73, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x54, 0x6f, 0x6f, 0x4c, 0x61, 0x72, - 0x67, 0x65, 0x12, 0x62, 0x0a, 0x13, 0x74, 0x72, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x73, 0x68, 0x6f, - 0x6f, 0x74, 0x5f, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x31, 0x2e, 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, - 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, - 0x54, 0x72, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x73, 0x68, 0x6f, 0x6f, 0x74, 0x52, 0x65, 0x70, 0x6f, - 0x72, 0x74, 0x52, 0x12, 0x74, 0x72, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x73, 0x68, 0x6f, 0x6f, 0x74, - 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x42, 0x13, 0x0a, 0x11, 0x5f, 0x6f, 0x72, 0x69, 0x67, 0x69, - 0x6e, 0x61, 0x6c, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x42, 0x15, 0x0a, 0x13, 0x5f, - 0x6e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, - 0x61, 0x64, 0x42, 0x0c, 0x0a, 0x0a, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x69, 0x64, - 0x42, 0xb2, 0x02, 0x0a, 0x21, 0x63, 0x6f, 0x6d, 0x2e, 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, - 0x61, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x2e, 0x76, 0x31, - 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x42, 0x11, 0x4c, 0x69, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x63, 0x67, 0x69, 0x74, - 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, - 0x2d, 0x64, 0x61, 0x74, 0x61, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x2f, 0x62, 0x61, - 0x63, 0x6b, 0x65, 0x6e, 0x64, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x67, - 0x65, 0x6e, 0x2f, 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2f, 0x61, 0x70, 0x69, 0x2f, - 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, - 0x3b, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, - 0xa2, 0x02, 0x03, 0x52, 0x41, 0x43, 0xaa, 0x02, 0x1d, 0x52, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, - 0x61, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x43, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x2e, 0x56, 0x31, - 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02, 0x1d, 0x52, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, - 0x61, 0x5c, 0x41, 0x70, 0x69, 0x5c, 0x43, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x5c, 0x56, 0x31, - 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0xe2, 0x02, 0x29, 0x52, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, - 0x61, 0x5c, 0x41, 0x70, 0x69, 0x5c, 0x43, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x5c, 0x56, 0x31, - 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0xea, 0x02, 0x20, 0x52, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x3a, 0x3a, 0x41, - 0x70, 0x69, 0x3a, 0x3a, 0x43, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x3a, 0x3a, 0x56, 0x31, 0x61, - 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x61, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x1a, 0xbd, 0x03, + 0x0a, 0x0b, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x21, 0x0a, + 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, + 0x12, 0x16, 0x0a, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x50, 0x0a, 0x0b, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, + 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x72, 0x65, + 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x6f, + 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, + 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0b, 0x63, 0x6f, 0x6d, + 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x29, 0x0a, 0x10, 0x69, 0x73, 0x5f, 0x74, + 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0f, 0x69, 0x73, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x61, 0x6c, 0x12, 0x4a, 0x0a, 0x07, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x18, 0x06, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x61, 0x6c, + 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4b, 0x61, 0x66, 0x6b, 0x61, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, + 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x07, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x12, + 0x43, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x72, + 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6e, 0x73, + 0x6f, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4b, 0x61, 0x66, + 0x6b, 0x61, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x12, 0x47, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x08, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, + 0x68, 0x61, 0x31, 0x2e, 0x4b, 0x61, 0x66, 0x6b, 0x61, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x50, + 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x1a, 0x24, 0x0a, + 0x0c, 0x50, 0x68, 0x61, 0x73, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x14, 0x0a, + 0x05, 0x70, 0x68, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x70, 0x68, + 0x61, 0x73, 0x65, 0x1a, 0x65, 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x2b, 0x0a, 0x11, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x10, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x73, 0x75, + 0x6d, 0x65, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x62, 0x79, 0x74, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x6e, + 0x73, 0x75, 0x6d, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x62, 0x79, 0x74, + 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x64, 0x1a, 0xae, 0x01, 0x0a, 0x16, 0x53, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x65, 0x6c, 0x61, 0x70, 0x73, 0x65, 0x64, + 0x5f, 0x6d, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x65, 0x6c, 0x61, 0x70, 0x73, + 0x65, 0x64, 0x4d, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x73, 0x5f, 0x63, 0x61, 0x6e, 0x63, 0x65, + 0x6c, 0x6c, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x69, 0x73, 0x43, 0x61, + 0x6e, 0x63, 0x65, 0x6c, 0x6c, 0x65, 0x64, 0x12, 0x2b, 0x0a, 0x11, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x10, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x62, 0x79, 0x74, 0x65, 0x73, 0x5f, 0x63, 0x6f, + 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x62, 0x79, + 0x74, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x64, 0x1a, 0x28, 0x0a, 0x0c, 0x45, + 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x11, 0x0a, 0x0f, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, + 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xd8, 0x03, 0x0a, 0x12, 0x4b, 0x61, 0x66, + 0x6b, 0x61, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, + 0x2e, 0x0a, 0x10, 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x61, 0x6c, 0x5f, 0x70, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x48, 0x00, 0x52, 0x0f, 0x6f, 0x72, 0x69, + 0x67, 0x69, 0x6e, 0x61, 0x6c, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x88, 0x01, 0x01, 0x12, + 0x32, 0x0a, 0x12, 0x6e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x70, 0x61, + 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x48, 0x01, 0x52, 0x11, 0x6e, + 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, + 0x88, 0x01, 0x01, 0x12, 0x4a, 0x0a, 0x08, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x69, 0x6e, 0x67, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x61, + 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x45, 0x6e, 0x63, + 0x6f, 0x64, 0x69, 0x6e, 0x67, 0x52, 0x08, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x69, 0x6e, 0x67, 0x12, + 0x20, 0x0a, 0x09, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x05, 0x48, 0x02, 0x52, 0x08, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x49, 0x64, 0x88, 0x01, + 0x01, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x73, 0x69, 0x7a, + 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, + 0x53, 0x69, 0x7a, 0x65, 0x12, 0x2f, 0x0a, 0x14, 0x69, 0x73, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, + 0x61, 0x64, 0x5f, 0x74, 0x6f, 0x6f, 0x5f, 0x6c, 0x61, 0x72, 0x67, 0x65, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x11, 0x69, 0x73, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x54, 0x6f, 0x6f, + 0x4c, 0x61, 0x72, 0x67, 0x65, 0x12, 0x62, 0x0a, 0x13, 0x74, 0x72, 0x6f, 0x75, 0x62, 0x6c, 0x65, + 0x73, 0x68, 0x6f, 0x6f, 0x74, 0x5f, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x07, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, + 0x61, 0x31, 0x2e, 0x54, 0x72, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x73, 0x68, 0x6f, 0x6f, 0x74, 0x52, + 0x65, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x12, 0x74, 0x72, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x73, 0x68, + 0x6f, 0x6f, 0x74, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x42, 0x13, 0x0a, 0x11, 0x5f, 0x6f, 0x72, + 0x69, 0x67, 0x69, 0x6e, 0x61, 0x6c, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x42, 0x15, + 0x0a, 0x13, 0x5f, 0x6e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x70, 0x61, + 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x42, 0x0c, 0x0a, 0x0a, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x5f, 0x69, 0x64, 0x42, 0xb2, 0x02, 0x0a, 0x21, 0x63, 0x6f, 0x6d, 0x2e, 0x72, 0x65, 0x64, 0x70, + 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, + 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x42, 0x11, 0x4c, 0x69, 0x73, 0x74, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x63, + 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x72, 0x65, 0x64, 0x70, 0x61, + 0x6e, 0x64, 0x61, 0x2d, 0x64, 0x61, 0x74, 0x61, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, + 0x2f, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x6e, 0x64, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x67, 0x65, 0x6e, 0x2f, 0x72, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x2f, 0x61, + 0x70, 0x69, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, + 0x68, 0x61, 0x31, 0x3b, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x76, 0x31, 0x61, 0x6c, 0x70, + 0x68, 0x61, 0x31, 0xa2, 0x02, 0x03, 0x52, 0x41, 0x43, 0xaa, 0x02, 0x1d, 0x52, 0x65, 0x64, 0x70, + 0x61, 0x6e, 0x64, 0x61, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x43, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, + 0x2e, 0x56, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02, 0x1d, 0x52, 0x65, 0x64, 0x70, + 0x61, 0x6e, 0x64, 0x61, 0x5c, 0x41, 0x70, 0x69, 0x5c, 0x43, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, + 0x5c, 0x56, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0xe2, 0x02, 0x29, 0x52, 0x65, 0x64, 0x70, + 0x61, 0x6e, 0x64, 0x61, 0x5c, 0x41, 0x70, 0x69, 0x5c, 0x43, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, + 0x5c, 0x56, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x20, 0x52, 0x65, 0x64, 0x70, 0x61, 0x6e, 0x64, 0x61, + 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x43, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x65, 0x3a, 0x3a, + 0x56, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( diff --git a/backend/pkg/serde/service.go b/backend/pkg/serde/service.go index 02003d99f..e09caa860 100644 --- a/backend/pkg/serde/service.go +++ b/backend/pkg/serde/service.go @@ -120,7 +120,7 @@ func (s *Service) deserializePayload(ctx context.Context, record *kgo.Record, pa rp.OriginalPayload = payload } - if len(payload) > opts.MaxPayloadSize { + if !opts.IgnoreMaxSizeLimit && len(payload) > opts.MaxPayloadSize { rp.IsPayloadTooLarge = true rp.NormalizedPayload = nil } @@ -160,6 +160,9 @@ type DeserializationOptions struct { // IncludeRawData can be enabled to include raw binary data in the returned output. IncludeRawData bool + + // IgnoreMaxSizeLimit can be used to force returning deserialized payloads even if too large. + IgnoreMaxSizeLimit bool } // SerializeRecord will serialize the input. diff --git a/backend/pkg/tools/openapi-generator/main.go b/backend/pkg/tools/openapi-generator/main.go index 98a919d5d..6cea44e7a 100644 --- a/backend/pkg/tools/openapi-generator/main.go +++ b/backend/pkg/tools/openapi-generator/main.go @@ -68,8 +68,11 @@ func main() { //nolint:cyclop,gocognit // this is just some tool // Delete DummyService tag for i, tag := range doc3.Tags { - if tag.Name == "DummyService" { - doc3.Tags = slices.Delete(doc3.Tags, i, i+1) + tag := tag + if tag != nil { + if tag.Name == "DummyService" { + doc3.Tags = slices.Delete(doc3.Tags, i, i+1) + } } } diff --git a/frontend/src/protogen/redpanda/api/console/v1alpha1/list_messages_pb.ts b/frontend/src/protogen/redpanda/api/console/v1alpha1/list_messages_pb.ts index af8cb9401..ead7ff005 100644 --- a/frontend/src/protogen/redpanda/api/console/v1alpha1/list_messages_pb.ts +++ b/frontend/src/protogen/redpanda/api/console/v1alpha1/list_messages_pb.ts @@ -90,6 +90,13 @@ export class ListMessagesRequest extends Message { */ valueDeserializer?: PayloadEncoding; + /** + * Optionally ignore configured maximum payload size limit. + * + * @generated from field: bool ignore_max_size_limit = 12; + */ + ignoreMaxSizeLimit = false; + constructor(data?: PartialMessage) { super(); proto3.util.initPartial(data, this); @@ -109,6 +116,7 @@ export class ListMessagesRequest extends Message { { no: 9, name: "include_original_raw_payload", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, { no: 10, name: "key_deserializer", kind: "enum", T: proto3.getEnumType(PayloadEncoding), opt: true }, { no: 11, name: "value_deserializer", kind: "enum", T: proto3.getEnumType(PayloadEncoding), opt: true }, + { no: 12, name: "ignore_max_size_limit", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, ]); static fromBinary(bytes: Uint8Array, options?: Partial): ListMessagesRequest { diff --git a/proto/redpanda/api/console/v1alpha1/list_messages.proto b/proto/redpanda/api/console/v1alpha1/list_messages.proto index 848f642b6..8f46d8a22 100644 --- a/proto/redpanda/api/console/v1alpha1/list_messages.proto +++ b/proto/redpanda/api/console/v1alpha1/list_messages.proto @@ -32,6 +32,9 @@ message ListMessagesRequest { optional PayloadEncoding key_deserializer = 10; // Optionally specify key payload deserialization strategy to use. optional PayloadEncoding value_deserializer = 11; // Optionally specify value payload deserialization strategy to use. + + bool ignore_max_size_limit = 12; // Optionally ignore configured maximum payload size limit. + // Used to force returning deserialized payloads. } // ListMessagesResponse is the response for ListMessages call. diff --git a/taskfiles/backend.yaml b/taskfiles/backend.yaml index 68fc8a226..12346d298 100644 --- a/taskfiles/backend.yaml +++ b/taskfiles/backend.yaml @@ -157,7 +157,7 @@ tasks: - generate cmds: - | - {{ .PATH_PREFIX }} go test -v -race --tags=integration -p=1 ./... + {{ .PATH_PREFIX }} go test -v -race --tags=integration -p=1 {{.CLI_ARGS | default "./..." }} cover: dir: "{{.BACKEND_ROOT}}" @@ -168,7 +168,7 @@ tasks: cmds: - task: 'lint' - | - {{ .PATH_PREFIX }} go test -race --tags=integration -json -cover ./... | {{ .BUILD_ROOT }}/bin/go/tparse + {{ .PATH_PREFIX }} go test -race --tags=integration -json -cover {{.CLI_ARGS | default "./..." }} | {{ .BUILD_ROOT }}/bin/go/tparse verify: dir: "{{.BACKEND_ROOT}}"